[tsv] migrate tsvv3 to separate module, separate tsv hal, tsv phylo

parent c8185ce7
Pipeline #7985 passed with stages
in 58 minutes and 39 seconds
...@@ -20,7 +20,8 @@ import Data.Tuple.Extra (both) ...@@ -20,7 +20,8 @@ import Data.Tuple.Extra (both)
import Data.Vector qualified as DV import Data.Vector qualified as DV
import GHC.Generics import GHC.Generics
import Gargantext.Core.Text.Context (TermList) import Gargantext.Core.Text.Context (TermList)
import Gargantext.Core.Text.Corpus.Parsers.TSV (readTSVFile, tsv_title, tsv_abstract, tsv_publication_year, fromMIntOrDec, defaultYear) import Gargantext.Core.Text.Corpus.Parsers.TSV (readTSVFile, fromMIntOrDec, defaultYear)
import Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 (tsv_title, tsv_abstract, tsv_publication_year)
import Gargantext.Core.Text.List.Formats.TSV (tsvMapTermList) import Gargantext.Core.Text.List.Formats.TSV (tsvMapTermList)
import Gargantext.Core.Text.Metrics.Count (coocOnContexts, Coocs) import Gargantext.Core.Text.Metrics.Count (coocOnContexts, Coocs)
import Gargantext.Core.Text.Terms.WithList ( Patterns, buildPatterns, extractTermsWithList ) import Gargantext.Core.Text.Terms.WithList ( Patterns, buildPatterns, extractTermsWithList )
......
...@@ -14,7 +14,8 @@ import Gargantext.API.Ngrams.Prelude (toTermList) ...@@ -14,7 +14,8 @@ import Gargantext.API.Ngrams.Prelude (toTermList)
import Gargantext.API.Ngrams.Types import Gargantext.API.Ngrams.Types
import Gargantext.Core.Text.Context (TermList) import Gargantext.Core.Text.Context (TermList)
import Gargantext.Core.Text.Corpus.Parsers (FileFormat(..), FileType(..), parseFile) import Gargantext.Core.Text.Corpus.Parsers (FileFormat(..), FileType(..), parseFile)
import Gargantext.Core.Text.Corpus.Parsers.TSV (tsv_title, tsv_abstract, tsv_publication_year, tsv_publication_month, tsv_publication_day, tsv'_source, tsv'_title, tsv'_abstract, tsv'_publication_year, tsv'_publication_month, tsv'_publication_day, tsv'_weight) import Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 (tsv_title, tsv_abstract, tsv_publication_year, tsv_publication_month, tsv_publication_day)
import Gargantext.Core.Text.Corpus.Parsers.TSV.TsvPhylo (tsv'_source, tsv'_title, tsv'_abstract, tsv'_publication_year, tsv'_publication_month, tsv'_publication_day, tsv'_weight)
import Gargantext.Core.Text.Corpus.Parsers.TSV qualified as Tsv import Gargantext.Core.Text.Corpus.Parsers.TSV qualified as Tsv
import Gargantext.Core.Text.List.Formats.TSV (tsvMapTermList) import Gargantext.Core.Text.List.Formats.TSV (tsvMapTermList)
import Gargantext.Core.Text.Ngrams (NgramsType(..)) import Gargantext.Core.Text.Ngrams (NgramsType(..))
...@@ -95,15 +96,17 @@ tsvToDocs parser patterns time path = ...@@ -95,15 +96,17 @@ tsvToDocs parser patterns time path =
, hour = Def.hour , hour = Def.hour
, minute = Def.minute , minute = Def.minute
, sec = Def.second } , sec = Def.second }
Tsv' limit -> Vector.toList Tsv' limit -> do
<$> Vector.take limit (Right contents) <- Tsv.readTSVFileDelim path Tsv.Tab
<$> Vector.map (\row -> Document { date = toPhyloDate (rowToUTCTimeR row) time pure $ Vector.toList
$ Vector.take limit
(Vector.map (\row -> Document { date = toPhyloDate (rowToUTCTimeR row) time
, date' = toPhyloDate' (rowToUTCTimeR row) time , date' = toPhyloDate' (rowToUTCTimeR row) time
, text = termsInText patterns $ (tsv'_title row) <> " " <> (tsv'_abstract row) , text = termsInText patterns $ tsv'_title row <> " " <> tsv'_abstract row
, weight = Just $ tsv'_weight row , weight = Just $ tsv'_weight row
, sources = map (T.strip . pack) $ splitOn ";" (unpack $ (tsv'_source row)) , sources = map (T.strip . pack) $ splitOn ";" (unpack $ tsv'_source row)
, docTime = time } , docTime = time }
) <$> snd <$> Tsv.readWeightedTsv path ) $ snd contents)
where where
rowToUTCTimeR row = UTCTimeR { year = tsv'_publication_year row rowToUTCTimeR row = UTCTimeR { year = tsv'_publication_year row
, month = tsv'_publication_month row , month = tsv'_publication_month row
......
...@@ -233,6 +233,10 @@ library ...@@ -233,6 +233,10 @@ library
Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec
Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3
Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv4 Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv4
Gargantext.Core.Text.Corpus.Parsers.TSV.TsvHal
Gargantext.Core.Text.Corpus.Parsers.TSV.TsvPhylo
Gargantext.Core.Text.Corpus.Parsers.TSV.Types
Gargantext.Core.Text.Corpus.Parsers.TSV.Utils
Gargantext.Core.Text.Corpus.Query Gargantext.Core.Text.Corpus.Query
Gargantext.Core.Text.List Gargantext.Core.Text.List
Gargantext.Core.Text.List.Formats.TSV Gargantext.Core.Text.List.Formats.TSV
......
...@@ -22,11 +22,12 @@ import Data.Text qualified as Text ...@@ -22,11 +22,12 @@ import Data.Text qualified as Text
import Gargantext.Core (Lang(..)) import Gargantext.Core (Lang(..))
import Gargantext.Core.Text.Corpus.Parsers (cleanText) import Gargantext.Core.Text.Corpus.Parsers (cleanText)
import Gargantext.Core.Text.Corpus.Parsers.TSV (writeDocs2Tsv) import Gargantext.Core.Text.Corpus.Parsers.TSV (writeDocs2Tsv)
import Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 qualified as TSVv3
import Gargantext.Core.Text.Corpus.Parsers.Date qualified as Date import Gargantext.Core.Text.Corpus.Parsers.Date qualified as Date
import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) ) import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) )
import Gargantext.Defaults qualified as Defaults import Gargantext.Defaults qualified as Defaults
import Gargantext.Prelude hiding (get) import Gargantext.Prelude hiding (get)
import Isidore qualified as Isidore import Isidore qualified
import Isidore.Client import Isidore.Client
import Servant.Client ( ClientError(DecodeFailure) ) import Servant.Client ( ClientError(DecodeFailure) )
...@@ -47,15 +48,14 @@ get lang l q a = do ...@@ -47,15 +48,14 @@ get lang l q a = do
iDocs <- either printErr _content <$> Isidore.get l q a iDocs <- either printErr _content <$> Isidore.get l q a
hDocs <- mapM (isidoreToDoc lang) (toIsidoreDocs iDocs) mapM (isidoreToDoc lang) (toIsidoreDocs iDocs)
pure hDocs
isidore2tsvFile :: FilePath -> Lang -> Maybe Isidore.Limit isidore2tsvFile :: FilePath -> Lang -> Maybe Isidore.Limit
-> Maybe Isidore.TextQuery -> Maybe Isidore.AuthorQuery -> Maybe Isidore.TextQuery -> Maybe Isidore.AuthorQuery
-> IO () -> IO ()
isidore2tsvFile fp lang li tq aq = do isidore2tsvFile fp lang li tq aq = do
hdocs <- get lang li tq aq hdocs <- get lang li tq aq
writeDocs2Tsv fp hdocs writeDocs2Tsv TSVv3.headerTsvGargV3 TSVv3.doc2tsv fp hdocs
isidoreToDoc :: Lang -> IsidoreDoc -> IO HyperdataDocument isidoreToDoc :: Lang -> IsidoreDoc -> IO HyperdataDocument
isidoreToDoc lang (IsidoreDoc t a d u s as) = do isidoreToDoc lang (IsidoreDoc t a d u s as) = do
...@@ -73,7 +73,7 @@ isidoreToDoc lang (IsidoreDoc t a d u s as) = do ...@@ -73,7 +73,7 @@ isidoreToDoc lang (IsidoreDoc t a d u s as) = do
langText (OnlyText t2 ) = t2 langText (OnlyText t2 ) = t2
langText (ArrayText ts ) = Text.unwords $ map langText ts langText (ArrayText ts ) = Text.unwords $ map langText ts
let mDateS = maybe (Just $ Text.pack $ show Defaults.year) Just d let mDateS = d <|> Just (Text.pack $ show Defaults.year)
let (utcTime, (pub_year, pub_month, pub_day)) = Date.mDateSplit mDateS let (utcTime, (pub_year, pub_month, pub_day)) = Date.mDateSplit mDateS
pure HyperdataDocument pure HyperdataDocument
......
...@@ -54,6 +54,8 @@ import Gargantext.Core.Text.Corpus.Parsers.JSON (parseJSONC, parseIstex) ...@@ -54,6 +54,8 @@ import Gargantext.Core.Text.Corpus.Parsers.JSON (parseJSONC, parseIstex)
import Gargantext.Core.Text.Corpus.Parsers.RIS qualified as RIS import Gargantext.Core.Text.Corpus.Parsers.RIS qualified as RIS
import Gargantext.Core.Text.Corpus.Parsers.RIS.Presse (presseEnrich) import Gargantext.Core.Text.Corpus.Parsers.RIS.Presse (presseEnrich)
import Gargantext.Core.Text.Corpus.Parsers.TSV (parseHal, parseTsv, parseTsvC) import Gargantext.Core.Text.Corpus.Parsers.TSV (parseHal, parseTsv, parseTsvC)
import Gargantext.Core.Text.Corpus.Parsers.TSV.TsvHal qualified as TsvHal
import Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 qualified as TSVv3
import Gargantext.Core.Text.Corpus.Parsers.Types import Gargantext.Core.Text.Corpus.Parsers.Types
import Gargantext.Core.Text.Corpus.Parsers.WOS qualified as WOS import Gargantext.Core.Text.Corpus.Parsers.WOS qualified as WOS
import Gargantext.Core.Text.Ngrams (NgramsType(..)) import Gargantext.Core.Text.Ngrams (NgramsType(..))
...@@ -111,10 +113,10 @@ parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0 ...@@ -111,10 +113,10 @@ parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0
-> DB.ByteString -> DB.ByteString
-> m (Either DT.Text (Integer, ConduitT () (ParseCorpusResult HyperdataDocument) IO ())) -> m (Either DT.Text (Integer, ConduitT () (ParseCorpusResult HyperdataDocument) IO ()))
do_parse TsvGargV3 Plain bs = do do_parse TsvGargV3 Plain bs = do
let eParsedC = parseTsvC $ DBL.fromStrict bs let eParsedC = parseTsvC TSVv3.tsv2doc $ DBL.fromStrict bs
pure (second (transPipe (pure . runIdentity)) <$> eParsedC) pure (second (transPipe (pure . runIdentity)) <$> eParsedC)
do_parse TsvHal Plain bs = do do_parse TsvHal Plain bs = do
let eParsedC = parseTsvC $ DBL.fromStrict bs let eParsedC = parseTsvC TsvHal.tsv2doc $ DBL.fromStrict bs
pure (second (transPipe (pure . runIdentity)) <$> eParsedC) pure (second (transPipe (pure . runIdentity)) <$> eParsedC)
do_parse Istex Plain bs = do do_parse Istex Plain bs = do
ep <- liftBase $ parseIstex EN $ DBL.fromStrict bs ep <- liftBase $ parseIstex EN $ DBL.fromStrict bs
...@@ -221,7 +223,7 @@ parseFile :: FileType ...@@ -221,7 +223,7 @@ parseFile :: FileType
-> FileFormat -> FileFormat
-> FilePath -> FilePath
-> IO (Either Text [HyperdataDocument]) -> IO (Either Text [HyperdataDocument])
parseFile TsvGargV3 Plain p = parseTsv p parseFile TsvGargV3 Plain p = parseTsv TSVv3.tsv2doc p
parseFile TsvHal Plain p = parseHal p parseFile TsvHal Plain p = parseHal p
parseFile RisPresse Plain p = do parseFile RisPresse Plain p = do
docs <- join $ mapM (toDoc RIS) <$> snd <$> enrichWith RisPresse <$> readFileWith RIS p docs <- join $ mapM (toDoc RIS) <$> snd <$> enrichWith RisPresse <$> readFileWith RIS p
......
{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE TypeApplications #-}
{-# LANGUAGE ViewPatterns #-}
{-| {-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV Module : Gargantext.Core.Text.Corpus.Parsers.TSV
Description : Description :
...@@ -18,12 +14,13 @@ TSV parser for Gargantext corpus files. ...@@ -18,12 +14,13 @@ TSV parser for Gargantext corpus files.
module Gargantext.Core.Text.Corpus.Parsers.TSV ( module Gargantext.Core.Text.Corpus.Parsers.TSV (
readTSVFile readTSVFile
, readTSVFileDelim
, Delimiter(..) , Delimiter(..)
, parseHal
, parseTsv , parseTsv
, parseTsvC , parseTsvC
, tsvDecodeOptions , tsvDecodeOptions
, writeDocs2Tsv , writeDocs2Tsv
, parseHal
, readTsvHal , readTsvHal
, TsvHal(..) , TsvHal(..)
-- * Used in tests -- * Used in tests
...@@ -31,477 +28,20 @@ module Gargantext.Core.Text.Corpus.Parsers.TSV ( ...@@ -31,477 +28,20 @@ module Gargantext.Core.Text.Corpus.Parsers.TSV (
, testErrorPerLine , testErrorPerLine
, validTextField , validTextField
, validNumber , validNumber
, Tsv'(..)
, TsvDoc(..)
, getHeaders , getHeaders
, findDelimiter , findDelimiter
, delimiter , delimiter
-- * Used in the main executables -- * Used in the main executables
, readWeightedTsv
, defaultDay , defaultDay
, defaultYear , defaultYear
, fromMIntOrDec , fromMIntOrDec
, defaultMonth , defaultMonth
) where ) where
import Conduit ( ConduitT, (.|), yieldMany, mapC ) import Gargantext.Core.Text.Corpus.Parsers.TSV.Diagnostics
import Control.Lens hiding ((.=)) import Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec
import Data.ByteString (StrictByteString) import Gargantext.Core.Text.Corpus.Parsers.TSV.TsvHal
import Data.ByteString.Lazy qualified as BL import Gargantext.Core.Text.Corpus.Parsers.TSV.Types
import Data.ByteString.Lazy.Char8 qualified as B8L import Gargantext.Core.Text.Corpus.Parsers.TSV.Utils
import Data.Csv
import Data.Csv.Incremental qualified as CSVI
import Data.Text (pack)
import Data.Text qualified as T
import Data.Text.Lazy qualified as TL
import Data.Text.Lazy.Encoding qualified as TL
import Data.Text.Read qualified as DTR
import Data.Time.Segment (jour)
import Data.Vector (Vector)
import Data.Vector qualified as V
import Gargantext.Core.Text.Corpus.Parsers.Types
import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) )
import Gargantext.Prelude hiding (length, show)
import Prelude (String)
import Protolude
------------------------------------------------------------------------
testValue :: BL.ByteString -> Text -> Int -> Either Text Bool
testValue val columnHeader ligne = case columnHeader of
"Publication Day" -> validNumber val columnHeader ligne
"Publication Month" -> validNumber val columnHeader ligne
"Publication Year" -> validNumber val columnHeader ligne
"Authors" -> validTextField val columnHeader ligne
"Title" -> validTextField val columnHeader ligne
"Source" -> validTextField val columnHeader ligne
"Abstract" -> validTextField val columnHeader ligne
"Publication Date" -> validTextField val columnHeaer ligne
_ -> Right True
testErrorPerLine :: [BL.ByteString] -> Delimiter -> [Text] -> Int -> Either Text Bool
testErrorPerLine [] _ [] _ = Right True
testErrorPerLine _ del [] l | del == Comma = Left (pack $ "Too many fields at line " <> show l <> ". Try using tabulation as a delimiter. Other delimiter like comma (,) may appear in some text.")
| otherwise = Left (pack $ "Too many fields at line " <> show l)
testErrorPerLine [] _ _ l = Left (pack $ "Missing one field at line " <> show l)
testErrorPerLine (v:val) del (h:headers) ligne =
case testValue v h ligne of
Left _err -> Left _err
Right _ -> testErrorPerLine val del headers ligne
checkNextLine :: Vector BL.ByteString -> Delimiter -> [Text] -> BL.ByteString -> Int -> Either Text (Int,[BL.ByteString])
checkNextLine bl del headers res x = do
case BL.splitWith (==delimiter del) <$> ((V.!?) bl (x+1)) of
Nothing -> Right (x, (BL.splitWith (==delimiter del) res))
Just value -> if length value > 1
then Right (x, (BL.splitWith (==delimiter del) res))
else case BL.append res <$> ((V.!?) bl (x+1)) of
Nothing -> Left "checkNextLine2"
Just val -> checkNextLine bl del headers val (x+1)
getMultipleLinefile :: Vector BL.ByteString -> Delimiter -> [Text] -> BL.ByteString -> Int -> Either Text (Int,[BL.ByteString])
getMultipleLinefile bl del headers res x = do
let tmp = BL.splitWith (==delimiter del) res in
if length tmp == length headers
then checkNextLine bl del headers res x
else
if (length tmp > length headers) || (V.length bl == (x + 1))
then
Left (pack $ "Cannot parse the file at line " <> show x <> ". Maybe because of a delimiter")
else do
case BL.append res <$> ((V.!?) bl (x+1)) of
Nothing -> Left "getMultipleLinefile"
Just val -> getMultipleLinefile bl del headers val (x+1)
anx :: Vector BL.ByteString -> Delimiter -> [Text] -> Int -> Either Text Delimiter
anx bl del headers x
| length bl == x = Right del
| otherwise =
case (V.!?) bl x of
Nothing -> Left "anx"
Just bs ->
case getMultipleLinefile bl del headers bs x of
Left _err -> Left _err
Right (y, val) -> case testErrorPerLine val del headers (x + 1) of
Left _err -> Left _err
Right _ -> anx bl del headers (y+1)
testIfErrorInFile :: [BL.ByteString] -> Delimiter -> [Text] -> Either Text Delimiter
testIfErrorInFile bl del headers = anx (V.fromList bl) del headers 1
testCorrectFile :: BL.ByteString -> Either Text Delimiter
testCorrectFile bs =
case findDelimiter bs of
Left _err -> Left _err
Right del -> do
let bl = BL.splitWith (==delimiter Line) bs in
case getHeaders bl del of
Left _err -> Left _err
Right headers -> testIfErrorInFile bl del headers
-- Detects delimiter based on the first line
detectDelimiter :: BL.ByteString -> Either Text Delimiter
detectDelimiter input =
case B8L.lines input of
(firstLine : _) ->
let candidates = [(',', count ',' firstLine), ('\t', count '\t' firstLine)]
in case fst $ maximumBy (comparing snd) candidates of
'\n' -> Right Line
'\t' -> Right Tab
',' -> Right Comma
_ -> Left $ "Invalid delimiter detected for input tsv document."
_ -> Left "Couldn't detect a valid delimiter for the input document."
-- Count occurrences of a character in a ByteString
count :: Char -> BL.ByteString -> Int64
count c = BL.count (fromIntegral (fromEnum c))
----------Test headers added to ggt
-- use a map to remove \r that sometimes appear at the end of a line
testAllHeadersPresence :: [Text] -> Either Text [Text]
testAllHeadersPresence headers = do
let listHeaders = filter (`notElem` (map (T.replace (T.pack "\r") (T.pack "")) headers)) ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"]
if null listHeaders
then Right headers
else Left ((pack " Missing column : ") <> T.intercalate ", " listHeaders)
getHeaders :: [BL.ByteString] -> Delimiter -> Either Text [Text]
getHeaders bl del = do
let vec = V.fromList bl in
case BL.splitWith (==delimiter del) <$> ((V.!?) vec 0) of
Nothing -> Left "Error getHeaders"
Just headers -> testAllHeadersPresence (map (\x -> T.replace (T.pack "\"") (T.pack "") (lBLToText x)) headers)
------------------------------------------------------------------------
-- | TODO use readFileLazy
readTSVFile :: FilePath -> IO (Either Text (Header, Vector TsvDoc))
readTSVFile fp = do
file <- BL.readFile fp
case detectDelimiter file of
Left err -> pure $ Left err
Right del -> pure $ readTsvLazyBS del file
type TsvDocParser = CSVI.Parser (ParseCorpusResult TsvDoc)
-- | Parses the input 'ByteString' identifying a TSV document
-- using the provided delimiter. This function is /incremental/, i.e.
-- it will correctly identify rows which can't be parsed and skip them,
-- collecting the error along the way.
-- It returns a list of 'ParseCorpusResult', which will contain either
-- the parsed record or a diagnostic on the error on the given row.
parseTsvWithDiagnostics :: Delimiter
-> BL.ByteString
-> [ParseCorpusResult TsvDoc]
parseTsvWithDiagnostics d bs =
case drainParser (BL.foldrChunks go mkHdrParser bs) of
MkHeaderParsingContext _p ->
[ ParseTsvRecordFailed (AtRow 0 ("The parsing choked on the header (delimiter was " <> show d <> "). This might be a malformed TSV we can't recover from.", mempty)) ]
MkRecordParsingContext (RecordParsingContext{..})
-- As we accumulate records by consing (for efficiency sake),
-- we need a final 'reverse' at the end.
-> reverse $ _prs_ctx_parsed_records
where
drainParser :: ParsingContext (ParseCorpusResult TsvDoc)
-> ParsingContext (ParseCorpusResult TsvDoc)
drainParser = go mempty . go mempty -- step twice, once to apply the empty string, once to drain.
mkHdrParser :: ParsingContext (ParseCorpusResult TsvDoc)
mkHdrParser = MkHeaderParsingContext (CSVI.decodeByNameWith (tsvDecodeOptions d))
go :: StrictByteString
-> ParsingContext (ParseCorpusResult TsvDoc)
-> ParsingContext (ParseCorpusResult TsvDoc)
go strict_chunk ctx = case ctx of
MkHeaderParsingContext p -> go_hdr strict_chunk p
MkRecordParsingContext p -> go_rec strict_chunk p
-- Specialised parser for the header: once it fails
-- or completes, it morphs into a record parser.
go_hdr :: StrictByteString
-> CSVI.HeaderParser TsvDocParser
-> ParsingContext (ParseCorpusResult TsvDoc)
go_hdr chunk p = case p of
CSVI.FailH unconsumed err ->
MkRecordParsingContext $ stepParser chunk $ toRecordParsingCtx (CSVI.Fail unconsumed err)
CSVI.PartialH continue ->
MkHeaderParsingContext (continue chunk)
CSVI.DoneH _header rec_parser ->
-- Turn this into a record parser by feeding the unconsumed plus the input.
MkRecordParsingContext $ stepParser chunk $ toRecordParsingCtx rec_parser
-- Specialised parser for the individual records.
go_rec :: StrictByteString
-> RecordParsingContext (ParseCorpusResult TsvDoc)
-> ParsingContext (ParseCorpusResult TsvDoc)
go_rec input_bs ctx = MkRecordParsingContext $ stepParser input_bs ctx
-- Main workhorse: given an input 'StrictByteString', it steps
-- the parser. If the parser reached a terminal state, it gets
-- removed from the input 'RecordParsingContext'.
stepParser :: StrictByteString
-> RecordParsingContext (ParseCorpusResult TsvDoc)
-> RecordParsingContext (ParseCorpusResult TsvDoc)
stepParser input_bs ctx = case ctx ^. prs_ctx_parser of
Nothing
-> ctx
Just (CSVI.Fail unconsumed err)
-> ctx & over prs_ctx_parsed_records (mkErr ctx (T.pack err, unconsumed) :)
& prs_ctx_parser .~ Nothing
Just (CSVI.Many parsed continue)
-> (addParsedRecords parsed ctx) & prs_ctx_parser .~ (Just $ continue input_bs)
Just (CSVI.Done parsed)
-> (addParsedRecords parsed ctx) & prs_ctx_parser .~ Nothing
-- Convert a parser inside a 'ParsingContext' from a header parser into a record parser.
toRecordParsingCtx :: TsvDocParser
-> RecordParsingContext (ParseCorpusResult TsvDoc)
toRecordParsingCtx p =
RecordParsingContext
{ _prs_ctx_parser = Just p
, _prs_ctx_parsed_records = []
, _prs_ctx_row_cursor = 1
}
mkErr :: RecordParsingContext (ParseCorpusResult TsvDoc) -> (Text, ByteString) -> ParseCorpusResult TsvDoc
mkErr ctx pair = ParseTsvRecordFailed (AtRow (ctx ^. prs_ctx_row_cursor) pair)
addParsedRecords :: [Either String (ParseCorpusResult TsvDoc)]
-> RecordParsingContext (ParseCorpusResult TsvDoc)
-> RecordParsingContext (ParseCorpusResult TsvDoc)
addParsedRecords recs ctx = foldl' process_record ctx recs
process_record :: RecordParsingContext (ParseCorpusResult TsvDoc)
-> Either String (ParseCorpusResult TsvDoc)
-> RecordParsingContext (ParseCorpusResult TsvDoc)
process_record ctx result =
let ctx' = ctx & over prs_ctx_row_cursor succ
in case result of
Left err -> ctx' & over prs_ctx_parsed_records (mkErr ctx' (T.pack err, mempty) :)
Right rec -> ctx' & over prs_ctx_parsed_records (rec :)
-- | TODO use readByteStringLazy
readTsvLazyBS :: Delimiter
-> BL.ByteString
-> Either Text (Header, Vector TsvDoc)
readTsvLazyBS d bs = first pack $ decodeByNameWith (tsvDecodeOptions d) bs
------------------------------------------------------------------------
-- | TODO use readFileLazy
readTsvHal :: FilePath -> IO (Either Text (Header, Vector TsvHal))
readTsvHal fp = do
c <- BL.readFile fp
pure $ readTsvHalLazyBS c
-- | TODO use readByteStringLazy
readTsvHalLazyBS :: BL.ByteString -> Either Text (Header, Vector TsvHal)
readTsvHalLazyBS bs = first pack $ decodeByNameWith (tsvDecodeOptions Tab) bs
writeDocs2Tsv :: FilePath -> [HyperdataDocument] -> IO ()
writeDocs2Tsv fp hs = BL.writeFile fp $ hyperdataDocument2tsv hs
hyperdataDocument2tsv :: [HyperdataDocument] -> BL.ByteString
hyperdataDocument2tsv hs = encodeByNameWith (tsvEncodeOptions Tab) headerTsvGargV3 (map hyperdataDocument2tsvDoc hs)
------------------------------------------------------------------------
-- Hal Format
data TsvHal = TsvHal
{ tsvHal_title :: !Text
, tsvHal_source :: !Text
, tsvHal_publication_year :: !Integer
, tsvHal_publication_month :: !Int
, tsvHal_publication_day :: !Int
, tsvHal_abstract :: !Text
, tsvHal_authors :: !Text
, tsvHal_url :: !Text
, tsvHal_isbn_s :: !Text
, tsvHal_issue_s :: !Text
, tsvHal_journalPublisher_s:: !Text
, tsvHal_language_s :: !Text
, tsvHal_doiId_s :: !Text
, tsvHal_authId_i :: !Text
, tsvHal_instStructId_i :: !Text
, tsvHal_deptStructId_i :: !Text
, tsvHal_labStructId_i :: !Text
, tsvHal_rteamStructId_i :: !Text
, tsvHal_docType_s :: !Text
}
deriving (Show)
instance FromNamedRecord TsvHal where
parseNamedRecord r = do
tsvHal_title <- r .: "title"
tsvHal_source <- r .: "source"
tsvHal_publication_year <- r .: "publication_year"
tsvHal_publication_month <- r .: "publication_month"
tsvHal_publication_day <- r .: "publication_day"
tsvHal_abstract <- r .: "abstract"
tsvHal_authors <- r .: "authors"
tsvHal_url <- r .: "url"
tsvHal_isbn_s <- r .: "isbn_s"
tsvHal_issue_s <- r .: "issue_s"
tsvHal_journalPublisher_s <- r .: "journalPublisher_s"
tsvHal_language_s <- r .: "language_s"
tsvHal_doiId_s <- r .: "doiId_s"
tsvHal_authId_i <- r .: "authId_i"
tsvHal_instStructId_i <- r .: "instStructId_i"
tsvHal_deptStructId_i <- r .: "deptStructId_i"
tsvHal_labStructId_i <- r .: "labStructId_i"
tsvHal_rteamStructId_i <- r .: "rteamStructId_i"
tsvHal_docType_s <- r .: "docType_s"
pure $ TsvHal { .. }
instance ToNamedRecord TsvHal where
--toNamedRecord (TsvHal t s py pm pd abst aut url isbn iss j lang doi auth inst dept lab team doct) =
toNamedRecord (TsvHal { .. }) =
namedRecord [ "title" .= tsvHal_title
, "source" .= tsvHal_source
, "publication_year" .= tsvHal_publication_year
, "publication_month" .= tsvHal_publication_month
, "publication_day" .= tsvHal_publication_day
, "abstract" .= tsvHal_abstract
, "authors" .= tsvHal_authors
, "url" .= tsvHal_url
, "isbn_s" .= tsvHal_isbn_s
, "issue_s" .= tsvHal_issue_s
, "journalPublisher_s" .= tsvHal_journalPublisher_s
, "language_s" .= tsvHal_language_s
, "doiId_s" .= tsvHal_doiId_s
, "authId_i" .= tsvHal_authId_i
, "instStructId_i" .= tsvHal_instStructId_i
, "deptStructId_i" .= tsvHal_deptStructId_i
, "labStructId_i" .= tsvHal_labStructId_i
, "rteamStructId_i" .= tsvHal_rteamStructId_i
, "docType_s" .= tsvHal_docType_s
]
tsvHal2doc :: TsvHal -> HyperdataDocument
tsvHal2doc (TsvHal { .. }) =
HyperdataDocument { _hd_bdd = Just "TsvHal"
, _hd_doi = Just tsvHal_doiId_s
, _hd_url = Just tsvHal_url
, _hd_page = Nothing
, _hd_title = Just tsvHal_title
, _hd_authors = Just tsvHal_authors
, _hd_institutes = Just tsvHal_instStructId_i
, _hd_source = Just tsvHal_source
, _hd_abstract = Just tsvHal_abstract
, _hd_publication_date = Just $ pack . show $ jour tsvHal_publication_year
tsvHal_publication_month
tsvHal_publication_day
, _hd_publication_year = Just $ fromIntegral tsvHal_publication_year
, _hd_publication_month = Just tsvHal_publication_month
, _hd_publication_day = Just tsvHal_publication_day
, _hd_publication_hour = Nothing
, _hd_publication_minute = Nothing
, _hd_publication_second = Nothing
, _hd_language_iso2 = Nothing
, _hd_institutes_tree = Nothing }
tsv2doc :: TsvDoc -> HyperdataDocument
tsv2doc (TsvDoc { .. })
= HyperdataDocument { _hd_bdd = Just "TsvHal"
, _hd_doi = Nothing
, _hd_url = Nothing
, _hd_page = Nothing
, _hd_title = Just tsv_title
, _hd_authors = Just tsv_authors
, _hd_institutes = Nothing
, _hd_source = Just tsv_source
, _hd_abstract = Just tsv_abstract
, _hd_publication_date = Just $ pack . show $ jour (fromIntegral pubYear)
pubMonth
pubDay
, _hd_publication_year = Just pubYear
, _hd_publication_month = Just pubMonth
, _hd_publication_day = Just pubDay
, _hd_publication_hour = Nothing
, _hd_publication_minute = Nothing
, _hd_publication_second = Nothing
, _hd_language_iso2 = Nothing
, _hd_institutes_tree = Nothing }
where
pubYear = fromMIntOrDec defaultYear tsv_publication_year
pubMonth = fromMaybe defaultMonth tsv_publication_month
pubDay = fromMaybe defaultDay tsv_publication_day
------------------------------------------------------------------------
parseHal :: FilePath -> IO (Either Text [HyperdataDocument])
parseHal fp = do
r <- readTsvHal fp
pure $ V.toList . V.map tsvHal2doc . snd <$> r
------------------------------------------------------------------------
parseTsv :: FilePath -> IO (Either Text [HyperdataDocument])
parseTsv fp = fmap (V.toList . V.map tsv2doc . snd) <$> readTSVFile fp
parseTsvC :: BL.ByteString
-> Either Text (Integer, ConduitT () (ParseCorpusResult HyperdataDocument) Identity ())
parseTsvC bs = convert_result <$> eResult
where
eResult :: Either Text [ParseCorpusResult TsvDoc]
eResult = case detectDelimiter bs of
Left err -> Left err
Right del -> Right $ parseTsvWithDiagnostics del bs
convert_result :: [ParseCorpusResult TsvDoc]
-> (Integer, ConduitT () (ParseCorpusResult HyperdataDocument) Identity ())
convert_result rs =
(fromIntegral $ length rs, yieldMany rs .| mapC tsvResult2doc)
tsvResult2doc :: ParseCorpusResult TsvDoc -> ParseCorpusResult HyperdataDocument
tsvResult2doc = \case
ParseRecordSucceeded r -> ParseRecordSucceeded (tsv2doc r)
ParseTsvRecordFailed err -> ParseTsvRecordFailed err
------------------------------------------------------------------------
-- Tsv v3 weighted for phylo
data Tsv' = Tsv'
{ tsv'_title :: !Text
, tsv'_source :: !Text
, tsv'_publication_year :: !Int
, tsv'_publication_month :: !Int
, tsv'_publication_day :: !Int
, tsv'_abstract :: !Text
, tsv'_authors :: !Text
, tsv'_weight :: !Double } deriving (Show)
instance FromNamedRecord Tsv' where
parseNamedRecord r = do
tsv'_title <- r .: "title"
tsv'_source <- r .: "source"
tsv'_publication_year <- r .: "publication_year"
tsv'_publication_month <- r .: "publication_month"
tsv'_publication_day <- r .: "publication_day"
tsv'_abstract <- r .: "abstract"
tsv'_authors <- r .: "authors"
tsv'_weight <- r .: "weight"
pure $ Tsv' { .. }
readWeightedTsv :: FilePath -> IO (Header, Vector Tsv')
readWeightedTsv fp =
fmap (\bs ->
case decodeByNameWith (tsvDecodeOptions Tab) bs of
Left e -> panicTrace (pack e)
Right corpus -> corpus
) $ BL.readFile fp
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV.Diagnostics
Description :
Copyright : (c) CNRS, 2017-Present
License : AGPL + CECILL v3
Maintainer : team@gargantext.org
Stability : experimental
Portability : POSIX
-}
module Gargantext.Core.Text.Corpus.Parsers.TSV.Diagnostics where module Gargantext.Core.Text.Corpus.Parsers.TSV.Diagnostics where
import Conduit ( ConduitT, (.|), yieldMany, mapC )
import Control.Lens (over)
import Data.ByteString (StrictByteString)
import Data.ByteString.Lazy qualified as BL
import Data.Csv import Data.Csv
import Data.Csv.Incremental qualified as CSVI
import Data.Text (pack)
import Data.Text qualified as T
import Data.Text.Lazy qualified as TL
import Data.Text.Lazy.Encoding qualified as TL
import Data.Text.Read qualified as DTR
import Data.Vector (Vector)
import Data.Vector qualified as V
import Gargantext.Core.Text.Corpus.Parsers.Types (AtRow(..), ParseCorpusResult(..))
import Gargantext.Core.Text.Corpus.Parsers.TSV.Types import Gargantext.Core.Text.Corpus.Parsers.TSV.Types
import Gargantext.Core.Text.Corpus.Parsers.TSV.Utils (detectDelimiter)
import Gargantext.Prelude import Gargantext.Prelude
import Prelude (String)
type ColumnName = Text
type FieldValue = BL.ByteString
type Validator = FieldValue -> ColumnName -> Int -> Either Text Bool
type ColumnValidator = (ColumnName, Validator)
------------------------------------------------------------------------
parseTsvC :: forall tsvDoc result. FromNamedRecord tsvDoc
=> (tsvDoc -> result)
-> BL.ByteString
-> Either Text (Integer, ConduitT () (ParseCorpusResult result) Identity ())
parseTsvC tsv2doc bs = convert_result <$> eResult
where
eResult :: Either Text [ParseCorpusResult tsvDoc]
eResult = case detectDelimiter bs of
Left err -> Left err
Right del -> Right $ parseTsvWithDiagnostics del bs
convert_result :: [ParseCorpusResult tsvDoc]
-> (Integer, ConduitT () (ParseCorpusResult result) Identity ())
convert_result rs =
(fromIntegral $ length rs, yieldMany rs .| mapC tsvResult2doc)
tsvResult2doc :: ParseCorpusResult tsvDoc -> ParseCorpusResult result
tsvResult2doc = \case
ParseRecordSucceeded r -> ParseRecordSucceeded (tsv2doc r)
ParseTsvRecordFailed err -> ParseTsvRecordFailed err
-- | Parses the input 'ByteString' identifying a TSV document
-- using the provided delimiter. This function is /incremental/, i.e.
-- it will correctly identify rows which can't be parsed and skip them,
-- collecting the error along the way.
-- It returns a list of 'ParseCorpusResult', which will contain either
-- the parsed record or a diagnostic on the error on the given row.
parseTsvWithDiagnostics :: forall tsvDoc. FromNamedRecord tsvDoc
=> Delimiter
-> BL.ByteString
-> [ParseCorpusResult tsvDoc]
parseTsvWithDiagnostics d bs =
case drainParser (BL.foldrChunks go mkHdrParser bs) of
MkHeaderParsingContext _p ->
[ ParseTsvRecordFailed (AtRow 0 ("The parsing choked on the header (delimiter was " <> show d <> "). This might be a malformed TSV we can't recover from.", mempty)) ]
MkRecordParsingContext (RecordParsingContext{..})
-- As we accumulate records by consing (for efficiency sake),
-- we need a final 'reverse' at the end.
-> reverse $ _prs_ctx_parsed_records
where
drainParser :: ParsingContext (ParseCorpusResult tsvDoc)
-> ParsingContext (ParseCorpusResult tsvDoc)
drainParser = go mempty . go mempty -- step twice, once to apply the empty string, once to drain.
mkHdrParser :: ParsingContext (ParseCorpusResult tsvDoc)
mkHdrParser = MkHeaderParsingContext (CSVI.decodeByNameWith (tsvDecodeOptions d))
go :: StrictByteString
-> ParsingContext (ParseCorpusResult tsvDoc)
-> ParsingContext (ParseCorpusResult tsvDoc)
go strict_chunk ctx = case ctx of
MkHeaderParsingContext p -> go_hdr strict_chunk p
MkRecordParsingContext p -> go_rec strict_chunk p
-- Specialised parser for the header: once it fails
-- or completes, it morphs into a record parser.
go_hdr :: StrictByteString
-> CSVI.HeaderParser (CSVI.Parser (ParseCorpusResult tsvDoc))
-> ParsingContext (ParseCorpusResult tsvDoc)
go_hdr chunk p = case p of
CSVI.FailH unconsumed err ->
MkRecordParsingContext $ stepParser chunk $ toRecordParsingCtx (CSVI.Fail unconsumed err)
CSVI.PartialH continue ->
MkHeaderParsingContext (continue chunk)
CSVI.DoneH _header rec_parser ->
-- Turn this into a record parser by feeding the unconsumed plus the input.
MkRecordParsingContext $ stepParser chunk $ toRecordParsingCtx rec_parser
-- Specialised parser for the individual records.
go_rec :: StrictByteString
-> RecordParsingContext (ParseCorpusResult tsvDoc)
-> ParsingContext (ParseCorpusResult tsvDoc)
go_rec input_bs ctx = MkRecordParsingContext $ stepParser input_bs ctx
-- Main workhorse: given an input 'StrictByteString', it steps
-- the parser. If the parser reached a terminal state, it gets
-- removed from the input 'RecordParsingContext'.
stepParser :: StrictByteString
-> RecordParsingContext (ParseCorpusResult tsvDoc)
-> RecordParsingContext (ParseCorpusResult tsvDoc)
stepParser input_bs ctx = case ctx ^. prs_ctx_parser of
Nothing
-> ctx
Just (CSVI.Fail unconsumed err)
-> ctx & over prs_ctx_parsed_records (mkErr ctx (T.pack err, unconsumed) :)
& prs_ctx_parser .~ Nothing
Just (CSVI.Many parsed continue)
-> (addParsedRecords parsed ctx) & prs_ctx_parser .~ (Just $ continue input_bs)
Just (CSVI.Done parsed)
-> (addParsedRecords parsed ctx) & prs_ctx_parser .~ Nothing
-- Convert a parser inside a 'ParsingContext' from a header parser into a record parser.
toRecordParsingCtx :: CSVI.Parser (ParseCorpusResult tsvDoc)
-> RecordParsingContext (ParseCorpusResult tsvDoc)
toRecordParsingCtx p =
RecordParsingContext
{ _prs_ctx_parser = Just p
, _prs_ctx_parsed_records = []
, _prs_ctx_row_cursor = 1
}
mkErr :: RecordParsingContext (ParseCorpusResult tsvDoc) -> (Text, ByteString) -> ParseCorpusResult tsvDoc
mkErr ctx pair = ParseTsvRecordFailed (AtRow (ctx ^. prs_ctx_row_cursor) pair)
addParsedRecords :: [Either String (ParseCorpusResult tsvDoc)]
-> RecordParsingContext (ParseCorpusResult tsvDoc)
-> RecordParsingContext (ParseCorpusResult tsvDoc)
addParsedRecords recs ctx = foldl' process_record ctx recs
process_record :: RecordParsingContext (ParseCorpusResult tsvDoc)
-> Either String (ParseCorpusResult tsvDoc)
-> RecordParsingContext (ParseCorpusResult tsvDoc)
process_record ctx result =
let ctx' = ctx & over prs_ctx_row_cursor succ
in case result of
Left err -> ctx' & over prs_ctx_parsed_records (mkErr ctx' (T.pack err, mempty) :)
Right rec -> ctx' & over prs_ctx_parsed_records (rec :)
testCorrectFile :: [ColumnValidator] -> BL.ByteString -> Either Text Delimiter
testCorrectFile columnValidators bs =
case findDelimiter bs of
Left _err -> Left _err
Right del -> do
let bl = BL.splitWith (==delimiter Line) bs in
case getHeaders columnValidators bl del of
Left _err -> Left _err
Right headers -> testIfErrorInFile columnValidators bl del headers
----------Test headers added to ggt
-- use a map to remove \r that sometimes appear at the end of a line
testAllHeadersPresence :: [ColumnValidator] -> [Text] -> Either Text [Text]
testAllHeadersPresence columnValidators headers = do
let expectedHeaders = fst <$> columnValidators
let listHeaders = filter (`notElem` (map (T.replace (T.pack "\r") (T.pack "")) headers)) expectedHeaders
if null listHeaders
then Right headers
else Left ((pack " Missing column : ") <> T.intercalate ", " listHeaders)
getHeaders :: [ColumnValidator] -> [BL.ByteString] -> Delimiter -> Either Text [Text]
getHeaders columnValidators bl del = do
let vec = V.fromList bl in
case BL.splitWith (==delimiter del) <$> ((V.!?) vec 0) of
Nothing -> Left "Error getHeaders"
Just headers -> testAllHeadersPresence columnValidators (map (\x -> T.replace (T.pack "\"") (T.pack "") (lBLToText x)) headers)
testValue :: [ColumnValidator] -> FieldValue -> ColumnName -> Int -> Either Text Bool
testValue [] _val _columnHeader _lineno = Right True
testValue ((fieldName, validator):vs) val columnHeader lineno =
if fieldName == columnHeader then
validator val columnHeader lineno
else
testValue vs val columnHeader lineno
testErrorPerLine :: [ColumnValidator] -> [BL.ByteString] -> Delimiter -> [Text] -> Int -> Either Text Bool
testErrorPerLine _ [] _ [] _ = Right True
testErrorPerLine _ _ del [] l | del == Comma = Left (pack $ "Too many fields at line " <> show l <> ". Try using tabulation as a delimiter. Other delimiter like comma (,) may appear in some text.")
| otherwise = Left (pack $ "Too many fields at line " <> show l)
testErrorPerLine _ [] _ _ l = Left (pack $ "Missing one field at line " <> show l)
testErrorPerLine columnValidators (v:val) del (h:headers) lineno =
case testValue columnValidators v h lineno of
Left _err -> Left _err
Right _ -> testErrorPerLine columnValidators val del headers lineno
checkNextLine :: Vector BL.ByteString -> Delimiter -> [Text] -> BL.ByteString -> Int -> Either Text (Int,[BL.ByteString])
checkNextLine bl del headers res x = do
case BL.splitWith (==delimiter del) <$> ((V.!?) bl (x+1)) of
Nothing -> Right (x, (BL.splitWith (==delimiter del) res))
Just value -> if length value > 1
then Right (x, (BL.splitWith (==delimiter del) res))
else case BL.append res <$> ((V.!?) bl (x+1)) of
Nothing -> Left "checkNextLine2"
Just val -> checkNextLine bl del headers val (x+1)
getMultipleLinefile :: Vector BL.ByteString -> Delimiter -> [Text] -> BL.ByteString -> Int -> Either Text (Int,[BL.ByteString])
getMultipleLinefile bl del headers res x = do
let tmp = BL.splitWith (==delimiter del) res in
if length tmp == length headers
then checkNextLine bl del headers res x
else
if (length tmp > length headers) || (V.length bl == (x + 1))
then
Left (pack $ "Cannot parse the file at line " <> show x <> ". Maybe because of a delimiter")
else do
case BL.append res <$> ((V.!?) bl (x+1)) of
Nothing -> Left "getMultipleLinefile"
Just val -> getMultipleLinefile bl del headers val (x+1)
anx :: [ColumnValidator] -> Vector BL.ByteString -> Delimiter -> [Text] -> Int -> Either Text Delimiter
anx columnValidators bl del headers x
| length bl == x = Right del
| otherwise =
case (V.!?) bl x of
Nothing -> Left "anx"
Just bs ->
case getMultipleLinefile bl del headers bs x of
Left _err -> Left _err
Right (y, val) -> case testErrorPerLine columnValidators val del headers (x + 1) of
Left _err -> Left _err
Right _ -> anx columnValidators bl del headers (y+1)
testIfErrorInFile :: [ColumnValidator] -> [BL.ByteString] -> Delimiter -> [Text] -> Either Text Delimiter
testIfErrorInFile columnValidators bl del headers = anx columnValidators (V.fromList bl) del headers 1
testDelimiter :: Delimiter -> BL.ByteString -> Bool testDelimiter :: Delimiter -> BL.ByteString -> Bool
...@@ -34,23 +286,23 @@ lBLToText :: BL.ByteString -> Text ...@@ -34,23 +286,23 @@ lBLToText :: BL.ByteString -> Text
lBLToText b = TL.toStrict $ TL.decodeUtf8 b lBLToText b = TL.toStrict $ TL.decodeUtf8 b
validNumber :: BL.ByteString -> Text -> Int -> Either Text Bool validNumber :: BL.ByteString -> Text -> Int -> Either Text Bool
validNumber x columnHeader ligne = do validNumber x columnHeader lineno = do
let number = T.replace (T.pack "\"") (T.pack "") (lBLToText x) let number = T.replace (T.pack "\"") (T.pack "") (lBLToText x)
case isNumeric number of case isNumeric number of
Right val Right val
| val < 0 -> Left $ ("Value of column '" <> columnHeader <> "' at line " <> pack (show ligne) <> " is negative") | val < 0 -> Left $ ("Value of column '" <> columnHeader <> "' at line " <> pack (show lineno) <> " is negative")
|otherwise -> Right True |otherwise -> Right True
Left _ -> Left $ ("Error in column '" <> columnHeader <> "' at line " <> pack (show ligne) <> " : value is not a number ") Left _ -> Left $ ("Error in column '" <> columnHeader <> "' at line " <> pack (show lineno) <> " : value is not a number ")
validTextField :: BL.ByteString -> Text -> Int -> Either Text Bool validTextField :: BL.ByteString -> Text -> Int -> Either Text Bool
validTextField x columnHeader ligne = do validTextField x columnHeader lineno = do
let xs = T.replace (T.pack "\"\"") (T.pack "") (lBLToText x) in let xs = T.replace (T.pack "\"\"") (T.pack "") (lBLToText x) in
if not (T.null xs) if not (T.null xs)
then then
if (T.length xs > 0) && ((T.length (T.filter (== '\"') xs) == 0) || ((T.head xs == '"') && (T.last xs == '"') && (T.length (T.filter (== '\"') xs) == 2))) if (T.length xs > 0) && ((T.length (T.filter (== '\"') xs) == 0) || ((T.head xs == '"') && (T.last xs == '"') && (T.length (T.filter (== '\"') xs) == 2)))
then return True then return True
else Left $ ("Encapsulation problem at line " <> pack (show ligne) <> " in column '" <> columnHeader <> "' : the caracter \" must only appear at the beginning and the end of a field ") else Left $ ("Encapsulation problem at line " <> pack (show lineno) <> " in column '" <> columnHeader <> "' : the caracter \" must only appear at the beginning and the end of a field ")
else return True else return True
-- else Left $ ("The column '" <> columnHeader <> "' at line " <> pack (show ligne) <> " is empty") -- else Left $ ("The column '" <> columnHeader <> "' at line " <> pack (show ligne) <> " is empty")
-- Put a warning for the user to know their is a problem (empty column) -- Put a warning for the user to know their is a problem (empty column)
{-# LANGUAGE BangPatterns #-} {-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3
Description :
Copyright : (c) CNRS, 2017-Present
License : AGPL + CECILL v3
Maintainer : team@gargantext.org
Stability : experimental
Portability : POSIX
-}
module Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 where module Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 where
import Data.Csv import Data.Csv
import Data.Text (pack)
import Data.Time.Segment (jour)
import Gargantext.Core.Text.Corpus.Parsers.TSV.Diagnostics
import Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec import Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec
import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) ) import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) )
import Gargantext.Prelude import Gargantext.Prelude
...@@ -45,6 +57,34 @@ data TsvDoc = TsvDoc ...@@ -45,6 +57,34 @@ data TsvDoc = TsvDoc
deriving (Show) deriving (Show)
tsv2doc :: TsvDoc -> HyperdataDocument
tsv2doc (TsvDoc { .. })
= HyperdataDocument { _hd_bdd = Just "TsvHal"
, _hd_doi = Nothing
, _hd_url = Nothing
, _hd_page = Nothing
, _hd_title = Just tsv_title
, _hd_authors = Just tsv_authors
, _hd_institutes = Nothing
, _hd_source = Just tsv_source
, _hd_abstract = Just tsv_abstract
, _hd_publication_date = Just $ pack . show $ jour (fromIntegral pubYear)
pubMonth
pubDay
, _hd_publication_year = Just pubYear
, _hd_publication_month = Just pubMonth
, _hd_publication_day = Just pubDay
, _hd_publication_hour = Nothing
, _hd_publication_minute = Nothing
, _hd_publication_second = Nothing
, _hd_language_iso2 = Nothing
, _hd_institutes_tree = Nothing }
where
pubYear = fromMIntOrDec defaultYear tsv_publication_year
pubMonth = fromMaybe defaultMonth tsv_publication_month
pubDay = fromMaybe defaultDay tsv_publication_day
instance FromNamedRecord TsvDoc where instance FromNamedRecord TsvDoc where
parseNamedRecord r = do parseNamedRecord r = do
...@@ -69,8 +109,8 @@ instance ToNamedRecord TsvDoc where ...@@ -69,8 +109,8 @@ instance ToNamedRecord TsvDoc where
] ]
hyperdataDocument2tsvDoc :: HyperdataDocument -> TsvDoc doc2tsv :: HyperdataDocument -> TsvDoc
hyperdataDocument2tsvDoc h = TsvDoc { tsv_title = m $ _hd_title h doc2tsv h = TsvDoc { tsv_title = m $ _hd_title h
, tsv_source = m $ _hd_source h , tsv_source = m $ _hd_source h
, tsv_publication_year = Just $ IntOrDec $ mI $ _hd_publication_year h , tsv_publication_year = Just $ IntOrDec $ mI $ _hd_publication_year h
, tsv_publication_month = Just $ mI $ _hd_publication_month h , tsv_publication_month = Just $ mI $ _hd_publication_month h
...@@ -81,3 +121,17 @@ hyperdataDocument2tsvDoc h = TsvDoc { tsv_title = m $ _hd_title h ...@@ -81,3 +121,17 @@ hyperdataDocument2tsvDoc h = TsvDoc { tsv_title = m $ _hd_title h
where where
m = maybe "" identity m = maybe "" identity
mI = maybe 0 identity mI = maybe 0 identity
validators :: [ColumnValidator]
validators =
[ ("Publication Day", validNumber)
, ("Publication Month", validNumber)
, ("Publication Year", validNumber)
, ("Authors", validTextField)
, ("Title", validTextField)
, ("Source", validTextField)
, ("Abstract", validTextField)
-- TODO validDateField
-- , ("Publication Date", validTextField)
]
{-# LANGUAGE BangPatterns #-} {-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv4
Description :
Copyright : (c) CNRS, 2017-Present
License : AGPL + CECILL v3
Maintainer : team@gargantext.org
Stability : experimental
Portability : POSIX
-}
module Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv4 where module Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv4 where
import Data.Csv import Data.Csv
import Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec -- import Gargantext.Core.Text.Corpus.Parsers.TSV.IntOrDec
import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) ) import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) )
import Gargantext.Prelude import Gargantext.Prelude
......
{-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV.TsvHal
Description :
Copyright : (c) CNRS, 2017-Present
License : AGPL + CECILL v3
Maintainer : team@gargantext.org
Stability : experimental
Portability : POSIX
-}
module Gargantext.Core.Text.Corpus.Parsers.TSV.TsvHal where
import Data.ByteString.Lazy qualified as BL
import Data.Csv
import Data.Text (pack)
import Data.Time.Segment (jour)
import Data.Vector (Vector)
import Data.Vector qualified as V
import Gargantext.Core.Text.Corpus.Parsers.TSV.Types (Delimiter(..))
import Gargantext.Core.Text.Corpus.Parsers.TSV.Utils (readTsvLazyBS)
import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) )
import Gargantext.Prelude
------------------------------------------------------------------------
-- Hal Format
data TsvHal = TsvHal
{ tsvHal_title :: !Text
, tsvHal_source :: !Text
, tsvHal_publication_year :: !Integer
, tsvHal_publication_month :: !Int
, tsvHal_publication_day :: !Int
, tsvHal_abstract :: !Text
, tsvHal_authors :: !Text
, tsvHal_url :: !Text
, tsvHal_isbn_s :: !Text
, tsvHal_issue_s :: !Text
, tsvHal_journalPublisher_s:: !Text
, tsvHal_language_s :: !Text
, tsvHal_doiId_s :: !Text
, tsvHal_authId_i :: !Text
, tsvHal_instStructId_i :: !Text
, tsvHal_deptStructId_i :: !Text
, tsvHal_labStructId_i :: !Text
, tsvHal_rteamStructId_i :: !Text
, tsvHal_docType_s :: !Text
}
deriving (Show)
instance FromNamedRecord TsvHal where
parseNamedRecord r = do
tsvHal_title <- r .: "title"
tsvHal_source <- r .: "source"
tsvHal_publication_year <- r .: "publication_year"
tsvHal_publication_month <- r .: "publication_month"
tsvHal_publication_day <- r .: "publication_day"
tsvHal_abstract <- r .: "abstract"
tsvHal_authors <- r .: "authors"
tsvHal_url <- r .: "url"
tsvHal_isbn_s <- r .: "isbn_s"
tsvHal_issue_s <- r .: "issue_s"
tsvHal_journalPublisher_s <- r .: "journalPublisher_s"
tsvHal_language_s <- r .: "language_s"
tsvHal_doiId_s <- r .: "doiId_s"
tsvHal_authId_i <- r .: "authId_i"
tsvHal_instStructId_i <- r .: "instStructId_i"
tsvHal_deptStructId_i <- r .: "deptStructId_i"
tsvHal_labStructId_i <- r .: "labStructId_i"
tsvHal_rteamStructId_i <- r .: "rteamStructId_i"
tsvHal_docType_s <- r .: "docType_s"
pure $ TsvHal { .. }
instance ToNamedRecord TsvHal where
--toNamedRecord (TsvHal t s py pm pd abst aut url isbn iss j lang doi auth inst dept lab team doct) =
toNamedRecord (TsvHal { .. }) =
namedRecord [ "title" .= tsvHal_title
, "source" .= tsvHal_source
, "publication_year" .= tsvHal_publication_year
, "publication_month" .= tsvHal_publication_month
, "publication_day" .= tsvHal_publication_day
, "abstract" .= tsvHal_abstract
, "authors" .= tsvHal_authors
, "url" .= tsvHal_url
, "isbn_s" .= tsvHal_isbn_s
, "issue_s" .= tsvHal_issue_s
, "journalPublisher_s" .= tsvHal_journalPublisher_s
, "language_s" .= tsvHal_language_s
, "doiId_s" .= tsvHal_doiId_s
, "authId_i" .= tsvHal_authId_i
, "instStructId_i" .= tsvHal_instStructId_i
, "deptStructId_i" .= tsvHal_deptStructId_i
, "labStructId_i" .= tsvHal_labStructId_i
, "rteamStructId_i" .= tsvHal_rteamStructId_i
, "docType_s" .= tsvHal_docType_s
]
tsv2doc :: TsvHal -> HyperdataDocument
tsv2doc (TsvHal { .. }) =
HyperdataDocument { _hd_bdd = Just "TsvHal"
, _hd_doi = Just tsvHal_doiId_s
, _hd_url = Just tsvHal_url
, _hd_page = Nothing
, _hd_title = Just tsvHal_title
, _hd_authors = Just tsvHal_authors
, _hd_institutes = Just tsvHal_instStructId_i
, _hd_source = Just tsvHal_source
, _hd_abstract = Just tsvHal_abstract
, _hd_publication_date = Just $ pack . show $ jour tsvHal_publication_year
tsvHal_publication_month
tsvHal_publication_day
, _hd_publication_year = Just $ fromIntegral tsvHal_publication_year
, _hd_publication_month = Just tsvHal_publication_month
, _hd_publication_day = Just tsvHal_publication_day
, _hd_publication_hour = Nothing
, _hd_publication_minute = Nothing
, _hd_publication_second = Nothing
, _hd_language_iso2 = Nothing
, _hd_institutes_tree = Nothing }
------------------------------------------------------------------------
-- | TODO use readFileLazy
readTsvHal :: FilePath -> IO (Either Text (Header, Vector TsvHal))
readTsvHal fp = do
c <- BL.readFile fp
pure $ readTsvLazyBS Tab c
------------------------------------------------------------------------
parseHal :: FilePath -> IO (Either Text [HyperdataDocument])
parseHal fp = do
r <- readTsvHal fp
pure $ V.toList . V.map tsv2doc . snd <$> r
{-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV.TsvPhylo
Description :
Copyright : (c) CNRS, 2017-Present
License : AGPL + CECILL v3
Maintainer : team@gargantext.org
Stability : experimental
Portability : POSIX
-}
module Gargantext.Core.Text.Corpus.Parsers.TSV.TsvPhylo where
import Data.Csv
import Gargantext.Prelude
------------------------------------------------------------------------
-- Tsv v3 weighted for phylo
data Tsv' = Tsv'
{ tsv'_title :: !Text
, tsv'_source :: !Text
, tsv'_publication_year :: !Int
, tsv'_publication_month :: !Int
, tsv'_publication_day :: !Int
, tsv'_abstract :: !Text
, tsv'_authors :: !Text
, tsv'_weight :: !Double } deriving (Show)
instance FromNamedRecord Tsv' where
parseNamedRecord r = do
tsv'_title <- r .: "title"
tsv'_source <- r .: "source"
tsv'_publication_year <- r .: "publication_year"
tsv'_publication_month <- r .: "publication_month"
tsv'_publication_day <- r .: "publication_day"
tsv'_abstract <- r .: "abstract"
tsv'_authors <- r .: "authors"
tsv'_weight <- r .: "weight"
pure $ Tsv' { .. }
{-|
Module : Gargantext.Core.Text.Corpus.Parsers.TSV.Utils
Description :
Copyright : (c) CNRS, 2017-Present
License : AGPL + CECILL v3
Maintainer : team@gargantext.org
Stability : experimental
Portability : POSIX
-}
module Gargantext.Core.Text.Corpus.Parsers.TSV.Utils where
import Data.ByteString.Lazy qualified as BL
import Data.ByteString.Lazy.Char8 qualified as B8L
import Data.Csv
import Data.Text (pack)
import Data.Vector (Vector)
import Data.Vector qualified as V
import Gargantext.Core.Text.Corpus.Parsers.TSV.Types
import Gargantext.Prelude
parseTsv :: FromNamedRecord tsvDoc
=> (tsvDoc -> result)
-> FilePath
-> IO (Either Text [result])
parseTsv tsv2doc fp = fmap (V.toList . V.map tsv2doc . snd) <$> readTSVFile fp
-- | TODO use readByteStringLazy
readTsvLazyBS :: FromNamedRecord tsvDoc
=> Delimiter
-> BL.ByteString
-> Either Text (Header, Vector tsvDoc)
readTsvLazyBS d bs = first pack $ decodeByNameWith (tsvDecodeOptions d) bs
-- | TODO use readFileLazy
readTSVFile :: FromNamedRecord tsvDoc
=> FilePath
-> IO (Either Text (Header, Vector tsvDoc))
readTSVFile fp = do
file <- BL.readFile fp
case detectDelimiter file of
Left err -> pure $ Left err
Right del -> pure $ readTsvLazyBS del file
readTSVFileDelim :: FromNamedRecord tsvDoc
=> FilePath
-> Delimiter
-> IO (Either Text (Header, Vector tsvDoc))
readTSVFileDelim fp del = do
file <- BL.readFile fp
pure $ readTsvLazyBS del file
-- Detects delimiter based on the first line
detectDelimiter :: BL.ByteString -> Either Text Delimiter
detectDelimiter input =
case B8L.lines input of
(firstLine : _) ->
let candidates = [(',', count ',' firstLine), ('\t', count '\t' firstLine)]
in case fst $ maximumBy (comparing snd) candidates of
'\n' -> Right Line
'\t' -> Right Tab
',' -> Right Comma
_ -> Left $ "Invalid delimiter detected for input tsv document."
_ -> Left "Couldn't detect a valid delimiter for the input document."
-- Count occurrences of a character in a ByteString
count :: Char -> BL.ByteString -> Int64
count c = BL.count (fromIntegral (fromEnum c))
------------------------------------------------------------------------
writeDocs2Tsv :: ToNamedRecord tsvDoc
=> Header
-> (doc -> tsvDoc)
-> FilePath
-> [doc]
-> IO ()
writeDocs2Tsv hdr doc2tsv fp hs =
BL.writeFile fp $ encodeByNameWith (tsvEncodeOptions Tab) hdr (doc2tsv <$> hs)
...@@ -3,6 +3,7 @@ ...@@ -3,6 +3,7 @@
module Test.Core.Text.Corpus.TSV (tests) where module Test.Core.Text.Corpus.TSV (tests) where
import Gargantext.Core.Text.Corpus.Parsers.TSV import Gargantext.Core.Text.Corpus.Parsers.TSV
import Gargantext.Core.Text.Corpus.Parsers.TSV.TSVv3 qualified as TSVv3
import Test.QuickCheck import Test.QuickCheck
import Test.QuickCheck.Instances () import Test.QuickCheck.Instances ()
import Data.ByteString.Lazy.UTF8 as BLU import Data.ByteString.Lazy.UTF8 as BLU
...@@ -182,7 +183,7 @@ testTestErrorPerLine = forAll generateRandomCorpus (\tsv -> do ...@@ -182,7 +183,7 @@ testTestErrorPerLine = forAll generateRandomCorpus (\tsv -> do
let line = createLineFromCorpus tsv del let line = createLineFromCorpus tsv del
let headers = Prelude.map DT.pack ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"] let headers = Prelude.map DT.pack ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"]
let splitLine = BL.splitWith (==delimiter del) line let splitLine = BL.splitWith (==delimiter del) line
case testErrorPerLine splitLine del headers 1 of case testErrorPerLine TSVv3.validators splitLine del headers 1 of
Right _ -> True Right _ -> True
Left _ -> validRandomCorpus tsv del) Left _ -> validRandomCorpus tsv del)
...@@ -194,13 +195,13 @@ testTestErrorPerLine = forAll generateRandomCorpus (\tsv -> do ...@@ -194,13 +195,13 @@ testTestErrorPerLine = forAll generateRandomCorpus (\tsv -> do
testTestCorrectFile :: Property testTestCorrectFile :: Property
testTestCorrectFile = forAll generateFile (\file -> do testTestCorrectFile = forAll generateFile (\file -> do
let tsv = createFile file let tsv = createFile file
case testCorrectFile tsv of case testCorrectFile TSVv3.validators tsv of
Right del -> del == fDelimiter file Right del -> del == fDelimiter file
Left _ -> Prelude.all (\x -> do Left _ -> Prelude.all (\x -> do
let del = fDelimiter file let del = fDelimiter file
let headers = Prelude.map DT.pack ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"] let headers = Prelude.map DT.pack ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"]
let splitLine = BL.splitWith (==delimiter del) $ createLineFromCorpus x del let splitLine = BL.splitWith (==delimiter del) $ createLineFromCorpus x del
case testErrorPerLine splitLine del headers 1 of case testErrorPerLine TSVv3.validators splitLine del headers 1 of
Right _ -> True Right _ -> True
Left _ -> validRandomCorpus x del) (allCorpus file)) Left _ -> validRandomCorpus x del) (allCorpus file))
...@@ -209,13 +210,13 @@ testTestCorrectFile = forAll generateFile (\file -> do ...@@ -209,13 +210,13 @@ testTestCorrectFile = forAll generateFile (\file -> do
testTestCorrectFileWithNewLine :: Property testTestCorrectFileWithNewLine :: Property
testTestCorrectFileWithNewLine = forAll generateFile (\file -> do testTestCorrectFileWithNewLine = forAll generateFile (\file -> do
let tsv = createFileWithNewLine file let tsv = createFileWithNewLine file
case testCorrectFile tsv of case testCorrectFile TSVv3.validators tsv of
Right _ -> True Right _ -> True
Left _ -> Prelude.all (\x -> do Left _ -> Prelude.all (\x -> do
let del = fDelimiter file let del = fDelimiter file
let headers = Prelude.map DT.pack ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"] let headers = Prelude.map DT.pack ["Publication Day", "Publication Month", "Publication Year", "Authors", "Title", "Source", "Abstract"]
let splitLine = BL.splitWith (==delimiter del) $ createLineFromCorpus x del let splitLine = BL.splitWith (==delimiter del) $ createLineFromCorpus x del
case testErrorPerLine splitLine del headers 1 of case testErrorPerLine TSVv3.validators splitLine del headers 1 of
Right _ -> True Right _ -> True
Left _ -> validRandomCorpus x del) (allCorpus file)) Left _ -> validRandomCorpus x del) (allCorpus file))
...@@ -233,7 +234,7 @@ testFindDelimiter = forAll generateFileDelimiter (\file -> do ...@@ -233,7 +234,7 @@ testFindDelimiter = forAll generateFileDelimiter (\file -> do
testGetHeader :: Property testGetHeader :: Property
testGetHeader = forAll randomHeaderList (\headers -> do testGetHeader = forAll randomHeaderList (\headers -> do
let headersLines = (BL.intercalate (delimiterBS Tab) $ Prelude.map BLU.fromString headers):[] let headersLines = (BL.intercalate (delimiterBS Tab) $ Prelude.map BLU.fromString headers):[]
case getHeaders headersLines Tab of case getHeaders TSVv3.validators headersLines Tab of
Right _ -> True Right _ -> True
Left _ | not ("Publication Day" `Prelude.elem` headers) -> True Left _ | not ("Publication Day" `Prelude.elem` headers) -> True
| not ("Publication Month" `Prelude.elem` headers) -> True | not ("Publication Month" `Prelude.elem` headers) -> True
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment