{-|
Module      : Gargantext.API.Node.Corpus.New
Description : New corpus API
Copyright   : (c) CNRS, 2017-Present
License     : AGPL + CECILL v3
Maintainer  : team@gargantext.org
Stability   : experimental
Portability : POSIX

New corpus means either:
- new corpus
- new data in existing corpus
-}

{-# LANGUAGE TemplateHaskell    #-}
{-# LANGUAGE TypeOperators      #-}
{-# LANGUAGE ViewPatterns       #-}

module Gargantext.API.Node.Corpus.New
      where


import Conduit ((.|), yieldMany, mapMC, mapC, transPipe)
import Control.Lens ( view, non )
import Data.ByteString.Base64 qualified as BSB64
import Data.Conduit.Internal (zipSources)
import Data.Swagger ( ToSchema(..) )
import Data.Text.Encoding qualified as TE
import Data.Text qualified as T
import Gargantext.API.Admin.Orchestrator.Types qualified as API
import Gargantext.API.Ngrams (commitStatePatch, Versioned(..))
import Gargantext.API.Node.Corpus.New.Types ( FileFormat(..), FileType(..) )
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.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.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)
import Gargantext.Database.Action.Node (mkNodeWithParent)
import Gargantext.Database.Action.User (getUserId)
import Gargantext.Database.Admin.Types.Hyperdata.Document ( ToHyperdataDocument(..) )
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 (IsDBCmd)
import Gargantext.Database.Query.Table.Node.Error (HasNodeError)
import Gargantext.Database.Query.Table.Node (getNodeWith, getOrMkList)
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.Monad (JobHandle, MonadJobStatus(..))

------------------------------------------------------------------------
{-
data Query = Query { query_query      :: Text
                   , query_node_id    :: Int
                   , query_lang       :: Lang
                   , query_databases  :: [DataOrigin]
                   }
                   deriving (Eq, Generic)

deriveJSON (unPrefix "query_") 'Query

instance Arbitrary Query where
    arbitrary = elements [ Query q n la fs
                         | q <- ["honeybee* AND collapse"
                                ,"covid 19"
                                ]
                         , n <- [0..10]
                         , la <- allLangs
                         , fs <- take 3 $ repeat allDataOrigins
                         ]

instance ToSchema Query where
  declareNamedSchema = genericDeclareNamedSchema (unPrefixSwagger "query_")
-}

------------------------------------------------------------------------

{-
type Api = PostApi
        :<|> GetApi

type PostApi = Summary "New Corpus endpoint"
             :> ReqBody '[JSON] Query
             :> Post    '[JSON] CorpusId
type GetApi = Get '[JSON] ApiInfo
-}

-- | TODO manage several apis
-- TODO-ACCESS
-- TODO this is only the POST
{-
api :: (FlowCmdM env err m) => UserId -> Query -> m CorpusId
api uid (Query q _ as) = do
  cId <- case head as of
    Nothing      -> flowCorpusSearchInDatabase (UserDBId uid) EN q
    Just API.All -> flowCorpusSearchInDatabase (UserDBId uid) EN q
    Just a   -> do
      docs <- liftBase $ API.get a q (Just 1000)
      cId' <- flowCorpus (UserDBId uid) (Left q) (Multi EN) [docs]
      pure cId'

  pure cId
-}

------------------------------------------------
-- TODO use this route for Client implementation
data ApiInfo = ApiInfo { api_info :: [API.ExternalAPIs]}
  deriving (Generic)

deriveJSON (unPrefix "") 'ApiInfo

instance ToSchema ApiInfo

info :: ApiInfo
info = ApiInfo API.externalAPIs

{-
type AddWithFile = Summary "Add with MultipartData to corpus endpoint"
   :> "corpus"
     :> Capture "corpus_id" CorpusId
   :> "add"
   :> "file"
     :> MultipartForm Mem (MultipartData Mem)
     :> QueryParam "fileType"  FileType
   :> "async"
     :> AsyncJobs JobLog '[JSON] () JobLog
-}


------------------------------------------------------------------------
-- TODO WithQuery also has a corpus id


addToCorpusWithQuery :: ( FlowCmdM env err m
                        , MonadJobStatus m
                        , HasNodeStoryImmediateSaver env
                        , HasNodeArchiveStoryImmediateSaver env
                        )
                       => User
                       -> CorpusId
                       -> WithQuery
                       -> Maybe API.Limit
                       -> JobHandle m
                       -> m ()
addToCorpusWithQuery user cid (WithQuery { _wq_query = q
                                         , _wq_datafield = datafield
                                         , _wq_lang = l
                                         , _wq_flowListWith = flw }) maybeLimit jobHandle = do
  -- TODO ...
  $(logLocM) DEBUG $ "[addToCorpusWithQuery] cid " <> show cid
  $(logLocM) DEBUG $ "[addToCorpusWithQuery] datafield  " <> show datafield
  $(logLocM) DEBUG $ "[addToCorpusWithQuery] flowListWith " <> show flw

  $(logLocM) DEBUG $ "[addToCorpusWithQuery] addLanguageToCorpus " <> show cid <> ", " <> show l
  addLanguageToCorpus cid l
  $(logLocM) DEBUG "[addToCorpusWithQuery] after addLanguageToCorpus"
  
  case datafield of
    Web -> do
      $(logLocM) DEBUG $ "[addToCorpusWithQuery] processing web request " <> show datafield

      markStarted 1 jobHandle

      _ <- triggerSearxSearch user cid q l jobHandle

      markComplete jobHandle

    _ -> do
      markStarted 3 jobHandle

      -- TODO add cid
      -- TODO if cid is folder -> create Corpus
      --      if cid is corpus -> add to corpus
      --      if cid is root   -> create corpus in Private
      $(logLocM) DEBUG $ "[addToCorpusWithQuery] getDataText with query: " <> show q
      let db = datafield2origin datafield
      -- mPubmedAPIKey <- getUserPubmedAPIKey user
      -- printDebug "[addToCorpusWithQuery] mPubmedAPIKey" mPubmedAPIKey
      eTxt <- getDataText db (Multi l) q maybeLimit

      -- printDebug "[G.A.N.C.New] lTxts" lTxts
      case eTxt of
        Right txt -> do
          -- TODO Sum lenghts of each txt elements
          $(logLocM) DEBUG "[addToCorpusWithQuery] Processing dataText results"
          markProgress 1 jobHandle

          corpusId <- flowDataText user txt (Multi l) cid (Just flw) jobHandle
          $(logLocM) DEBUG $ "[addToCorpusWithQuery] corpus id " <> show corpusId

          _ <- commitCorpus cid user
          $(logLocM) DEBUG $ "[addToCorpusWithQuery] corpus comitted"

          -- printDebug "sending email" ("xxxxxxxxxxxxxxxxxxxxx" :: Text)
          sendMail user
          -- TODO ...
          markComplete jobHandle

        Left err -> do
          -- printDebug "Error: " err
          $(logLocM) ERROR $ "[addToCorpusWithQuery] error: " <> show err -- log the full error
          markFailed (Just err) jobHandle

addToCorpusWithForm :: ( FlowCmdM env err m
                       , MonadJobStatus m
                       , HasNodeStoryImmediateSaver env
                       , HasNodeArchiveStoryImmediateSaver env
                       )
                    => User
                    -> CorpusId
                    -> NewWithForm
                    -> JobHandle m
                    -> m ()
addToCorpusWithForm user cid nwf jobHandle = do
  -- printDebug "[addToCorpusWithForm] Parsing corpus: " cid
  -- printDebug "[addToCorpusWithForm] fileType" ft
  -- printDebug "[addToCorpusWithForm] fileFormat" ff

  let l = nwf ^. wf_lang . non defaultLanguage
  addLanguageToCorpus cid l

  limit' <- view $ hasConfig . gc_jobs . jc_max_docs_parsers
  let limit = fromIntegral limit' :: Integer
  let
    parseC = case (nwf ^. wf_filetype) of
      TSV       -> Parser.parseFormatC Parser.TsvGargV3
      TSV_HAL   -> Parser.parseFormatC Parser.TsvHal
      Iramuteq  -> Parser.parseFormatC Parser.Iramuteq
      Istex     -> Parser.parseFormatC Parser.Istex
      JSON      -> Parser.parseFormatC Parser.JSON
      PresseRIS -> Parser.parseFormatC Parser.RisPresse
      WOS       -> Parser.parseFormatC Parser.WOS

  -- TODO granularity of the logStatus
  let data' = case (nwf ^. wf_fileformat) of
        Plain -> cs (nwf ^. wf_data)
        ZIP   -> case BSB64.decode $ TE.encodeUtf8 (nwf ^. wf_data) of
          Left err -> panicTrace $ T.pack "[addToCorpusWithForm] error decoding base64: " <> T.pack err
          Right decoded -> decoded
  eDocsC <- liftBase $ parseC (nwf ^. wf_fileformat) data'
  case eDocsC of
    Right (count, docsC) -> do
      -- TODO Add progress (jobStatus) update for docs - this is a
      -- long action

      let docsC' = zipSources (yieldMany [1..]) docsC
                  .| mapMC (\(idx, doc) ->
                        if idx > limit then do
                          --printDebug "[addToCorpusWithForm] number of docs exceeds the limit" (show limit)
                          let panicMsg' = [ "[addToCorpusWithForm] number of docs "
                                          , "exceeds the MAX_DOCS_PARSERS limit ("
                                          , show limit
                                          , ")" ]
                          let panicMsg = T.concat $ T.pack <$> panicMsg'
                          --logStatus $ jobLogFailTotalWithMessage panicMsg jobLog
                          panicTrace panicMsg
                        else
                          pure doc)
                  .| mapC toHyperdataDocument

      --printDebug "Parsing corpus finished : " cid
      --logStatus jobLog2

      --printDebug "Starting extraction     : " cid
      -- TODO granularity of the logStatus
      -- printDebug "flowCorpus with (corpus_id, lang)" (cid, l)

      _cid' <- flowCorpus (MkCorpusUserNormalCorpusIds user [cid])
                          (Multi l)
                          (Just (nwf ^. wf_selection))
                          --(Just $ fromIntegral $ length docs, docsC')
                          (count, transPipe liftBase docsC') -- TODO fix number of docs
                          --(map (map toHyperdataDocument) docs)
                          jobHandle

      _ <- commitCorpus cid user

      -- printDebug "Extraction finished   : " cid
      -- printDebug "sending email" ("xxxxxxxxxxxxxxxxxxxxx" :: Text)
      -- TODO uncomment this
      --sendMail user

      markComplete jobHandle
    Left parseErr -> do
      $(logLocM) ERROR $ "[addToCorpusWithForm] parse error: " <> (Parser._ParseFormatError parseErr)
      markFailed (Just parseErr) jobHandle

{-
addToCorpusWithFile :: FlowCmdM env err m
                    => CorpusId
                    -> MultipartData Mem
                    -> Maybe FileType
                    -> (JobLog -> m ())
                    -> m JobLog
addToCorpusWithFile cid input filetype logStatus = do
  logStatus JobLog { _scst_succeeded = Just 10
                          , _scst_failed    = Just 2
                          , _scst_remaining = Just 138
                          , _scst_events    = Just []
                          }
  printDebug "addToCorpusWithFile" cid
  _h <- postUpload cid filetype input

  pure      JobLog { _scst_succeeded = Just 137
                          , _scst_failed    = Just 13
                          , _scst_remaining = Just 0
                          , _scst_events    = Just []
                          }
-}

addToCorpusWithFile :: (FlowCmdM env err m, MonadJobStatus m)
                    => User
                    -> CorpusId
                    -> NewWithFile
                    -> JobHandle m
                    -> m ()
addToCorpusWithFile user cid nwf@(NewWithFile _d (withDefaultLanguage -> l) fName) jobHandle = do

  addLanguageToCorpus cid l

  $(logLocM) DEBUG $ "[addToCorpusWithFile] Uploading file to corpus: " <> show cid
  markStarted 1 jobHandle

  fPath <- GargDB.writeFile nwf
  $(logLocM) DEBUG $ "[addToCorpusWithFile] File saved as: " <> show fPath

  uId <- getUserId user
  nIds <- mkNodeWithParent NodeFile (Just cid) uId fName

  _ <- case nIds of
    [nId] -> do
        node <- getNodeWith nId (Proxy :: Proxy HyperdataFile)
        let hl = node ^. node_hyperdata
        _ <- updateHyperdata nId $ hl { _hff_name = fName
                                      , _hff_path = T.pack fPath }

        $(logLocM) DEBUG $ "[addToCorpusWithFile] Created node with id: " <> show nId
    _     -> pure ()

  $(logLocM) DEBUG $ "[addToCorpusWithFile] File upload to corpus finished: " <> show cid

  $(logLocM) DEBUG $ "[addToCorpusWithFile] sending email: " <> ("xxxxxxxxxxxxxxxxxxxxx" :: Text)
  sendMail user

  markComplete jobHandle



--- UTILITIES

commitCorpus :: ( IsDBCmd env err m
                , HasNodeStoryEnv env
                , HasNodeError err
                , HasNodeArchiveStoryImmediateSaver env
                , HasNodeStoryImmediateSaver env )
             => ParentId
             -> User
             -> m (Versioned NgramsStatePatch')
commitCorpus cid user = do
  userId <- getUserId user
  listId <- getOrMkList cid userId
  v <- currentVersion listId
  commitStatePatch listId (Versioned v mempty)