Commit 9156c234 authored by Fabien Maniere's avatar Fabien Maniere

Merge branch 'adinapoli/issue-498-part-2' into 'dev'

Prevent importing ngrams which will lead to loops

See merge request !435
parents 062f0c68 19df569a
Pipeline #7882 passed with stages
in 60 minutes and 23 seconds
......@@ -36,6 +36,7 @@ import Gargantext.Database.Query.Tree hiding (treeError)
import Gargantext.Utils.Jobs.Monad (JobError(..))
import Network.HTTP.Types.Status qualified as HTTP
import Servant.Server (ServerError(..), err404, err500)
import Gargantext.Core.NodeStory (NodeStoryError(..), renderLoop, BuildForestError (..))
$(deriveHttpStatusCode ''BackendErrorCode)
......@@ -91,6 +92,12 @@ backendErrorToFrontendError = \case
AccessPolicyErrorReason reason
-> mkFrontendErr' "A policy check failed"
$ FE_policy_check_error reason
InternalNodeStoryError nodeStoryError
-> case nodeStoryError of
NodeStoryUpsertFailed (BFE_loop_detected visited)
-- FIXME(adn) proper constructor.
-> let msg = "A loop was detected in ngrams: " <> renderLoop visited
in mkFrontendErr' msg $ FE_internal_server_error msg
frontendErrorToGQLServerError :: FrontendError -> ServerError
frontendErrorToGQLServerError fe@(FrontendError diag ty _) =
......
......@@ -48,9 +48,10 @@ module Gargantext.API.Errors.Types (
) where
import Control.Lens ((#), makePrisms, Prism')
import Control.Lens.Prism (prism')
import Control.Monad.Fail (fail)
import Data.Aeson.Types (typeMismatch, emptyArray)
import Data.Aeson (Value(..), (.:), (.=), object, withObject)
import Data.Aeson.Types (typeMismatch, emptyArray)
import Data.List.NonEmpty qualified as NE
import Data.Singletons.TH ( SingI(sing), SingKind(fromSing) )
import Data.Text qualified as T
......@@ -59,6 +60,7 @@ import Gargantext.API.Admin.Auth.Types (AuthenticationError)
import Gargantext.API.Errors.Class (HasAuthenticationError(..))
import Gargantext.API.Errors.TH ( deriveIsFrontendErrorData )
import Gargantext.API.Errors.Types.Backend
import Gargantext.Core.NodeStory.Types
import Gargantext.Core.Types (HasValidationError(..))
import Gargantext.Core.Types.Individu (Username)
import Gargantext.Database.Admin.Types.Node
......@@ -68,7 +70,6 @@ import Gargantext.Prelude hiding (Location, WithStacktrace)
import Gargantext.Utils.Dict (Dict(..))
import Gargantext.Utils.Jobs.Monad qualified as Jobs
import Servant (ServerError)
import Control.Lens.Prism (prism')
-- | A 'WithStacktrace' carries an error alongside its
-- 'CallStack', to be able to print the correct source location
......@@ -120,6 +121,7 @@ data BackendInternalError
| InternalValidationError !Validation
| InternalWorkerError !IOException
| AccessPolicyError !AccessPolicyErrorReason
| InternalNodeStoryError !NodeStoryError
deriving (Show, Typeable)
makePrisms ''BackendInternalError
......@@ -142,7 +144,21 @@ instance ToJSON BackendInternalError where
_ -> ""
toJSON err = object [("error", String $ T.pack $ show err)]
instance Exception BackendInternalError
instance Jobs.ToHumanFriendlyError BackendInternalError where
mkHumanFriendly e = case e of
InternalAuthenticationError{} -> show e
InternalJobError{} -> show e
InternalNodeError{} -> show e
InternalServerError{} -> show e
InternalTreeError{} -> show e
InternalUnexpectedError{} -> show e
InternalValidationError{} -> show e
InternalWorkerError{} -> show e
AccessPolicyError{} -> show e
InternalNodeStoryError nodeStoryErr -> Jobs.mkHumanFriendly nodeStoryErr
instance Exception BackendInternalError where
displayException = T.unpack . Jobs.mkHumanFriendly
instance HasNodeError BackendInternalError where
_NodeError = _InternalNodeError
......@@ -159,6 +175,9 @@ instance HasServerError BackendInternalError where
instance HasAuthenticationError BackendInternalError where
_AuthenticationError = _InternalAuthenticationError
instance HasNodeStoryError BackendInternalError where
_NodeStoryError = _InternalNodeStoryError
-- | An error that can be returned to the frontend. It carries a human-friendly
-- diagnostic, the 'type' of the error as well as some context-specific data.
data FrontendError where
......
......@@ -105,12 +105,12 @@ import Data.Map.Strict.Patch qualified as PM
import Data.Patch.Class (Action(act), Transformable(..), ours)
import Data.Set qualified as Set
import Data.Text (isInfixOf, toLower, unpack)
import Data.Text qualified as T
import Data.Text.Lazy.IO as DTL ( writeFile )
import Data.Tree
import Gargantext.API.Ngrams.Tools (getNodeStory)
import Gargantext.API.Ngrams.Types
import Gargantext.Core.NodeStory hiding (buildForest)
import Gargantext.Core.NodeStory qualified as NodeStory
import Gargantext.Core.Text.Ngrams (Ngrams, NgramsType)
import Gargantext.Core.Types (ListType(..), NodeId, ListId, TODO, assertValid, ContextId, HasValidationError)
import Gargantext.Core.Types.Query (Limit(..), Offset(..), MinSize(..), MaxSize(..))
......@@ -218,6 +218,13 @@ addListNgrams listId ngramsType nes = do
-- | TODO: incr the Version number
-- && should use patch
-- UNSAFE
-- FIXME(adinapoli): This function used to be very dangerous as it didn't
-- prevent imports from creating loops: if we had a list of imported terms with a tree
-- referencing an existing node in a forest, we could accidentally create loops. The most
-- efficient way would be to use the patch API to generate a patch for the input, apply it
-- to the current state and handle conflicts, discovering loops there. However, given that
-- it's complex to do that, for the moment we use the Forest API to detect loops, failing
-- if one is found.
setListNgrams :: NodeStoryEnv err
-> NodeId
-> NgramsType
......@@ -230,18 +237,6 @@ setListNgrams env listId ngramsType ns = do
Nothing -> Just ns
Just ns' -> Just $ ns <> ns')
saveNodeStory env listId a'
-- liftBase $ atomically $ do
-- nls <- readTVar var
-- writeTVar var $
-- ( unNodeStory
-- . at listId . _Just
-- . a_state
-- . at ngramsType
-- %~ (\mns' -> case mns' of
-- Nothing -> Just ns
-- Just ns' -> Just $ ns <> ns')
-- ) nls
-- saveNodeStory
newNgramsFromNgramsStatePatch :: NgramsStatePatch' -> [Ngrams]
......@@ -439,7 +434,7 @@ matchingNode :: Maybe ListType
-> (NgramsTerm -> Bool)
-> Tree NgramsElement
-> Bool
matchingNode listType minSize maxSize searchQuery (Node inputNode children) =
matchingNode listType minSize maxSize searchFn (Node inputNode children) =
let nodeSize = inputNode ^. ne_size
matchesListType = maybe (const True) (==) listType
respectsMinSize = maybe (const True) ((<=) . getMinSize) minSize
......@@ -448,66 +443,13 @@ matchingNode listType minSize maxSize searchQuery (Node inputNode children) =
in respectsMinSize nodeSize
&& respectsMaxSize nodeSize
-- Search for the query either in the root or in the children.
&& (searchQuery (inputNode ^. ne_ngrams) || any (matchingNode listType minSize maxSize searchQuery) children)
&& (searchFn (inputNode ^. ne_ngrams) || any (matchingNode listType minSize maxSize searchFn) children)
&& matchesListType (inputNode ^. ne_list)
-- | Errors returned by 'buildForest'.
data BuildForestError
= -- We found a loop, something that shouldn't normally happen if the calling
-- code is correct by construction, but if that does happen, the value will
-- contain the full path to the cycle.
BFE_loop_detected !(Set VisitedNode)
deriving (Show, Eq)
renderLoop :: Set VisitedNode -> T.Text
renderLoop = T.intercalate " -> " . map (unNgramsTerm . _vn_term) . Set.toAscList
-- | Keeps track of the relative order in which visited a node, to be able to print cycles.
data VisitedNode =
VN { _vn_position :: !Int, _vn_term :: !NgramsTerm }
deriving (Show)
instance Eq VisitedNode where
(VN _ t1) == (VN _ t2) = t1 == t2
instance Ord VisitedNode where
compare (VN _ t1) (VN _ t2) = t1 `compare` t2
type TreeNode = (NgramsTerm, NgramsElement)
-- | Version of 'buildForest' specialised over the 'NgramsElement' as the values of the tree.
-- We can't use a single function to \"rule them all\" because the 'NgramsRepoElement', that
-- the 'NodeStory' uses does not have an 'ngrams' we can use as the key when building and
-- destroying a forest.
-- /IMPORTANT/: This functions returns an error in case we found a loop.
buildForest :: Map NgramsTerm NgramsElement -> Either BuildForestError (Forest NgramsElement)
buildForest mp = fmap (map (fmap snd)) . unfoldForestM unfoldNode $ Map.toList mp
where
unfoldNode :: TreeNode -> Either BuildForestError (TreeNode, [TreeNode])
unfoldNode (n, el) = flip evalState (1 :: Int, mempty) . runExceptT $ do
let initialChildren = getChildren (mSetToList $ _ne_children el)
go initialChildren *> pure (mkTreeNode (n, el))
where
go :: [ NgramsElement ]
-> ExceptT BuildForestError (State (Int, Set VisitedNode)) ()
go [] = pure ()
go (x:xs) = do
(pos, visited) <- get
let nt = _ne_ngrams x
case Set.member (VN pos nt) visited of
True -> throwError $ BFE_loop_detected visited
False -> do
put (pos + 1, Set.insert (VN (pos + 1) nt) visited)
go (getChildren (mSetToList $ _ne_children x) <> xs)
mkTreeNode :: TreeNode -> (TreeNode, [TreeNode])
mkTreeNode (k, el) = ((k, el), mapMaybe findChildren $ mSetToList (el ^. ne_children))
findChildren :: NgramsTerm -> Maybe TreeNode
findChildren t = Map.lookup t mp <&> \el -> (t, el)
getChildren :: [NgramsTerm] -> [NgramsElement]
getChildren = mapMaybe (`Map.lookup` mp)
buildForest = fmap (map (fmap snd)) . NodeStory.buildForest
-- | Folds an Ngrams forest back to a table map.
-- This function doesn't aggregate information, but merely just recostructs the original
......
......@@ -40,10 +40,10 @@ TODO:
- charger les listes
-}
{-# LANGUAGE Arrows #-}
{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE QuasiQuotes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TupleSections #-}
module Gargantext.Core.NodeStory
( module Gargantext.Core.NodeStory.Types
......@@ -62,14 +62,18 @@ module Gargantext.Core.NodeStory
, fixNodeStoryVersions
, getParentsChildren
-- * Operations on trees and forests
, TreeNode
, BuildForestError(..)
, VisitedNode(..)
, buildForest
, pruneForest
) where
import Control.Lens ((%~), non, _Just, at, over, Lens')
import Control.Lens ((%~), non, _Just, at, over, Lens', (#))
import Data.ListZipper
import Data.Map.Strict qualified as Map
import Data.Set qualified as Set
import Data.Tree
import Database.PostgreSQL.Simple qualified as PGS
import Database.PostgreSQL.Simple.SqlQQ (sql)
import Database.PostgreSQL.Simple.ToField qualified as PGS
......@@ -77,11 +81,10 @@ import Gargantext.API.Ngrams.Types
import Gargantext.Core.NodeStory.DB
import Gargantext.Core.NodeStory.Types
import Gargantext.Core.Text.Ngrams qualified as Ngrams
import Gargantext.Database.Admin.Types.Node ( ListId, NodeId(..) )
import Gargantext.Database.Admin.Config ()
import Gargantext.Database.Admin.Types.Node ( ListId, NodeId(..) )
import Gargantext.Database.Prelude
import Gargantext.Prelude hiding (to)
import Data.Tree
class HasNgramChildren e where
ngramsElementChildren :: Lens' e (MSet NgramsTerm)
......@@ -109,35 +112,60 @@ instance HasNgramParent NgramsElement where
-- piece of a data structure.
type ArchiveStateForest = ListZipper (Tree (NgramsTerm, NgramsRepoElement))
buildForestsFromArchiveState :: NgramsState' -> Map Ngrams.NgramsType (Forest (NgramsTerm, NgramsRepoElement))
buildForestsFromArchiveState = Map.map buildForest
type TreeNode e = (NgramsTerm, e)
destroyArchiveStateForest :: Map Ngrams.NgramsType (Forest (NgramsTerm, NgramsRepoElement)) -> NgramsState'
buildForestsFromArchiveState :: NgramsState'
-> Either BuildForestError (Map Ngrams.NgramsType (Forest (TreeNode NgramsRepoElement)))
buildForestsFromArchiveState = traverse buildForest
destroyArchiveStateForest :: Map Ngrams.NgramsType (Forest (TreeNode NgramsRepoElement)) -> NgramsState'
destroyArchiveStateForest = Map.map destroyForest
-- | Builds an ngrams forest from the input ngrams table map.
buildForest :: forall e. HasNgramChildren e => Map NgramsTerm e -> Forest (NgramsTerm, e)
buildForest mp = unfoldForest mkTreeNode (Map.toList mp)
buildForest :: forall e. HasNgramChildren e => Map NgramsTerm e -> Either BuildForestError (Forest (TreeNode e))
buildForest mp = unfoldForestM unfoldNode $ Map.toList mp
where
unfoldNode :: TreeNode e -> Either BuildForestError (TreeNode e, [TreeNode e])
unfoldNode (n, el) = flip evalState (1 :: Int, mempty) . runExceptT $ do
let initialChildren = getChildren (mSetToList $ el ^. ngramsElementChildren)
go initialChildren *> pure (mkTreeNode (n, el))
where
mkTreeNode :: (NgramsTerm, e) -> ((NgramsTerm, e), [(NgramsTerm, e)])
-- This function is quite simple: the internal 'State' keeps track of the current
-- position of the visit, and if we discover a term we already seen before, we throw
-- an error, otherwise we store it in the state at the current position and carry on.
go :: [ TreeNode e ] -> ExceptT BuildForestError (State (Int, Set VisitedNode)) ()
go [] = pure ()
go (x:xs) = do
(!pos, !visited) <- get
let nt = fst x
case Set.member (VN pos nt) visited of
True -> throwError $ BFE_loop_detected visited
False -> do
put (pos + 1, Set.insert (VN (pos + 1) nt) visited)
go (getChildren (mSetToList $ snd x ^. ngramsElementChildren) <> xs)
mkTreeNode :: TreeNode e -> (TreeNode e, [TreeNode e])
mkTreeNode (k, el) = ((k, el), mapMaybe findChildren $ mSetToList (el ^. ngramsElementChildren))
findChildren :: NgramsTerm -> Maybe (NgramsTerm, e)
findChildren :: NgramsTerm -> Maybe (TreeNode e)
findChildren t = Map.lookup t mp <&> \el -> (t, el)
getChildren :: [NgramsTerm] -> [TreeNode e]
getChildren = mapMaybe (\t -> (t,) <$> Map.lookup t mp)
-- | Folds an Ngrams forest back to a table map.
-- This function doesn't aggregate information, but merely just recostructs the original
-- map without loss of information. To perform operations on the forest, use the appropriate
-- functions.
destroyForest :: Forest (NgramsTerm, NgramsRepoElement) -> Map NgramsTerm NgramsRepoElement
destroyForest :: Forest (TreeNode NgramsRepoElement) -> Map NgramsTerm NgramsRepoElement
destroyForest f = Map.fromList . map (foldTree destroyTree) $ f
where
destroyTree :: (NgramsTerm, NgramsRepoElement)
-> [(NgramsTerm, NgramsRepoElement)]
-> (NgramsTerm, NgramsRepoElement)
destroyTree :: TreeNode NgramsRepoElement
-> [TreeNode NgramsRepoElement]
-> TreeNode NgramsRepoElement
destroyTree (k, rootEl) childrenEl = (k, squashElements rootEl childrenEl)
squashElements :: e -> [(NgramsTerm, e)] -> e
squashElements :: e -> [TreeNode e] -> e
squashElements r _ = r
-- | Prunes the input 'Forest' of 'NgramsElement' by keeping only the roots, i.e. the
......@@ -357,17 +385,14 @@ getParentsChildren ns = (nsParents, nsChildren)
------------------------------------
mkNodeStoryEnv :: NodeStoryEnv err
mkNodeStoryEnv :: HasNodeStoryError err => NodeStoryEnv err
mkNodeStoryEnv = do
let saver_immediate nId a = do
-- |NOTE Fixing a_state is kinda a hack. We shouldn't land
-- |with bad state in the first place.
upsertNodeStories nId $
a & a_state %~ (
destroyArchiveStateForest
. fixChildrenWithNoParent
. buildForestsFromArchiveState
)
forests <- dbCheckOrFail (first (\e -> _NodeStoryError # NodeStoryUpsertFailed e) $ buildForestsFromArchiveState $ a ^. a_state)
upsertNodeStories nId $ do
a & a_state .~ (destroyArchiveStateForest . fixChildrenWithNoParent $ forests)
let archive_saver_immediate nId a = do
insertNodeArchiveHistory nId (a ^. a_version) $ reverse $ a ^. a_history
pure $ a & a_history .~ []
......
......@@ -8,10 +8,9 @@ Stability : experimental
Portability : POSIX
-}
{-# LANGUAGE Arrows #-}
{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE QuasiQuotes #-}
{-# LANGUAGE TemplateHaskell #-}
{-# LANGUAGE ViewPatterns #-}
module Gargantext.Core.NodeStory.Types
( HasNodeStory
......@@ -42,25 +41,35 @@ module Gargantext.Core.NodeStory.Types
, combineState
, ArchiveState
, ArchiveStateSet
, ArchiveStateList )
, ArchiveStateList
-- * Errors
, HasNodeStoryError(..)
, NodeStoryError(..)
, BuildForestError(..)
, VisitedNode(..)
, renderLoop
)
where
import Codec.Serialise.Class ( Serialise )
import Control.Lens (Getter, Lens')
import Control.Lens (Getter, Lens', Prism', prism')
import Data.Aeson hiding ((.=), decode)
import Data.Map.Strict qualified as Map
import Data.Profunctor.Product.TH (makeAdaptorAndInstance)
import Data.Set qualified as Set
import Database.PostgreSQL.Simple.FromField (FromField(fromField), fromJSONField)
import Gargantext.API.Ngrams.Types
import Gargantext.Database.Admin.Types.Node ( NodeId(..) )
import Gargantext.Core.Text.Ngrams qualified as Ngrams
import Gargantext.Core.Utils.Prefix (unPrefix)
import Gargantext.Database.Admin.Config ()
import Gargantext.Database.Admin.Types.Node ( NodeId(..) )
import Gargantext.Database.Prelude
import Gargantext.Database.Query.Table.Node.Error (HasNodeError())
import Gargantext.Prelude hiding (to)
import Gargantext.Utils.Jobs.Error
import Opaleye (DefaultFromField(..), SqlJsonb, fromPGSFromField)
import qualified Data.Text as T
------------------------------------------------------------------------
......@@ -183,7 +192,47 @@ $(makeAdaptorAndInstance "pNodeArchiveStory" ''NodeStoryArchivePoly)
type ArchiveList = Archive NgramsState' NgramsStatePatch'
-- | Errors returned by 'buildForest'.
data BuildForestError
= -- We found a loop, something that shouldn't normally happen if the calling
-- code is correct by construction, but if that does happen, the value will
-- contain the full path to the cycle.
BFE_loop_detected !(Set VisitedNode)
deriving (Show, Eq)
instance ToHumanFriendlyError BuildForestError where
mkHumanFriendly (BFE_loop_detected visited)
= "Loop detected in terms: " <> renderLoop visited
renderLoop :: Set VisitedNode -> T.Text
renderLoop (sortBy (comparing _vn_position) . Set.toList -> visited) = case visited of
[] -> mempty
(x : _) ->
let cycleWithoutRecursiveKnot = T.intercalate " -> " . map (unNgramsTerm . _vn_term) $ visited
-- Pretty print the first visited node last, so that the user can "see" the full recursive knot.
in cycleWithoutRecursiveKnot <> " -> " <> (unNgramsTerm . _vn_term $ x)
-- | Keeps track of the relative order in which visited a node, to be able to print cycles.
data VisitedNode =
VN { _vn_position :: !Int, _vn_term :: !NgramsTerm }
deriving (Show)
-- /NOTA BENE/: It's important to use this custom instance for the loop detector
-- to work correctly. If we stop comparing on the terms the loop detector .. will loop.
instance Eq VisitedNode where
(VN _ t1) == (VN _ t2) = t1 == t2
-- /NOTA BENE/: Same proviso as for the 'Eq' instance.
instance Ord VisitedNode where
compare (VN _ t1) (VN _ t2) = t1 `compare` t2
data NodeStoryError =
NodeStoryUpsertFailed BuildForestError
deriving (Show, Eq)
instance ToHumanFriendlyError NodeStoryError where
mkHumanFriendly e = case e of
NodeStoryUpsertFailed be -> mkHumanFriendly be
------------------------------------------------------------------------
data NodeStoryEnv err = NodeStoryEnv
......@@ -195,6 +244,12 @@ data NodeStoryEnv err = NodeStoryEnv
-- , _nse_lock :: !FileLock -- TODO (it depends on the option: if with database or file only)
}
class HasNodeStoryError e where
_NodeStoryError :: Prism' e NodeStoryError
instance HasNodeStoryError NodeStoryError where
_NodeStoryError = prism' identity Just
type HasNodeStory env err m = ( IsDBCmd env err m, HasNodeStoryEnv env err, HasNodeError err)
class HasNodeStoryEnv env err where
......
......@@ -40,6 +40,7 @@ import Text.Read (read)
data NgramsType = Authors | Institutes | Sources | NgramsTerms
deriving (Eq, Show, Read, Ord, Enum, Bounded, Generic)
instance NFData NgramsType
instance Serialise NgramsType
instance FromJSON NgramsType
where
......
......@@ -143,7 +143,7 @@ notifyJobFailed env (W.State { name }) bm exc = do
let ji = JobInfo { _ji_message_id = messageId bm
, _ji_mNode_id = getWorkerMNodeId job }
let jh = WorkerJobHandle { _w_job_info = ji }
runWorkerMonad env $ markFailed (Just $ UnsafeMkHumanFriendlyErrorText $ "Worker job failed: " <> show exc) jh
runWorkerMonad env $ markFailed (Just $ UnsafeMkHumanFriendlyErrorText $ T.pack $ displayException exc) jh
notifyJobKilled :: (HasWorkerBroker, HasCallStack)
=> WorkerEnv
......
......@@ -35,6 +35,7 @@ module Gargantext.Database.Transactional (
-- * Throwing and catching errors (which allows rollbacks)
, dbFail
, dbCheckOrFail
, catchDBTxError
, handleDBTxError
) where
......@@ -335,3 +336,7 @@ mkOpaDelete a = DBTx $ liftF (OpaDelete a id)
dbFail :: err -> DBTx err r b
dbFail = DBTx . liftF . DBFail
dbCheckOrFail :: Either err a -> DBTx err r a
dbCheckOrFail (Left e) = DBTx . liftF . DBFail $ e
dbCheckOrFail (Right r) = DBTx $ pure r
......@@ -31,6 +31,7 @@ module Test.API.Routes (
, get_corpus_sqlite_export
, addTeamMember
, importCorpus
, get_list_json
) where
import Data.Text.Encoding qualified as TE
......@@ -39,13 +40,13 @@ import Gargantext.API.Admin.Auth.Types (AuthRequest, AuthResponse, Token)
import Gargantext.API.Errors
import Gargantext.API.HashedResponse (HashedResponse)
import Gargantext.API.Ngrams.List.Types (WithJsonFile, WithTextFile)
import Gargantext.API.Ngrams.Types ( NgramsTable, NgramsTablePatch, OrderBy, TabType, Versioned, VersionedWithCount )
import Gargantext.API.Ngrams.Types
import Gargantext.API.Node.Corpus.Export.Types (CorpusSQLite)
import Gargantext.API.Node.Share.Types (ShareNodeParams(..))
import Gargantext.API.Routes.Client
import Gargantext.API.Routes.Named
import Gargantext.API.Routes.Named.Corpus (CorpusExportAPI(corpusSQLiteEp))
import Gargantext.API.Routes.Named.List (updateListJSONEp, updateListTSVEp)
import Gargantext.API.Routes.Named.List (updateListJSONEp, updateListTSVEp, listJSONEp, getListEp)
import Gargantext.API.Routes.Named.Node hiding (treeAPI)
import Gargantext.API.Routes.Named.Private hiding (tableNgramsAPI)
import Gargantext.API.Routes.Named.Publish (PublishAPI(..), PublishRequest(..))
......@@ -401,3 +402,21 @@ importCorpus (toServantToken -> token) corpusId params =
& ($ corpusId)
& workerAPIPost
& (\submitForm -> submitForm params)
get_list_json :: Token
-> ListId
-> ClientM (Headers '[Header "Content-Disposition" Text] NgramsList)
get_list_json (toServantToken -> token) lId =
clientRoutes & apiWithCustomErrorScheme
& ($ GES_new)
& backendAPI
& backendAPI'
& mkBackEndAPI
& gargAPIVersion
& gargPrivateAPI
& mkPrivateAPI
& ($ token)
& listGetAPI
& getListEp
& ($ lId)
& listJSONEp
......@@ -66,9 +66,10 @@ import Network.Wai.Handler.Warp qualified as Wai
import Paths_gargantext (getDataFileName)
import Prelude qualified
import Servant.Client.Streaming
import Servant.API qualified as Servant
import System.FilePath
import Test.API.Prelude (checkEither, newCorpusForUser, newPrivateFolderForUser, alice)
import Test.API.Routes (mkUrl, gqlUrl, get_table_ngrams, put_table_ngrams, toServantToken, clientRoutes, get_table, update_node, add_form_to_list, add_tsv_to_list)
import Test.API.Routes (mkUrl, gqlUrl, get_table_ngrams, put_table_ngrams, toServantToken, clientRoutes, get_table, update_node, add_form_to_list, add_tsv_to_list, get_list_json)
import Test.API.Setup (withTestDBAndPort, dbEnvSetup, SpecContext (..))
import Test.Database.Types
import Test.Hspec
......@@ -77,6 +78,10 @@ import Test.Hspec.Wai.JSON (json)
import Test.Types (JobPollHandle(..))
import Test.Utils (pollUntilWorkFinished, protectedJSON, withValidLogin, isJobFinished)
import Text.Printf (printf)
import qualified Data.Text.Encoding as TE
import qualified Data.ByteString as BIO
import Control.Lens (view)
import Gargantext.API.Admin.Orchestrator.Types
uploadJSONList :: LogConfig
......@@ -87,25 +92,24 @@ uploadJSONList :: LogConfig
-> ClientEnv
-> WaiSession () ListId
uploadJSONList log_cfg port token cId pathToNgrams clientEnv = do
simpleNgrams' <- liftIO (BIO.readFile =<< getDataFileName pathToNgrams)
uploadJSONListBS log_cfg port token cId simpleNgrams' clientEnv
uploadJSONListBS :: LogConfig
-> Wai.Port
-> Token
-> CorpusId
-> ByteString
-> ClientEnv
-> WaiSession () ListId
uploadJSONListBS log_cfg port token cId blob clientEnv = do
([listId] :: [NodeId]) <- protectedJSON token "POST" (mkUrl port ("/node/" <> build cId)) [aesonQQ|{"pn_typename":"NodeList","pn_name":"Testing"}|]
-- Upload the JSON doc
simpleNgrams' <- liftIO (TIO.readFile =<< getDataFileName pathToNgrams)
-- let (Just simpleNgrams) = JSON.decode $ BSL.fromStrict $ encodeUtf8 simpleNgrams'
-- let jsonFileFormData = [ (T.pack "_wjf_data", simpleNgrams)
-- , ("_wjf_filetype", "JSON")
-- , ("_wjf_name", "simple_ngrams.json")
-- ]
let params = WithJsonFile { _wjf_data = simpleNgrams'
let params = WithJsonFile { _wjf_data = TE.decodeUtf8 blob
, _wjf_name = "simple_ngrams.json" }
-- let url = "/lists/" +|listId|+ "/add/form/async"
-- let mkPollUrl j = "/corpus/" +|listId|+ "/add/form/async/" +|_jph_id j|+ "/poll?limit=1"
-- (j :: JobPollHandle) <- postJSONUrlEncoded token (mkUrl port url) (urlEncodeFormStable $ toForm jsonFileFormData)
-- j' <- pollUntilFinished token port mkPollUrl j
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' `shouldSatisfy` isJobFinished
pure listId
-- | Compares the ngrams returned via the input IO action with the ones provided as
......@@ -498,6 +502,197 @@ tests = sequential $ aroundAll withTestDBAndPort $ beforeAllWith dbEnvSetup $ do
) clientEnv
length (_ne_occurrences fortran_ngram') `shouldBe` 1
describe "Importing and exporting nested terms" $ do
-- As per #498, we want to test that even in the present of deep
-- nested hierarchy of ngrams, we can import and export them and we should
-- end up with the ngrams hierarchy we started from. In other terms, a
-- roundtrip property should be satisfied.
it "should roundtrip for JSON" $ \(SpecContext testEnv port app _) -> do
cId <- newCorpusForUser testEnv "alice"
cId2 <- newCorpusForUser testEnv "alice"
let log_cfg = (test_config testEnv) ^. gc_logging
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
-- Import the initial terms
let (Right initialTerms) = JSON.eitherDecode @NgramsList $ [json|
{
"Authors": {
"version": 11,
"data": {}
},
"Institutes": {
"version": 11,
"data": {}
},
"Sources": {
"version": 11,
"data": {}
},
"NgramsTerms": {
"version": 11,
"data": {
"boss ds-1": {
"root": "guitar effects",
"parent": "distortions",
"size": 1,
"list": "MapTerm",
"children": []
},
"distortions": {
"root": "guitar effects",
"parent": "guitar effects",
"size": 1,
"list": "MapTerm",
"children": [
"boss ds-1",
"rat"
]
},
"guitar effects": {
"size": 1,
"list": "MapTerm",
"children": [
"distortions",
"overdrives"
]
},
"guitar pedals": {
"root": "guitar effects",
"parent": "overdrives",
"size": 1,
"list": "MapTerm",
"children": [
"tube screamers"
]
},
"overdrives": {
"root": "guitar effects",
"parent": "guitar effects",
"size": 1,
"list": "MapTerm",
"children": [
"guitar pedals"
]
},
"rat": {
"root": "guitar effects",
"parent": "distortions",
"size": 1,
"list": "MapTerm",
"children": []
},
"tube screamers": {
"root": "guitar effects",
"parent": "guitar pedals",
"size": 1,
"list": "MapTerm",
"children": []
}
}
}
}
|]
listId <- uploadJSONListBS log_cfg port token cId (BL.toStrict $ JSON.encode initialTerms) clientEnv
-- Export them.
exported <- Servant.getResponse <$> (checkEither $ liftIO $ runClientM (get_list_json token listId) clientEnv)
let initialNgrams = view v_data <$> Map.lookup NgramsTerms initialTerms
let exportedNgrams = view v_data <$> Map.lookup NgramsTerms exported
liftIO $ exportedNgrams `shouldBe` initialNgrams
-- now we import them again, but this time on a different corpus, so that we don't
-- get conflicts and the occurrences count won't get messed up. Dealing with conflicts
-- is a separate type of test.
listId2 <- uploadJSONListBS log_cfg port token cId2 (BL.toStrict $ JSON.encode exported) clientEnv
-- Export them again.
exported2 <- Servant.getResponse <$> (checkEither $ liftIO $ runClientM (get_list_json token listId2) clientEnv)
let exportedNgrams2 = view v_data <$> Map.lookup NgramsTerms exported2
liftIO $ exportedNgrams `shouldBe` exportedNgrams2
-- We test that if we try to import terms which, when merged with the existing,
-- would cause a loop, GGTX is capable of rejecting the request.
it "refuses to import terms which will lead to a loop" $ \(SpecContext testEnv port app _) -> do
cId <- newCorpusForUser testEnv "alice"
let log_cfg = (test_config testEnv) ^. gc_logging
withApplication app $ do
withValidLogin port "alice" (GargPassword "alice") $ \clientEnv token -> do
-- Import the initial terms
let (Right initialTerms) = JSON.eitherDecode @NgramsList $ [json|
{
"Authors": {
"version": 1,
"data": {}
},
"Institutes": {
"version": 1,
"data": {}
},
"Sources": {
"version": 1,
"data": {}
},
"NgramsTerms": {
"version": 1,
"data": {
"foo": {
"size": 1,
"list": "MapTerm",
"children": ["bar"]
},
"bar": {
"root": "foo",
"parent": "foo",
"size": 1,
"list": "MapTerm",
"children": [
"quux"
]
},
"quux": {
"size": 1,
"list": "MapTerm",
"children": []
} } } }
|]
listId <- uploadJSONListBS log_cfg port token cId (BL.toStrict $ JSON.encode initialTerms) clientEnv
let (Right secondBatch) = JSON.eitherDecode @NgramsList $ [json|
{
"Authors": {
"version": 1,
"data": {}
},
"Institutes": {
"version": 1,
"data": {}
},
"Sources": {
"version": 1,
"data": {}
},
"NgramsTerms": {
"version": 1,
"data": {
"bar": {
"size": 1,
"list": "MapTerm",
"children": ["foo"]
}
} } }
|]
let params = WithJsonFile { _wjf_data = TE.decodeUtf8 (BL.toStrict $ JSON.encode secondBatch)
, _wjf_name = "simple_ngrams.json"
}
ji <- checkEither $ liftIO $ runClientM (add_form_to_list token listId params) clientEnv
ji' <- pollUntilWorkFinished log_cfg port ji
-- Unfortunately we don't have a better way then to match on the stringified exception, sigh.
case _scst_events ji' of
Just [ScraperEvent{..}]
| Just msg <- _scev_message
-> liftIO $ msg `shouldSatisfy` \txt -> "Loop detected in terms: foo -> bar -> foo" `T.isInfixOf` txt
| otherwise
-> fail "No suitable message in ScraperEvent."
_ -> fail "Expected job to fail, but it didn't"
createDocsList :: FilePath
-> TestEnv
-> Int
......
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