Commit 4d964e3e authored by Alfredo Di Napoli's avatar Alfredo Di Napoli

Merge branch 'adinapoli/issue-380' into 'dev'

Proper incremental TSV parser

See merge request !402
parents 1dc9db8a 6c1e5560
Pipeline #7531 passed with stages
in 38 minutes and 8 seconds
......@@ -56,6 +56,8 @@ data-files:
test-data/test_config.toml
test-data/issue-381/Termes_A_Ajouter_T4SC_Intellixir.tsv
test-data/issue-381/Termes_A_Ajouter_T4SC_Intellixir12.csv
test-data/issue-380/corpus.tsv
test-data/issue-380/malformed_row.tsv
.clippy.dhall
-- common options
......@@ -226,6 +228,7 @@ library
Gargantext.Core.Text.Corpus.API.OpenAlex
Gargantext.Core.Text.Corpus.API.Pubmed
Gargantext.Core.Text.Corpus.Parsers
Gargantext.Core.Text.Corpus.Parsers.Types
Gargantext.Core.Text.Corpus.Parsers.Date
Gargantext.Core.Text.Corpus.Parsers.TSV
Gargantext.Core.Text.Corpus.Query
......
......@@ -187,6 +187,8 @@ instance MonadJobStatus (GargM DevEnv err) where
markFailed _ _ = pure ()
emitWarning _ _ = pure ()
addMoreSteps _ _ = pure ()
instance HasConfig DevEnv where
......
......@@ -20,6 +20,7 @@ module Gargantext.API.Job (
, jobLogFailTotalWithMessage
, RemainingSteps(..)
, addErrorEvent
, addWarningEvent
) where
import Control.Lens (over, _Just)
......@@ -49,6 +50,9 @@ addEvent level message (JobLog { _scst_events = mEvts, .. }) = JobLog { _scst_ev
addErrorEvent :: ToHumanFriendlyError e => e -> JobLog -> JobLog
addErrorEvent message = addEvent "ERROR" (mkHumanFriendly message)
addWarningEvent :: ToHumanFriendlyError e => e -> JobLog -> JobLog
addWarningEvent message = addEvent "WARNING" (mkHumanFriendly message)
jobLogProgress :: Int -> JobLog -> JobLog
jobLogProgress n jl = over (scst_succeeded . _Just) (+ n) $
over (scst_remaining . _Just) (\x -> max 0 (x - n)) jl
......
......@@ -12,6 +12,7 @@ New corpus means either:
- new data in existing corpus
-}
{-# LANGUAGE LambdaCase #-}
{-# LANGUAGE TemplateHaskell #-}
{-# LANGUAGE TypeOperators #-}
{-# LANGUAGE ViewPatterns #-}
......@@ -20,10 +21,11 @@ module Gargantext.API.Node.Corpus.New
where
import Conduit ((.|), yieldMany, mapMC, mapC, transPipe)
import Conduit ((.|), yieldMany, mapMC, transPipe)
import Control.Exception.Safe (MonadMask)
import Control.Lens ( view, non )
import Data.Conduit.Internal (zipSources)
import Data.Conduit.List (mapMaybeM)
import Data.Swagger ( ToSchema(..) )
import Data.Text qualified as T
import Database.PostgreSQL.Simple.LargeObjects qualified as PSQL
......@@ -34,14 +36,15 @@ import Gargantext.API.Node.Corpus.Searx ( triggerSearxSearch )
import Gargantext.API.Node.Corpus.Types ( Datafield(Web), datafield2origin )
import Gargantext.API.Node.Corpus.Update (addLanguageToCorpus)
import Gargantext.API.Node.Types
import Gargantext.Core (withDefaultLanguage, defaultLanguage)
import Gargantext.Core.Config (gc_jobs, hasConfig)
import Gargantext.Core.Config.Types (jc_max_docs_parsers)
import Gargantext.Core.NodeStory (HasNodeStoryImmediateSaver, HasNodeArchiveStoryImmediateSaver, currentVersion, NgramsStatePatch', HasNodeStoryEnv)
import Gargantext.Core.Text.Corpus.Parsers qualified as Parser (FileType(..), parseFormatC, _ParseFormatError)
import Gargantext.Core.Text.Corpus.Parsers.Types
import Gargantext.Core.Text.Corpus.Query qualified as API
import Gargantext.Core.Types.Individu (User(..))
import Gargantext.Core.Utils.Prefix (unPrefix)
import Gargantext.Core (withDefaultLanguage, defaultLanguage)
import Gargantext.Database.Action.Flow (flowCorpus, getDataText, flowDataText, TermType(..){-, allDataOrigins-})
import Gargantext.Database.Action.Flow.Types (FlowCmdM)
import Gargantext.Database.Action.Mail (sendMail)
......@@ -52,13 +55,14 @@ import Gargantext.Database.Admin.Types.Hyperdata.File ( HyperdataFile(..) )
import Gargantext.Database.Admin.Types.Node (CorpusId, NodeType(..), ParentId)
import Gargantext.Database.GargDB qualified as GargDB
import Gargantext.Database.Prelude (readLargeObject, IsDBCmd)
import Gargantext.Database.Query.Table.Node.Error (HasNodeError)
import Gargantext.Database.Query.Table.Node (getNodeWith, getOrMkList)
import Gargantext.Database.Query.Table.Node.Error (HasNodeError)
import Gargantext.Database.Query.Table.Node.UpdateOpaleye (updateHyperdata)
import Gargantext.Database.Query.Tree.Root (MkCorpusUser(MkCorpusUserNormalCorpusIds))
import Gargantext.Database.Schema.Node (node_hyperdata)
import Gargantext.Prelude
import Gargantext.System.Logging ( logLocM, LogLevel(..) )
import Gargantext.Utils.Jobs.Error (HumanFriendlyErrorText(..))
import Gargantext.Utils.Jobs.Monad (JobHandle, MonadJobStatus(..))
------------------------------------------------------------------------
......@@ -226,9 +230,7 @@ addToCorpusWithTempFile :: ( MonadMask m
-> JobHandle m
-> m ()
addToCorpusWithTempFile user cid nwtf jobHandle = do
-- printDebug "[addToCorpusWithForm] Parsing corpus: " cid
-- printDebug "[addToCorpusWithForm] fileType" ft
-- printDebug "[addToCorpusWithForm] fileFormat" ff
$(logLocM) DEBUG $ "Adding documents to corpus: " <> show cid
let l = nwtf ^. wtf_lang . non defaultLanguage
addLanguageToCorpus cid l
......@@ -256,7 +258,7 @@ addToCorpusWithTempFile user cid nwtf jobHandle = do
-- TODO Add progress (jobStatus) update for docs - this is a
-- long action
let docsC' = zipSources (yieldMany [1..]) docsC
let docsC' = zipSources (yieldMany [1..]) (transPipe liftBase docsC)
.| mapMC (\(idx, doc) ->
if idx > limit then do
--printDebug "[addToCorpusWithForm] number of docs exceeds the limit" (show limit)
......@@ -269,7 +271,10 @@ addToCorpusWithTempFile user cid nwtf jobHandle = do
panicTrace panicMsg
else
pure doc)
.| mapC toHyperdataDocument
.| mapMaybeM (\case
ParseRecordSucceeded r -> pure $ Just $ toHyperdataDocument r
ParseTsvRecordFailed r -> emitTsvParseWarning jobHandle r >> pure Nothing
)
--printDebug "Parsing corpus finished : " cid
--logStatus jobLog2
......@@ -282,7 +287,7 @@ addToCorpusWithTempFile user cid nwtf jobHandle = do
(Multi l)
(Just (nwtf ^. wtf_selection))
--(Just $ fromIntegral $ length docs, docsC')
(count, transPipe liftBase docsC') -- TODO fix number of docs
(count, docsC') -- TODO fix number of docs
--(map (map toHyperdataDocument) docs)
jobHandle
......@@ -298,6 +303,11 @@ addToCorpusWithTempFile user cid nwtf jobHandle = do
$(logLocM) ERROR $ "[addToCorpusWithTempFile] parse error: " <> Parser._ParseFormatError parseErr
markFailed (Just parseErr) jobHandle
emitTsvParseWarning :: MonadJobStatus m => JobHandle m -> AtRow (Text, ByteString) -> m ()
emitTsvParseWarning jh (AtRow row_num (t, _r)) =
let msg = UnsafeMkHumanFriendlyErrorText ("Skipping record at row " <> T.pack (show row_num) <> " as parsing failed due to: " <> t)
in emitWarning msg jh
{-
addToCorpusWithFile :: FlowCmdM env err m
=> CorpusId
......
......@@ -13,6 +13,9 @@ Portability : POSIX
module Gargantext.API.Worker where
import Data.Aeson qualified as JSON
import Data.ByteString.Lazy qualified as BL
import Data.Text.Encoding qualified as TE
import Gargantext.API.Prelude (IsGargServer)
import Gargantext.Core.Worker.Jobs (sendJob)
import Gargantext.Core.Worker.Jobs.Types (Job(..), getWorkerMNodeId)
......@@ -41,7 +44,7 @@ serveWorkerAPIM mkJob = WorkerAPI { workerAPIPost }
where
workerAPIPost i = do
job <- mkJob i
logM DEBUG $ "[serveWorkerAPI] sending job " <> show job
logM DEBUG $ "[serveWorkerAPI] sending job " <> TE.decodeUtf8 (BL.toStrict $ JSON.encode job)
mId <- sendJob job
pure $ JobInfo { _ji_message_id = mId
, _ji_mNode_id = getWorkerMNodeId job }
......
......@@ -47,13 +47,14 @@ import Data.Text qualified as DT
import Data.Tuple.Extra (both) -- , first, second)
import Gargantext.API.Node.Corpus.New.Types (FileFormat(..))
import Gargantext.Core (Lang(..))
import Gargantext.Core.Text.Corpus.Parsers.TSV (parseHal, parseTsv, parseTsvC)
import Gargantext.Core.Text.Corpus.Parsers.Date qualified as Date
import Gargantext.Core.Text.Corpus.Parsers.FrameWrite (text2titleParagraphs)
import Gargantext.Core.Text.Corpus.Parsers.Iramuteq qualified as Iramuteq
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.Presse (presseEnrich)
import Gargantext.Core.Text.Corpus.Parsers.TSV (parseHal, parseTsv, parseTsvC)
import Gargantext.Core.Text.Corpus.Parsers.Types
import Gargantext.Core.Text.Corpus.Parsers.WOS qualified as WOS
import Gargantext.Core.Text.Ngrams (NgramsType(..))
import Gargantext.Database.Admin.Types.Hyperdata.Document ( HyperdataDocument(..) )
......@@ -101,14 +102,14 @@ parseFormatC :: forall m. MonadBaseControl IO m
=> FileType
-> FileFormat
-> DB.ByteString
-> m (Either ParseFormatError (Integer, ConduitT () HyperdataDocument IO ()))
-> m (Either ParseFormatError (Integer, ConduitT () (ParseCorpusResult HyperdataDocument) IO ()))
parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0
where
do_parse :: MonadBaseControl IO m
=> FileType
-> FileFormat
-> DB.ByteString
-> m (Either DT.Text (Integer, ConduitT () HyperdataDocument IO ()))
-> m (Either DT.Text (Integer, ConduitT () (ParseCorpusResult HyperdataDocument) IO ()))
do_parse TsvGargV3 Plain bs = do
let eParsedC = parseTsvC $ DBL.fromStrict bs
pure (second (transPipe (pure . runIdentity)) <$> eParsedC)
......@@ -117,7 +118,7 @@ parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0
pure (second (transPipe (pure . runIdentity)) <$> eParsedC)
do_parse Istex Plain bs = do
ep <- liftBase $ parseIstex EN $ DBL.fromStrict bs
pure $ (\p -> (1, yieldMany [p])) <$> ep
pure $ (\p -> (1, yieldMany [ParseRecordSucceeded p])) <$> ep
do_parse RisPresse Plain bs = do
--docs <- enrichWith RisPresse
let eDocs = runParser' RisPresse bs
......@@ -126,7 +127,7 @@ parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0
, yieldMany docs
.| mapC presseEnrich
.| mapC (map $ both decodeUtf8)
.| mapMC (toDoc RIS)) ) <$> eDocs
.| mapMC (fmap ParseRecordSucceeded . toDoc RIS)) ) <$> eDocs
do_parse WOS Plain bs = do
let eDocs = runParser' WOS bs
pure $ (\docs ->
......@@ -134,7 +135,7 @@ parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0
, yieldMany docs
.| mapC (map $ first WOS.keys)
.| mapC (map $ both decodeUtf8)
.| mapMC (toDoc WOS)) ) <$> eDocs
.| mapMC (fmap ParseRecordSucceeded . toDoc WOS)) ) <$> eDocs
do_parse Iramuteq Plain bs = do
let eDocs = runParser' Iramuteq bs
pure $ (\docs ->
......@@ -142,13 +143,13 @@ parseFormatC ft ff bs0 = first ParseFormatError <$> do_parse ft ff bs0
, yieldMany docs
.| mapC (map $ first Iramuteq.keys)
.| mapC (map $ both decodeUtf8)
.| mapMC (toDoc Iramuteq . map (second (DT.replace "_" " ")))
.| mapMC (fmap ParseRecordSucceeded . toDoc Iramuteq . map (second (DT.replace "_" " ")))
)
)
<$> eDocs
do_parse JSON Plain bs = do
let eParsedC = parseJSONC $ DBL.fromStrict bs
pure (second (transPipe (pure . runIdentity)) <$> eParsedC)
pure (second (mapOutput ParseRecordSucceeded . transPipe (pure . runIdentity)) <$> eParsedC)
do_parse fty ZIP bs = liftBase $ UZip.withZipFileBS bs $ do
fileNames <- filter (filterZIPFileNameP ft) . DM.keys <$> getEntries
printDebug "[do_parse] fileNames" fileNames
......
module Gargantext.Core.Text.Corpus.Parsers.Types where
import Data.ByteString
import Data.Csv
import Data.Text
import Prelude
-- | Allows the parser to report errors happening at a given row in the document.
data AtRow a = AtRow Int a
deriving Show
data ParseCorpusResult a
= ParseRecordSucceeded a
| ParseTsvRecordFailed (AtRow (Text, ByteString))
deriving Show
instance FromNamedRecord a => FromNamedRecord (ParseCorpusResult a) where
parseNamedRecord m = ParseRecordSucceeded <$> parseNamedRecord m
......@@ -18,8 +18,8 @@ Portability : POSIX
module Gargantext.Core.Worker where
import Async.Worker.Broker.Types (toA, getMessage, messageId)
import Async.Worker qualified as W
import Async.Worker.Broker.Types (toA, getMessage, messageId)
import Async.Worker.Types qualified as W
import Control.Exception.Safe qualified as CES
import Control.Lens (to)
......@@ -33,14 +33,14 @@ import Gargantext.API.Ngrams.List (postAsyncJSON)
import Gargantext.API.Node.Contact (addContact)
import Gargantext.API.Node.Corpus.Annuaire qualified as Annuaire
import Gargantext.API.Node.Corpus.New (addToCorpusWithTempFile, addToCorpusWithQuery)
import Gargantext.API.Node.DocumentsFromWriteNodes (documentsFromWriteNodes)
import Gargantext.API.Node.DocumentUpload (documentUploadAsync, remoteImportDocuments)
import Gargantext.API.Node.DocumentsFromWriteNodes (documentsFromWriteNodes)
import Gargantext.API.Node.File (addWithFile)
import Gargantext.API.Node.FrameCalcUpload (frameCalcUploadAsync)
import Gargantext.API.Node.New (postNode')
import Gargantext.API.Node.Types (_wtf_file_oid)
import Gargantext.API.Node.Update.Types (UpdateNodeParams(..), Granularity (..))
import Gargantext.API.Node.Update (updateNode)
import Gargantext.API.Node.Update.Types (UpdateNodeParams(..), Granularity (..))
import Gargantext.API.Server.Named.Ngrams (tableNgramsPostChartsAsync)
import Gargantext.Core.Config (hasConfig, gc_database_config, gc_jobs, gc_worker, gc_logging)
import Gargantext.Core.Config.Types (jc_max_docs_scrapers)
......@@ -56,7 +56,7 @@ import Gargantext.Core.Worker.Types (JobInfo(..))
import Gargantext.Database.Prelude (readLargeObject, removeLargeObject)
import Gargantext.Database.Query.Table.User (getUsersWithEmail)
import Gargantext.Prelude hiding (to)
import Gargantext.System.Logging ( logLocM, LogLevel(..), logMsg, withLogger )
import Gargantext.System.Logging
import Gargantext.Utils.Jobs.Error (HumanFriendlyErrorText(..))
import Gargantext.Utils.Jobs.Monad (MonadJobStatus(markStarted, markComplete, markFailed))
import System.Posix.Signals (Handler(Catch), installHandler, keyboardSignal)
......@@ -90,7 +90,7 @@ notifyJobStarted env (W.State { name }) bm = do
let j = toA $ getMessage bm
let job = W.job j
withLogger (env ^. w_env_config . gc_logging) $ \ioL ->
logMsg ioL DEBUG $ "[notifyJobStarted] [" <> name <> " :: " <> show mId <> "] starting job: " <> show j
$(logLoc) ioL DEBUG $ T.pack $ "[notifyJobStarted] [" <> name <> " :: " <> show mId <> "] starting job: " <> show j
let ji = JobInfo { _ji_message_id = messageId bm
, _ji_mNode_id = getWorkerMNodeId job }
let jh = WorkerJobHandle { _w_job_info = ji }
......
......@@ -27,7 +27,7 @@ import Data.Maybe (fromJust)
import Data.Pool qualified as Pool
import Database.PostgreSQL.Simple qualified as PSQL
import Gargantext.API.Admin.Orchestrator.Types (JobLog, noJobLog)
import Gargantext.API.Job (RemainingSteps(..), jobLogStart, jobLogProgress, jobLogFailures, jobLogComplete, addErrorEvent, jobLogFailTotal, jobLogFailTotalWithMessage, jobLogAddMore)
import Gargantext.API.Job (RemainingSteps(..), jobLogStart, jobLogProgress, jobLogFailures, jobLogComplete, addErrorEvent, jobLogFailTotal, jobLogFailTotalWithMessage, jobLogAddMore, addWarningEvent)
import Gargantext.API.Prelude (GargM)
import Gargantext.Core.Notifications.CentralExchange qualified as CE
import Gargantext.Core.Notifications.CentralExchange.Types qualified as CET
......@@ -229,6 +229,7 @@ instance MonadJobStatus WorkerMonad where
updateJobProgress jh (\latest -> case mb_msg of
Nothing -> jobLogFailTotal latest
Just msg -> jobLogFailTotalWithMessage msg latest)
emitWarning msg jh = updateJobProgress jh (addWarningEvent msg)
addMoreSteps steps jh = updateJobProgress jh (jobLogAddMore steps)
......
{-# LANGUAGE TemplateHaskell #-}
{-|
Module : Gargantext.Core.Worker.Jobs
Description : Worker job definitions
......@@ -22,7 +23,7 @@ import Gargantext.Core.Worker.Jobs.Types (Job(..))
import Gargantext.Core.Worker.PGMQTypes (HasWorkerBroker, MessageId, SendJob)
import Gargantext.Database.Prelude (Cmd)
import Gargantext.Prelude
import Gargantext.System.Logging (logMsg, withLogger, LogLevel(..))
import Gargantext.System.Logging
sendJob :: (HasWorkerBroker, HasConfig env)
......@@ -45,7 +46,7 @@ sendJobWithCfg gcConfig job = do
let queueName = _wdQueue wd
let job' = (updateJobData job $ W.mkDefaultSendJob' b queueName job) { W.delay = _wsDefaultDelay }
withLogger (gcConfig ^. gc_logging) $ \ioL ->
logMsg ioL DEBUG $ "[sendJob] sending job " <> show job <> " (delay " <> show (W.delay job') <> ")"
$(logLoc) ioL DEBUG $ "[sendJob] sending job " <> show job <> " (delay " <> show (W.delay job') <> ")"
W.sendJob' job'
-- | We want to fine-tune job metadata parameters, for each job type
......
......@@ -240,9 +240,10 @@ fromField' field mb = do
where
valueToHyperdata v = case fromJSON v of
Success a -> pure a
Error _err -> returnError ConversionFailed field
Error err -> returnError ConversionFailed field
$ DL.unwords [ "cannot parse hyperdata for JSON: "
, show v
, err
]
dbCheck :: DBCmd err Bool
......
......@@ -104,6 +104,9 @@ class MonadJobStatus m where
-- message to the failure.
markFailed :: forall e. ToHumanFriendlyError e => Maybe e -> JobHandle m -> m ()
-- Logs a new event with \"WARNING\" severity.
emitWarning :: forall e. ToHumanFriendlyError e => e -> JobHandle m -> m ()
-- | Add 'n' more steps to the running computation, they will be marked as remaining.
addMoreSteps :: MonadJobStatus m => Int -> JobHandle m -> m ()
......
Title Authors Source Abstract Publication Year Publication Month Publication Day
#Ecuador | 🚨Se insta antisananews mastodon.social #Ecuador | 🚨Se instaló la audiencia de juicio contra el exalcalde de #Quito, Jorge Yunda, y 13 personas más, procesadas por #peculado en la compra de 100.000 pruebas para detectar #COVID19 y que presuntamente abrían causado un perjuicio al Estado por 2’235.491,16 dólares.#Comparta #Suscríbase↩👍Sígame enRadio Antisana Media Online: https://antisananews.blogspot.com/TikTok: https://www.tiktok.com/@antisanamediaonline?lang=esTelegram: https://t.me/AntisanaMediaOnlineVK: https://vk.com/antisanamultimediosX: https://twitter.com/AntisanaNews 2024 02 06
#NorthCarolina bann MatthewChat@mstdn.social toot.io #NorthCarolina banned immunocompromized people from wearing masks in public. This does NOT apply to the #KKK, as their is a specific exemption for them. #covid #masking 2024 05 16
'The coronation of t BenHigbie@mastodon.social fosstodon.org 'The coronation of the Serbian Tsar Stefan Dušan as East Roman Emperor' from 'The Slav Epic' by Alphonse Mucha (1926) #art #arts #artist #artists #artlover #artlovers #arthistory #illustration #painting #paintings #inspiration #artmuseum #museum #artmuseums #museums #artnet 2024 05 05
3/5 Krankenstand 202 ToveHarris mastodon.social 3/5 Krankenstand 2022 und 2023 komme Verlust von 350.000 Beschäftigten gleich. Arbeitsausfall werde derzeit durch Überstunden + erhöhte Produktivität aufgefangen. Ohne diese Leistungen der Beschäftigten gäbe es eine Lücke von 700.000 Beschäftigten.Und #CovidIsNotOver #LongCOVID #COVID #COVID19 #Coronahttps://www.vfa.de/de/wirtschaft-politik/macroscope/macroscope-hoher-krankenstand-drueckt-deutschland-in-die-rezession 2024 01 27
@ABScientist @Hidde justafrog@mstdn.social mastodon.social @ABScientist @Hidde @roelgrif How about this one?https://nos.nl/artikel/2457983-viroloog-koopmans-coronagolf-in-china-nu-niet-heel-zorgelijk-voor-nederland 2023 12 28
@cassandra17lina In gemswinc counter.social @cassandra17lina In a way, Covid was a gift to introverts 2024 01 24
@erictopol This is t SpookieRobieTheCat@mastodon.social toot.io @erictopol This is the #CovidBrain that Trump and MAGA suffer from. All those #Antivaxx proponents will suffer the consequences too. And I'm ok with it. Be anti-science, be wilfully ignorant and live a life in agony, that's their choice. I shouldn't have to pay a dime for their stupidity. 2024 01 09
A bunch of maskless crowgirl@hachyderm.io toot.io "A bunch of maskless #Covid "experts" like Gregg Gonsalves are now trying to sound credible by warning about H5N1.I think H5N1 is a serious problem that requires airborne mitigations and decontaminating the food supply.And I also don't dine in restaurants. Gregg on the other hand is proud of his masklessness.Be warned. Don't let these professional Covid minimizers get away with this crap with #H5N1." 2024 06 03
Title Authors Source Abstract Publication Year Publication Month Publication Day
#Ecuador | 🚨Se insta antisananews mastodon.social #Ecuador | 🚨Se instaló la audiencia de juicio contra el exalcalde de #Quito, Jorge Yunda, y 13 personas más, procesadas por #peculado en la compra de 100.000 pruebas para detectar #COVID19 y que presuntamente abrían causado un perjuicio al Estado por 2’235.491,16 dólares.#Comparta #Suscríbase↩👍Sígame enRadio Antisana Media Online: https://antisananews.blogspot.com/TikTok: https://www.tiktok.com/@antisanamediaonline?lang=esTelegram: https://t.me/AntisanaMediaOnlineVK: https://vk.com/antisanamultimediosX: https://twitter.com/AntisanaNews 2024 02 06
#NorthCarolina bann MatthewChat@mstdn.social toot.io #NorthCarolina banned immunocompromized people from wearing masks in public. This does NOT apply to the #KKK, as their is a specific exemption for them. #covid #masking 2024 05 16
'The coronation of t BenHigbie@mastodon.social fosstodon.org 'The coronation of the Serbian Tsar Stefan Dušan as East Roman Emperor' from 'The Slav Epic' by Alphonse Mucha (1926) #art #arts #artist #artists #artlover #artlovers #arthistory #illustration #painting #paintings #inspiration #artmuseum #museum #artmuseums #museums #artnet 2024 05 05
3/5 Krankenstand 202 ToveHarris mastodon.social 3/5 Krankenstand 2022 und 2023 komme Verlust von 350.000 Beschäftigten gleich. Arbeitsausfall werde derzeit durch Überstunden + erhöhte Produktivität aufgefangen. Ohne diese Leistungen der Beschäftigten gäbe es eine Lücke von 700.000 Beschäftigten.Und #CovidIsNotOver #LongCOVID #COVID #COVID19 #Coronahttps://www.vfa.de/de/wirtschaft-politik/macroscope/macroscope-hoher-krankenstand-drueckt-deutschland-in-die-rezession 2024 01 27
@ABScientist @Hidde,justafrog@mstdn.social,mastodon.social,@ABScientist @Hidde @roelgrif How about this one?https://nos.nl/artikel/2457983-viroloog-koopmans-coronagolf-in-china-nu-niet-heel-zorgelijk-voor-nederland 2023 12 28
@cassandra17lina In gemswinc counter.social @cassandra17lina In a way, Covid was a gift to introverts 2024 01 24
@erictopol This is t SpookieRobieTheCat@mastodon.social toot.io @erictopol This is the #CovidBrain that Trump and MAGA suffer from. All those #Antivaxx proponents will suffer the consequences too. And I'm ok with it. Be anti-science, be wilfully ignorant and live a life in agony, that's their choice. I shouldn't have to pay a dime for their stupidity. 2024 01 09
A bunch of maskless crowgirl@hachyderm.io toot.io "A bunch of maskless #Covid "experts" like Gregg Gonsalves are now trying to sound credible by warning about H5N1.I think H5N1 is a serious problem that requires airborne mitigations and decontaminating the food supply.And I also don't dine in restaurants. Gregg on the other hand is proud of his masklessness.Be warned. Don't let these professional Covid minimizers get away with this crap with #H5N1." 2024 06 03
......@@ -9,15 +9,23 @@ module Test.API.Private.List (
) where
import Data.Aeson.QQ
import Data.Text qualified as T
import Data.Text.IO qualified as TIO
import Fmt
import Gargantext.API.Admin.Orchestrator.Types
import Gargantext.API.HashedResponse
import Gargantext.API.Ngrams.List.Types
import Gargantext.API.Ngrams.Types qualified as APINgrams
import Gargantext.API.Node.Corpus.New.Types qualified as FType
import Gargantext.API.Node.Types
import Gargantext.Core qualified as Lang
import Gargantext.Core.Config
import Gargantext.Core.Text.List.Social
import Gargantext.Core.Types
import Gargantext.Core.Types.Individu
import Gargantext.Prelude
import Paths_gargantext
import Prelude
import Servant.Client.Streaming
import Test.API.Prelude (newCorpusForUser, checkEither)
import Test.API.Routes
......@@ -27,7 +35,49 @@ import Test.Hspec (Spec, it, aroundAll, describe, sequential)
import Test.Hspec.Expectations
import Test.Hspec.Wai.Internal (withApplication)
import Test.Utils
import Fmt
mkNewWithForm :: T.Text -> T.Text -> NewWithForm
mkNewWithForm content name = NewWithForm
{ _wf_filetype = FType.TSV
, _wf_fileformat = FType.Plain
, _wf_data = content
, _wf_lang = Just Lang.EN
, _wf_name = name
, _wf_selection = FlowSocialListWithPriority MySelfFirst
}
importTermsTSV :: SpecContext () -> String -> IO (JobLog, CorpusId, ListId)
importTermsTSV (SpecContext testEnv port app _) name = do
cId <- liftIO $ newCorpusForUser testEnv "alice"
let log_cfg = test_config testEnv ^. gc_logging
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
([listId] :: [NodeId]) <- protectedJSON token "POST" (mkUrl port ("/node/" <> build cId)) [aesonQQ|{"pn_typename":"NodeList","pn_name":"Testing"}|]
-- Upload the CSV doc
simpleNgrams <- liftIO (TIO.readFile =<< getDataFileName name)
let params = WithTextFile { _wtf_filetype = FType.TSV
, _wtf_data = simpleNgrams
, _wtf_name = "simple.tsv" }
pendingJob <- checkEither $ liftIO $ runClientM (add_tsv_to_list token listId params) clientEnv
jobLog <- pollUntilWorkFinished log_cfg port pendingJob
pure (jobLog, cId, listId)
importCorpusTSV :: SpecContext () -> String -> IO (JobLog, CorpusId, ListId)
importCorpusTSV (SpecContext testEnv port app _) name = do
cId <- liftIO $ newCorpusForUser testEnv "alice"
let log_cfg = test_config testEnv ^. gc_logging
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
([listId] :: [NodeId]) <- protectedJSON token "POST" (mkUrl port ("/node/" <> build cId)) [aesonQQ|{"pn_typename":"NodeList","pn_name":"Testing"}|]
-- Upload the CSV doc
simpleNgrams <- liftIO (TIO.readFile =<< getDataFileName name)
let params = mkNewWithForm simpleNgrams "simple.tsv"
pendingJob <- checkEither $ liftIO $ runClientM (importCorpus token cId params) clientEnv
jobLog <- pollUntilWorkFinished log_cfg port pendingJob
pure (jobLog, cId, listId)
tests :: Spec
tests = sequential $ aroundAll withTestDBAndPort $ do
......@@ -39,22 +89,45 @@ tests = sequential $ aroundAll withTestDBAndPort $ do
describe "Importing terms as TSV" $ do
it "should work for TSV with a missing 'forms' column" $ \(SpecContext testEnv port app _) -> do
cId <- newCorpusForUser testEnv "alice"
let log_cfg = test_config testEnv ^. gc_logging
it "should work for TSV with a missing 'forms' column" $ \ctx@(SpecContext _ port app _) -> do
(_, cId, listId) <- importTermsTSV ctx "test-data/issue-381/Termes_A_Ajouter_T4SC_Intellixir.tsv"
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
([listId] :: [NodeId]) <- protectedJSON token "POST" (mkUrl port ("/node/" <> build cId)) [aesonQQ|{"pn_typename":"NodeList","pn_name":"Testing"}|]
-- Upload the CSV doc
simpleNgrams <- liftIO (TIO.readFile =<< getDataFileName "test-data/issue-381/Termes_A_Ajouter_T4SC_Intellixir.tsv")
let params = WithTextFile { _wtf_filetype = FType.TSV
, _wtf_data = simpleNgrams
, _wtf_name = "simple.tsv" }
pendingJob <- checkEither $ liftIO $ runClientM (add_tsv_to_list token listId params) clientEnv
_ <- pollUntilWorkFinished log_cfg port pendingJob
-- Now check that we can retrieve the ngrams, and the ngrams list is not empty!
liftIO $ do
eRes <- checkEither $ runClientM (get_table_ngrams token cId APINgrams.Terms listId 50 Nothing (Just MapTerm) Nothing Nothing Nothing Nothing) clientEnv
let (APINgrams.NgramsTable terms) = APINgrams._vc_data eRes
length terms `shouldSatisfy` (>= 1)
it "should handle dirty TSV as per issue #380" $ \ctx@(SpecContext _testEnv port app _) -> do
(_, cId, _listId) <- importCorpusTSV ctx "test-data/issue-380/corpus.tsv"
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
-- Now check that we can retrieve the ngrams, and the ngrams list is not empty!
liftIO $ do
eRes <- checkEither $ runClientM (get_table token cId (Just APINgrams.Docs) Nothing Nothing Nothing Nothing Nothing) clientEnv
let (HashedResponse _ TableResult{tr_docs}) = eRes
length tr_docs `shouldBe` 7
it "should skip problematic rows" $ \ctx@(SpecContext _testEnv port app _) -> do
(jobLogs, cId, _listId) <- importCorpusTSV ctx "test-data/issue-380/malformed_row.tsv"
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
-- Now check that we can retrieve the ngrams, and the ngrams list is not empty!
liftIO $ do
eRes <- checkEither $ runClientM (get_table token cId (Just APINgrams.Docs) Nothing Nothing Nothing Nothing Nothing) clientEnv
let (HashedResponse _ TableResult{tr_docs}) = eRes
length tr_docs `shouldBe` 6 -- it must have skipped the broken row
-- Check that the events include the two failures we encountered.
_scst_events jobLogs `shouldBe` Just [
ScraperEvent {
_scev_message = Just "Skipping record at row 6 as parsing failed due to: no field named \"Publication Year\""
, _scev_level = Just "WARNING"
, _scev_date = Nothing
}
,ScraperEvent {
_scev_message = Just "Skipping record at row 8 as parsing failed due to: parse error (endOfInput)"
, _scev_level = Just "WARNING"
, _scev_date = Nothing
}
]
......@@ -30,6 +30,7 @@ module Test.API.Routes (
, add_tsv_to_list
, get_corpus_sqlite_export
, addTeamMember
, importCorpus
) where
import Data.Text.Encoding qualified as TE
......@@ -66,6 +67,8 @@ import Servant (Headers, Header)
import Servant.Auth.Client qualified as S
import Servant.Client.Streaming
import Servant.Conduit ()
import Gargantext.API.Routes.Named.Corpus (addWithTempFileEp)
import Gargantext.API.Node.Types (NewWithForm)
-- This is for requests made by http.client directly to hand-crafted URLs.
......@@ -381,3 +384,20 @@ addTeamMember (toServantToken -> token) nodeId params = fmap UnsafeMkNodeId $
& shareAPI
& shareNodeEp
& ($ params)
importCorpus :: Token -> CorpusId -> NewWithForm -> ClientM JobInfo
importCorpus (toServantToken -> token) corpusId params =
clientRoutes & apiWithCustomErrorScheme
& ($ GES_new)
& backendAPI
& backendAPI'
& mkBackEndAPI
& gargAPIVersion
& gargPrivateAPI
& mkPrivateAPI
& ($ token)
& addWithTempFile
& addWithTempFileEp
& ($ corpusId)
& workerAPIPost
& (\submitForm -> submitForm params)
......@@ -24,6 +24,7 @@ module Test.API.UpdateList (
, updateNode
, createDocsList
, createFortranDocsList
, mkNewWithForm
) where
import Control.Lens (mapped, over)
......@@ -74,7 +75,7 @@ import Test.Hspec
import Test.Hspec.Wai.Internal (withApplication, WaiSession)
import Test.Hspec.Wai.JSON (json)
import Test.Types (JobPollHandle(..))
import Test.Utils (pollUntilWorkFinished, protectedJSON, withValidLogin)
import Test.Utils (pollUntilWorkFinished, protectedJSON, withValidLogin, isJobFinished)
import Text.Printf (printf)
......@@ -103,7 +104,7 @@ uploadJSONList log_cfg port token cId pathToNgrams clientEnv = do
ji <- checkEither $ liftIO $ runClientM (add_form_to_list token listId params) clientEnv
-- liftIO (_jph_status j' `shouldBe` "IsFinished")
ji' <- pollUntilWorkFinished log_cfg port ji
liftIO $ ji' `shouldBe` ji
liftIO $ ji' `shouldSatisfy` isJobFinished
pure listId
......@@ -356,7 +357,7 @@ createDocsList testDataPath testEnv port clientEnv token = do
let newWithForm = mkNewWithForm simpleDocs (T.pack $ takeBaseName testDataPath)
ji <- checkEither $ liftIO $ runClientM (add_file_async token corpusId newWithForm) clientEnv
ji' <- pollUntilWorkFinished log_cfg port ji
liftIO $ ji' `shouldBe` ji
liftIO $ ji' `shouldSatisfy` isJobFinished
pure corpusId
where
log_cfg = (test_config testEnv) ^. gc_logging
......@@ -375,7 +376,7 @@ updateNode log_cfg port clientEnv token nodeId = do
let params = UpdateNodeParamsTexts Both
ji <- checkEither $ liftIO $ runClientM (update_node token nodeId params) clientEnv
ji' <- pollUntilWorkFinished log_cfg port ji
liftIO $ ji' `shouldBe` ji
liftIO $ ji' `shouldSatisfy` isJobFinished
mkNewWithForm :: T.Text -> T.Text -> NewWithForm
mkNewWithForm content name = NewWithForm
......
......@@ -89,6 +89,7 @@ instance MonadJobStatus TestMonad where
markFailure _ _ _ = TestMonad $ pure ()
markComplete _ = TestMonad $ pure ()
markFailed _ _ = TestMonad $ pure ()
emitWarning _ _ = TestMonad $ pure ()
addMoreSteps _ _ = TestMonad $ pure ()
data DBHandle = DBHandle {
......
......@@ -28,6 +28,7 @@ module Test.Utils (
, waitUntil
, withValidLogin
, withValidLoginA
, isJobFinished
) where
import Control.Concurrent.STM.TChan (TChan, readTChan)
......@@ -256,10 +257,10 @@ pollUntilWorkFinished :: HasCallStack
=> LogConfig
-> Port
-> JobInfo
-> WaiSession () JobInfo
-> WaiSession () JobLog
pollUntilWorkFinished log_cfg port ji = do
let waitSecs = 60
isFinishedTVar <- liftIO $ newTVarIO False
isFinishedTVar <- liftIO $ newTVarIO Nothing
let wsConnect =
withWSConnection ("127.0.0.1", port) $ \conn -> do
-- We wait a bit before the server settles
......@@ -275,11 +276,11 @@ pollUntilWorkFinished log_cfg port ji = do
Just (DT.NUpdateWorkerProgress ji' jl) -> do
withLogger log_cfg $ \ioL ->
logMsg ioL DEBUG $ "[pollUntilWorkFinished] received " <> show ji' <> ", " <> show jl
if ji' == ji && isFinished jl
if ji' == ji && isJobFinished jl
then do
withLogger log_cfg $ \ioL ->
logMsg ioL DEBUG $ "[pollUntilWorkFinished] FINISHED! " <> show ji'
atomically $ writeTVar isFinishedTVar True
atomically $ writeTVar isFinishedTVar (Just jl)
else
pure ()
_ -> pure ()
......@@ -287,23 +288,24 @@ pollUntilWorkFinished log_cfg port ji = do
liftIO $ withAsync wsConnect $ \_ -> do
mRet <- Timeout.timeout (waitSecs * 1000 * millisecond) $ do
let go = do
finished <- readTVarIO isFinishedTVar
if finished
then do
finished_mb <- readTVarIO isFinishedTVar
case finished_mb of
Just job_log -> do
withLogger log_cfg $ \ioL ->
logMsg ioL DEBUG $ "[pollUntilWorkFinished] JOB FINISHED: " <> show ji
return True
else do
pure job_log
Nothing -> do
threadDelay (50 * millisecond)
go
go
case mRet of
Nothing -> panicTrace $ "[pollUntilWorkFinished] timed out while waiting to finish job " <> show ji
Just _ -> return ji
Just jl -> pure jl
where
isFinished (JobLog { .. }) = _scst_remaining == Just 0
isJobFinished :: JobLog -> Bool
isJobFinished (JobLog { .. }) = _scst_remaining == Just 0
-- | Like HUnit's '@?=', but With a nicer error message in case the two entities are not equal.
(@??=) :: (HasCallStack, ToExpr a, Eq a) => a -> a -> Assertion
......
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