1
1
mirror of https://github.com/github/semantic.git synced 2024-12-22 06:11:49 +03:00

Merge pull request #1714 from github/task-in-eff

Task in Eff
This commit is contained in:
Josh Vera 2018-04-05 17:52:08 -04:00 committed by GitHub
commit d20a0c554a
17 changed files with 572 additions and 398 deletions

View File

@ -135,13 +135,16 @@ library
, Rendering.Symbol
, Rendering.TOC
-- High-level flow & operational functionality (logging, stats, etc.)
, Semantic
, Semantic.CLI
, Semantic.Diff
, Semantic.Distribute
, Semantic.IO
, Semantic.Log
, Semantic.Parse
, Semantic.Queue
, Semantic.Stat
, Semantic.Task
, Semantic.Queue
, Semantic.Telemetry
, Semantic.Util
-- Custom Prelude
other-modules: Prologue

View File

@ -151,7 +151,6 @@ instance Members (EvaluatingEffects location term value) effects
instance ( Corecursive term
, Members (EvaluatingEffects location term value) effects
, MonadValue location value (Evaluating location term value effects)
, Recursive term
)
=> MonadAnalysis location term value (Evaluating location term value effects) where

View File

@ -1,9 +1,11 @@
{-# LANGUAGE RankNTypes, ScopedTypeVariables, TypeFamilies #-}
{-# LANGUAGE GADTs, RankNTypes, ScopedTypeVariables, TypeFamilies #-}
{-# OPTIONS_GHC -Wno-redundant-constraints #-} -- For runAnalysis
module Control.Abstract.Analysis
( MonadAnalysis(..)
, liftAnalyze
, runAnalysis
, SomeAnalysis(..)
, runSomeAnalysis
, module X
, Subterm(..)
, SubtermAlgebra
@ -62,3 +64,18 @@ runAnalysis :: ( Effectful m
=> m effects a
-> Final effects a
runAnalysis = X.run
-- | An abstraction over analyses.
data SomeAnalysis m result where
SomeAnalysis :: ( Effectful m
, effects ~ Effects location term value (m effects)
, MonadAnalysis location term value (m effects)
, RunEffects effects a
)
=> m effects a
-> SomeAnalysis m (Final effects a)
-- | Run an abstracted analysis.
runSomeAnalysis :: SomeAnalysis m result -> result
runSomeAnalysis (SomeAnalysis a) = X.run a

View File

@ -11,6 +11,7 @@ module Data.Abstract.Evaluatable
, evaluateModule
, evaluateModules
, evaluatePackage
, evaluatePackageBody
, throwLoadError
, resolve
, listModulesInDir

View File

@ -1,94 +0,0 @@
{-# LANGUAGE DataKinds, DeriveAnyClass, DeriveDataTypeable, GADTs, TypeOperators #-}
module Semantic
( parseBlobs
, parseBlob
, diffBlobPairs
, diffBlobPair
, diffTermPair
) where
import Prologue
import Analysis.ConstructorName (ConstructorName, constructorLabel)
import Analysis.IdentifierName (IdentifierName, identifierLabel)
import Analysis.Declaration (HasDeclaration, declarationAlgebra)
import Analysis.PackageDef (HasPackageDef, packageDefAlgebra)
import Data.Blob
import Data.Diff
import Data.JSON.Fields
import Data.Output
import Data.Record
import Data.Term
import Diffing.Algorithm (Diffable)
import Diffing.Interpreter
import Parsing.Parser
import Rendering.Renderer
import Semantic.Stat as Stat
import Semantic.Task as Task
-- This is the primary interface to the Semantic library which provides two
-- major classes of functionality: semantic parsing and diffing of source code
-- blobs.
--
-- Design goals:
-- - No knowledge of the filesystem or Git.
-- - Built in concurrency where appropriate.
-- - Easy to consume this interface from other application (e.g a cmdline or web server app).
parseBlobs :: Output output => TermRenderer output -> [Blob] -> Task ByteString
parseBlobs renderer blobs = toOutput' <$> distributeFoldMap (parseBlob renderer) blobs
where toOutput' = case renderer of
JSONTermRenderer -> toOutput . renderJSONTerms
SymbolsTermRenderer _ -> toOutput . renderSymbolTerms
_ -> toOutput
-- | A task to parse a 'Blob' and render the resulting 'Term'.
parseBlob :: TermRenderer output -> Blob -> Task output
parseBlob renderer blob@Blob{..}
| Just (SomeParser parser) <- someParser (Proxy :: Proxy '[ConstructorName, HasPackageDef, HasDeclaration, IdentifierName, Foldable, Functor, ToJSONFields1]) <$> blobLanguage
= parse parser blob >>= case renderer of
JSONTermRenderer -> decorate constructorLabel >=> decorate identifierLabel >=> render (renderJSONTerm blob)
SExpressionTermRenderer -> decorate constructorLabel . (Nil <$) >=> render renderSExpressionTerm
TagsTermRenderer -> decorate (declarationAlgebra blob) >=> render (renderToTags blob)
ImportsTermRenderer -> decorate (declarationAlgebra blob) >=> decorate (packageDefAlgebra blob) >=> render (renderToImports blob)
SymbolsTermRenderer fields -> decorate (declarationAlgebra blob) >=> render (renderToSymbols fields blob)
DOTTermRenderer -> render (renderDOTTerm blob)
| otherwise = throwError (SomeException (NoLanguageForBlob blobPath))
newtype NoLanguageForBlob = NoLanguageForBlob FilePath
deriving (Eq, Exception, Ord, Show, Typeable)
diffBlobPairs :: Output output => DiffRenderer output -> [BlobPair] -> Task ByteString
diffBlobPairs renderer blobs = toOutput' <$> distributeFoldMap (diffBlobPair renderer) blobs
where toOutput' = case renderer of
JSONDiffRenderer -> toOutput . renderJSONDiffs
_ -> toOutput
-- | A task to parse a pair of 'Blob's, diff them, and render the 'Diff'.
diffBlobPair :: DiffRenderer output -> BlobPair -> Task output
diffBlobPair renderer blobs
| Just (SomeParser parser) <- someParser (Proxy :: Proxy '[ConstructorName, Diffable, Eq1, GAlign, HasDeclaration, IdentifierName, Show1, ToJSONFields1, Traversable]) <$> effectiveLanguage
= case renderer of
ToCDiffRenderer -> run (\ blob -> parse parser blob >>= decorate (declarationAlgebra blob)) diffTerms renderToCDiff
JSONDiffRenderer -> run ( parse parser >=> decorate constructorLabel >=> decorate identifierLabel) diffTerms renderJSONDiff
SExpressionDiffRenderer -> run ( parse parser >=> decorate constructorLabel . (Nil <$)) diffTerms (const renderSExpressionDiff)
DOTDiffRenderer -> run ( parse parser) diffTerms renderDOTDiff
| otherwise = throwError (SomeException (NoLanguageForBlob effectivePath))
where effectivePath = pathForBlobPair blobs
effectiveLanguage = languageForBlobPair blobs
run :: (Foldable syntax, Functor syntax) => (Blob -> Task (Term syntax ann)) -> (Term syntax ann -> Term syntax ann -> Diff syntax ann ann) -> (BlobPair -> Diff syntax ann ann -> output) -> Task output
run parse diff renderer = do
terms <- bidistributeFor (runJoin blobs) parse parse
time "diff" languageTag $ do
diff <- diffTermPair diff terms
writeStat (Stat.count "diff.nodes" (bilength diff) languageTag)
render (renderer blobs) diff
where
languageTag = languageTagForBlobPair blobs
-- | A task to diff 'Term's, producing insertion/deletion 'Patch'es for non-existent 'Blob's.
diffTermPair :: Functor syntax => Differ syntax ann1 ann2 -> These (Term syntax ann1) (Term syntax ann2) -> Task (Diff syntax ann1 ann2)
diffTermPair _ (This t1 ) = pure (deleting t1)
diffTermPair _ (That t2) = pure (inserting t2)
diffTermPair differ (These t1 t2) = diff differ t1 t2

View File

@ -16,26 +16,27 @@ import Options.Applicative
import Rendering.Renderer
import qualified Paths_semantic as Library (version)
import Semantic.IO (languageForFilePath)
import qualified Semantic.Diff as Semantic (diffBlobPairs)
import qualified Semantic.Log as Log
import qualified Semantic.Parse as Semantic (parseBlobs)
import qualified Semantic.Task as Task
import System.IO (Handle, stdin, stdout)
import qualified Semantic (parseBlobs, diffBlobPairs)
import Text.Read
main :: IO ()
main = customExecParser (prefs showHelpOnEmpty) arguments >>= uncurry Task.runTaskWithOptions
runDiff :: SomeRenderer DiffRenderer -> Either Handle [Both (FilePath, Maybe Language)] -> Task.Task ByteString
runDiff :: SomeRenderer DiffRenderer -> Either Handle [Both (FilePath, Maybe Language)] -> Task.TaskEff ByteString
runDiff (SomeRenderer diffRenderer) = Semantic.diffBlobPairs diffRenderer <=< Task.readBlobPairs
runParse :: SomeRenderer TermRenderer -> Either Handle [(FilePath, Maybe Language)] -> Task.Task ByteString
runParse :: SomeRenderer TermRenderer -> Either Handle [(FilePath, Maybe Language)] -> Task.TaskEff ByteString
runParse (SomeRenderer parseTreeRenderer) = Semantic.parseBlobs parseTreeRenderer <=< Task.readBlobs
-- | A parser for the application's command-line arguments.
--
-- Returns a 'Task' to read the input, run the requested operation, and write the output to the specified output path or stdout.
arguments :: ParserInfo (Log.Options, Task.Task ())
arguments :: ParserInfo (Log.Options, Task.TaskEff ())
arguments = info (version <*> helper <*> ((,) <$> optionsParser <*> argumentsParser)) description
where
version = infoOption versionString (long "version" <> short 'v' <> help "Output the version of the program")

55
src/Semantic/Diff.hs Normal file
View File

@ -0,0 +1,55 @@
{-# LANGUAGE GADTs #-}
module Semantic.Diff where
import Prologue hiding (MonadError(..))
import Analysis.ConstructorName (ConstructorName, constructorLabel)
import Analysis.IdentifierName (IdentifierName, identifierLabel)
import Analysis.Declaration (HasDeclaration, declarationAlgebra)
import Data.Blob
import Data.Diff
import Data.JSON.Fields
import Data.Output
import Data.Record
import Data.Term
import Diffing.Algorithm (Diffable)
import Diffing.Interpreter
import Parsing.Parser
import Rendering.Renderer
import Semantic.IO (NoLanguageForBlob(..))
import Semantic.Stat as Stat
import Semantic.Task as Task
diffBlobPairs :: (Members '[Distribute WrappedTask, Task, Telemetry, Exc SomeException, IO] effs, Output output) => DiffRenderer output -> [BlobPair] -> Eff effs ByteString
diffBlobPairs renderer blobs = toOutput' <$> distributeFoldMap (WrapTask . diffBlobPair renderer) blobs
where toOutput' = case renderer of
JSONDiffRenderer -> toOutput . renderJSONDiffs
_ -> toOutput
-- | A task to parse a pair of 'Blob's, diff them, and render the 'Diff'.
diffBlobPair :: Members '[Distribute WrappedTask, Task, Telemetry, Exc SomeException, IO] effs => DiffRenderer output -> BlobPair -> Eff effs output
diffBlobPair renderer blobs
| Just (SomeParser parser) <- someParser (Proxy :: Proxy '[ConstructorName, Diffable, Eq1, GAlign, HasDeclaration, IdentifierName, Show1, ToJSONFields1, Traversable]) <$> effectiveLanguage
= case renderer of
ToCDiffRenderer -> run (WrapTask . (\ blob -> parse parser blob >>= decorate (declarationAlgebra blob))) diffTerms renderToCDiff
JSONDiffRenderer -> run (WrapTask . ( parse parser >=> decorate constructorLabel >=> decorate identifierLabel)) diffTerms renderJSONDiff
SExpressionDiffRenderer -> run (WrapTask . ( parse parser >=> decorate constructorLabel . (Nil <$))) diffTerms (const renderSExpressionDiff)
DOTDiffRenderer -> run (WrapTask . parse parser) diffTerms renderDOTDiff
| otherwise = throwError (SomeException (NoLanguageForBlob effectivePath))
where effectivePath = pathForBlobPair blobs
effectiveLanguage = languageForBlobPair blobs
run :: (Foldable syntax, Functor syntax) => Members [Distribute WrappedTask, Task, Telemetry, IO] effs => (Blob -> WrappedTask (Term syntax ann)) -> (Term syntax ann -> Term syntax ann -> Diff syntax ann ann) -> (BlobPair -> Diff syntax ann ann -> output) -> Eff effs output
run parse diff renderer = do
terms <- distributeFor blobs parse
time "diff" languageTag $ do
diff <- diffTermPair diff (runJoin terms)
writeStat (Stat.count "diff.nodes" (bilength diff) languageTag)
render (renderer blobs) diff
where
languageTag = languageTagForBlobPair blobs
-- | A task to diff 'Term's, producing insertion/deletion 'Patch'es for non-existent 'Blob's.
diffTermPair :: (Functor syntax, Member Task effs) => Differ syntax ann1 ann2 -> These (Term syntax ann1) (Term syntax ann2) -> Eff effs (Diff syntax ann1 ann2)
diffTermPair _ (This t1 ) = pure (deleting t1)
diffTermPair _ (That t2) = pure (inserting t2)
diffTermPair differ (These t1 t2) = diff differ t1 t2

View File

@ -0,0 +1,55 @@
{-# LANGUAGE GADTs, RankNTypes, TypeOperators, UndecidableInstances #-}
module Semantic.Distribute
( distribute
, distributeFor
, distributeFoldMap
, Distribute
, runDistribute
, Action(..)
) where
import qualified Control.Concurrent.Async as Async
import Control.Monad.Effect hiding (run)
import Control.Monad.Effect.Exception
import Control.Monad.Effect.Run
import Control.Monad.IO.Class
import Control.Parallel.Strategies
import Prologue hiding (MonadError (..))
-- | Distribute a 'Traversable' container of tasks over the available cores (i.e. execute them concurrently), collecting their results.
--
-- This is a concurrent analogue of 'sequenceA'.
distribute :: (Member (Distribute task) effs, Traversable t) => t (task output) -> Eff effs (t output)
distribute = send . Distribute
-- | Distribute the application of a function to each element of a 'Traversable' container of inputs over the available cores (i.e. perform the function concurrently for each element), collecting the results.
--
-- This is a concurrent analogue of 'for' or 'traverse' (with the arguments flipped).
distributeFor :: (Member (Distribute task) effs, Traversable t) => t a -> (a -> task output) -> Eff effs (t output)
distributeFor inputs toTask = distribute (fmap toTask inputs)
-- | Distribute the application of a function to each element of a 'Traversable' container of inputs over the available cores (i.e. perform the function concurrently for each element), combining the results 'Monoid'ally into a final value.
--
-- This is a concurrent analogue of 'foldMap'.
distributeFoldMap :: (Member (Distribute task) effs, Monoid output, Traversable t) => (a -> task output) -> t a -> Eff effs output
distributeFoldMap toTask inputs = fmap fold (distribute (fmap toTask inputs))
-- | Distribute effects run tasks concurrently.
data Distribute task output where
Distribute :: Traversable t => t (task output) -> Distribute task (t output)
-- | Evaluate a 'Distribute' effect concurrently.
runDistribute :: Members '[Exc SomeException, IO] effs => Eff (Distribute task ': effs) a -> Action task -> Eff effs a
runDistribute m action = interpret (\ (Distribute tasks) ->
liftIO (Async.mapConcurrently (runAction action) tasks) >>= either throwError pure . sequenceA . withStrategy (parTraversable (parTraversable rseq))) m
-- | An action evaluating @task@s to some output in 'IO', or failing with an exception.
--
-- This is necessary because GHC wont allow us to use a rank-n quantified type in the third parameter to our instance of 'Run', below.
newtype Action task = Action { runAction :: forall output . task output -> IO (Either SomeException output) }
instance (Members '[Exc SomeException, IO] effects, Run effects result rest) => Run (Distribute task ': effects) result (Action task -> rest) where
run = fmap run . runDistribute

View File

@ -1,4 +1,4 @@
{-# LANGUAGE OverloadedStrings, TypeSynonymInstances, DeriveAnyClass, DuplicateRecordFields, ScopedTypeVariables, TupleSections #-}
{-# LANGUAGE DeriveAnyClass, DeriveDataTypeable, DuplicateRecordFields, GADTs, ScopedTypeVariables, TupleSections, TypeOperators, UndecidableInstances #-}
module Semantic.IO
( readFile
, readFilePair
@ -8,22 +8,34 @@ module Semantic.IO
, readBlobsFromPaths
, readBlobsFromDir
, languageForFilePath
, NoLanguageForBlob(..)
, readBlobs
, readBlobPairs
, writeToOutput
, Files
, runFiles
, rethrowing
) where
import Prologue hiding (fail)
import qualified Control.Exception as Exc
import Control.Monad.Effect hiding (run)
import Control.Monad.Effect.Exception
import Control.Monad.Effect.Run
import Control.Monad.IO.Class
import Data.Aeson
import qualified Data.Blob as Blob
import Data.Language
import Data.Source
import Data.Bool
import qualified Data.ByteString as B
import qualified Data.ByteString.Lazy as BL
import Data.Language
import Data.Source
import Prelude hiding (readFile)
import Prologue hiding (MonadError (..), fail)
import System.Directory (doesDirectoryExist)
import System.Exit
import System.FilePath
import System.IO (Handle)
import System.FilePath.Glob
import System.Directory (doesDirectoryExist)
import System.IO (Handle)
import Text.Read
-- | Read a utf8-encoded file to a 'Blob'.
@ -44,7 +56,7 @@ readFilePair a b = do
_ -> fail "expected file pair with content on at least one side"
isDirectory :: MonadIO m => FilePath -> m Bool
isDirectory path = liftIO (doesDirectoryExist path) >>= pure
isDirectory path = liftIO (doesDirectoryExist path)
-- | Return a language based on a FilePath's extension, or Nothing if extension is not found or not supported.
languageForFilePath :: FilePath -> Maybe Language
@ -64,7 +76,7 @@ readBlobsFromHandle = fmap toBlobs . readFromHandle
where toBlobs BlobParse{..} = fmap toBlob blobs
readBlobsFromPaths :: MonadIO m => [(FilePath, Maybe Language)] -> m [Blob.Blob]
readBlobsFromPaths files = catMaybes <$> traverse (uncurry Semantic.IO.readFile) files
readBlobsFromPaths files = catMaybes <$> traverse (uncurry readFile) files
readBlobsFromDir :: MonadIO m => FilePath -> m [Blob.Blob]
readBlobsFromDir path = do
@ -111,3 +123,61 @@ instance FromJSON BlobPair where
(Just b, Nothing) -> pure $ Join (This b)
(Nothing, Just a) -> pure $ Join (That a)
_ -> fail "Expected object with 'before' and/or 'after' keys only"
-- | An exception indicating that weve tried to diff or parse a blob of unknown language.
newtype NoLanguageForBlob = NoLanguageForBlob FilePath
deriving (Eq, Exception, Ord, Show, Typeable)
-- | A task which reads a list of 'Blob's from a 'Handle' or a list of 'FilePath's optionally paired with 'Language's.
readBlobs :: Member Files effs => Either Handle [(FilePath, Maybe Language)] -> Eff effs [Blob.Blob]
readBlobs = send . ReadBlobs
-- | A task which reads a list of pairs of 'Blob's from a 'Handle' or a list of pairs of 'FilePath's optionally paired with 'Language's.
readBlobPairs :: Member Files effs => Either Handle [Both (FilePath, Maybe Language)] -> Eff effs [Blob.BlobPair]
readBlobPairs = send . ReadBlobPairs
-- | A task which writes a 'B.ByteString' to a 'Handle' or a 'FilePath'.
writeToOutput :: Member Files effs => Either Handle FilePath -> B.ByteString -> Eff effs ()
writeToOutput path = send . WriteToOutput path
-- | An effect to read/write 'Blob.Blob's from 'Handle's or 'FilePath's.
data Files out where
ReadBlobs :: Either Handle [(FilePath, Maybe Language)] -> Files [Blob.Blob]
ReadBlobPairs :: Either Handle [Both (FilePath, Maybe Language)] -> Files [Blob.BlobPair]
WriteToOutput :: Either Handle FilePath -> B.ByteString -> Files ()
-- | Run a 'Files' effect in 'IO'.
runFiles :: Members '[Exc SomeException, IO] effs => Eff (Files ': effs) a -> Eff effs a
runFiles = interpret $ \ files -> case files of
ReadBlobs (Left handle) -> rethrowing (readBlobsFromHandle handle)
ReadBlobs (Right paths@[(path, Nothing)]) -> rethrowing (isDirectory path >>= bool (readBlobsFromPaths paths) (readBlobsFromDir path))
ReadBlobs (Right paths) -> rethrowing (readBlobsFromPaths paths)
ReadBlobPairs source -> rethrowing (either readBlobPairsFromHandle (traverse (runBothWith readFilePair)) source)
WriteToOutput destination contents -> liftIO (either B.hPutStr B.writeFile destination contents)
instance (Members '[Exc SomeException, IO] effects, Run effects result rest) => Run (Files ': effects) result rest where
run = run . runFiles
-- | Catch exceptions in 'IO' actions embedded in 'Eff', handling them with the passed function.
--
-- Note that while the type allows 'IO' to occur anywhere within the effect list, it must actually occur at the end to be able to run the computation.
catchException :: ( Exc.Exception e
, Member IO r
)
=> Eff r a
-> (e -> Eff r a)
-> Eff r a
catchException m handler = interpose pure (\ m yield -> send (Exc.try m) >>= either handler yield) m
-- | Lift an 'IO' action into 'Eff', catching and rethrowing any exceptions it throws into an 'Exc' effect.
rethrowing :: ( Member (Exc SomeException) r
, Member IO r
)
=> IO a
-> Eff r a
rethrowing m = catchException (liftIO m) (throwError . toException @SomeException)

View File

@ -1,11 +1,12 @@
module Semantic.Log where
import Prologue
import Control.Monad.IO.Class
import Data.Error (withSGRCode)
import Data.List (intersperse)
import qualified Data.Time.Clock.POSIX as Time (getCurrentTime)
import qualified Data.Time.Format as Time
import qualified Data.Time.LocalTime as LocalTime
import Prologue
import Semantic.Queue
import System.Console.ANSI
import System.IO
@ -27,14 +28,14 @@ data Level
-- | Queue a message to be logged.
queueLogMessage :: AsyncQueue Message Options -> Level -> String -> [(String, String)] -> IO ()
queueLogMessage :: MonadIO io => AsyncQueue Message Options -> Level -> String -> [(String, String)] -> io ()
queueLogMessage q@AsyncQueue{..} level message pairs
| Just logLevel <- optionsLevel asyncQueueExtra, level <= logLevel = Time.getCurrentTime >>= LocalTime.utcToLocalZonedTime >>= queue q . Message level message pairs
| Just logLevel <- optionsLevel asyncQueueExtra, level <= logLevel = liftIO Time.getCurrentTime >>= liftIO . LocalTime.utcToLocalZonedTime >>= liftIO . queue q . Message level message pairs
| otherwise = pure ()
-- | Log a message to stderr.
logMessage :: Options -> Message -> IO ()
logMessage options@Options{..} = hPutStr stderr . optionsFormatter options
logMessage :: MonadIO io => Options -> Message -> io ()
logMessage options@Options{..} = liftIO . hPutStr stderr . optionsFormatter options
-- | Format log messaging using "logfmt".
--
@ -102,8 +103,8 @@ defaultOptions = Options
, optionsProcessID = 0
}
configureOptionsForHandle :: Handle -> Options -> IO Options
configureOptionsForHandle handle options = do
configureOptionsForHandle :: MonadIO io => Handle -> Options -> io Options
configureOptionsForHandle handle options = liftIO $ do
pid <- getProcessID
isTerminal <- hIsTerminalDevice handle
pure $ options

36
src/Semantic/Parse.hs Normal file
View File

@ -0,0 +1,36 @@
{-# LANGUAGE GADTs #-}
module Semantic.Parse where
import Analysis.ConstructorName (ConstructorName, constructorLabel)
import Analysis.IdentifierName (IdentifierName, identifierLabel)
import Analysis.Declaration (HasDeclaration, declarationAlgebra)
import Analysis.PackageDef (HasPackageDef, packageDefAlgebra)
import Data.Blob
import Data.JSON.Fields
import Data.Output
import Data.Record
import Parsing.Parser
import Prologue hiding (MonadError(..))
import Rendering.Renderer
import Semantic.IO (NoLanguageForBlob(..))
import Semantic.Task
parseBlobs :: (Members '[Distribute WrappedTask, Task, Exc SomeException] effs, Output output) => TermRenderer output -> [Blob] -> Eff effs ByteString
parseBlobs renderer blobs = toOutput' <$> distributeFoldMap (WrapTask . parseBlob renderer) blobs
where toOutput' = case renderer of
JSONTermRenderer -> toOutput . renderJSONTerms
SymbolsTermRenderer _ -> toOutput . renderSymbolTerms
_ -> toOutput
-- | A task to parse a 'Blob' and render the resulting 'Term'.
parseBlob :: Members '[Task, Exc SomeException] effs => TermRenderer output -> Blob -> Eff effs output
parseBlob renderer blob@Blob{..}
| Just (SomeParser parser) <- someParser (Proxy :: Proxy '[ConstructorName, HasPackageDef, HasDeclaration, IdentifierName, Foldable, Functor, ToJSONFields1]) <$> blobLanguage
= parse parser blob >>= case renderer of
JSONTermRenderer -> decorate constructorLabel >=> decorate identifierLabel >=> render (renderJSONTerm blob)
SExpressionTermRenderer -> decorate constructorLabel . (Nil <$) >=> render renderSExpressionTerm
TagsTermRenderer -> decorate (declarationAlgebra blob) >=> render (renderToTags blob)
ImportsTermRenderer -> decorate (declarationAlgebra blob) >=> decorate (packageDefAlgebra blob) >=> render (renderToImports blob)
SymbolsTermRenderer fields -> decorate (declarationAlgebra blob) >=> render (renderToSymbols fields blob)
DOTTermRenderer -> render (renderDOTTerm blob)
| otherwise = throwError (SomeException (NoLanguageForBlob blobPath))

View File

@ -22,18 +22,20 @@ module Semantic.Stat
) where
import Prologue
import Control.Monad.IO.Class
import qualified Data.ByteString.Char8 as B
import Data.List (intercalate)
import Data.List.Split (splitOneOf)
import Network.Socket (Socket(..), SocketType(..), socket, connect, close, getAddrInfo, addrFamily, addrAddress, defaultProtocol)
import qualified Data.Time.Clock as Time
import qualified Data.Time.Clock.POSIX as Time (getCurrentTime)
import Network.Socket
(Socket (..), SocketType (..), addrAddress, addrFamily, close, connect, defaultProtocol, getAddrInfo, socket)
import Network.Socket.ByteString
import Network.URI
import Numeric
import qualified Data.ByteString.Char8 as B
import Prologue
import System.Environment
import System.IO.Error
import qualified Data.Time.Clock as Time
import qualified Data.Time.Clock.POSIX as Time (getCurrentTime)
-- | A named piece of data you wish to record a specific 'Metric' for.
-- See https://docs.datadoghq.com/guides/dogstatsd/ for more details.
@ -77,14 +79,13 @@ timing :: String -> Double -> Tags -> Stat
timing n v = Stat n (Timer v)
-- | Run an IO Action and record timing
withTiming :: (Stat -> IO ()) -> String -> Tags -> IO a -> IO a
withTiming statter name tags f = do
start <- Time.getCurrentTime
result <- f
end <- Time.getCurrentTime
withTiming :: MonadIO io => String -> Tags -> io a -> io (a, Stat)
withTiming name tags action = do
start <- liftIO Time.getCurrentTime
result <- action
end <- liftIO Time.getCurrentTime
let duration = realToFrac (Time.diffUTCTime end start * 1000)
statter (timing name duration tags)
pure result
pure (result, timing name duration tags)
-- | Histogram measurement.
histogram :: String -> Double -> Tags -> Stat
@ -108,8 +109,8 @@ data StatsClient
-- * STATS_ADDR - String URI to send stats to in the form of `host:port`.
-- * DOGSTATSD_HOST - String hostname which will override the above host.
-- Generally used on kubes pods.
defaultStatsClient :: IO StatsClient
defaultStatsClient = do
defaultStatsClient :: MonadIO io => io StatsClient
defaultStatsClient = liftIO $ do
addr <- lookupEnv "STATS_ADDR"
let (host', port) = parseAddr (fmap ("statsd://" <>) addr)
@ -130,20 +131,20 @@ defaultStatsClient = do
-- | Create a StatsClient at the specified host and port with a namespace prefix.
statsClient :: String -> String -> String -> IO StatsClient
statsClient host port statsClientNamespace = do
statsClient :: MonadIO io => String -> String -> String -> io StatsClient
statsClient host port statsClientNamespace = liftIO $ do
(addr:_) <- getAddrInfo Nothing (Just host) (Just port)
sock <- socket (addrFamily addr) Datagram defaultProtocol
connect sock (addrAddress addr)
pure (StatsClient sock statsClientNamespace host port)
-- | Close the client's underlying socket.
closeStatClient :: StatsClient -> IO ()
closeStatClient StatsClient{..} = close statsClientUDPSocket
closeStatClient :: MonadIO io => StatsClient -> io ()
closeStatClient StatsClient{..} = liftIO (close statsClientUDPSocket)
-- | Send a stat over the StatsClient's socket.
sendStat :: StatsClient -> Stat -> IO ()
sendStat StatsClient{..} = void . tryIOError . sendAll statsClientUDPSocket . B.pack . renderDatagram statsClientNamespace
sendStat :: MonadIO io => StatsClient -> Stat -> io ()
sendStat StatsClient{..} = liftIO . void . tryIOError . sendAll statsClientUDPSocket . B.pack . renderDatagram statsClientNamespace
-- Datagram Rendering

View File

@ -1,82 +1,93 @@
{-# LANGUAGE DataKinds, GADTs, MultiParamTypeClasses, TypeOperators #-}
{-# LANGUAGE GADTs, GeneralizedNewtypeDeriving, TypeOperators, UndecidableInstances #-}
module Semantic.Task
( Task
, TaskEff
, WrappedTask(..)
, Level(..)
, RAlgebra
, Differ
, readBlobs
, readBlobPairs
, writeToOutput
-- * I/O
, IO.readBlobs
, IO.readBlobPairs
, IO.writeToOutput
-- * Telemetry
, writeLog
, writeStat
, time
-- * High-level flow
, parse
, parseModule
, parseModules
, parsePackage
, analyze
, decorate
, diff
, render
, graphImports
-- * Concurrency
, distribute
, distributeFor
, distributeFoldMap
, bidistribute
, bidistributeFor
-- * Configuration
, defaultOptions
, configureOptionsForHandle
, terminalFormatter
, logfmtFormatter
-- * Interpreting
, runTask
, runTaskWithOptions
-- * Re-exports
, Distribute
, Eff
, Exc
, throwError
, SomeException
, Telemetry
) where
import Prologue
import qualified Analysis.Abstract.ImportGraph as Abstract
import Analysis.Abstract.Evaluating
import Analysis.Decorator (decoratorWithAlgebra)
import qualified Assigning.Assignment as Assignment
import Control.Monad.IO.Class
import Control.Parallel.Strategies
import qualified Control.Concurrent.Async as Async
import Control.Monad.Free.Freer
import qualified Control.Abstract.Analysis as Analysis
import qualified Control.Exception as Exc
import Control.Monad.Effect.Exception
import Control.Monad.Effect.Internal as Eff hiding (run)
import Control.Monad.Effect.Reader
import Control.Monad.Effect.Run as Run
import Data.Abstract.Address
import qualified Data.Abstract.Evaluatable as Analysis
import Data.Abstract.FreeVariables
import Data.Abstract.Located
import Data.Abstract.Module
import Data.Abstract.Package as Package
import Data.Abstract.Value (Value)
import Data.Blob
import Data.Bool
import qualified Data.ByteString as B
import Data.Diff
import qualified Data.Error as Error
import Data.Language
import Data.Record
import qualified Data.Syntax as Syntax
import Data.Term
import Parsing.Parser
import Parsing.CMark
import Parsing.Parser
import Parsing.TreeSitter
import System.Exit (die)
import System.IO (Handle, stderr)
import Prologue hiding (MonadError(..))
import Semantic.Distribute
import qualified Semantic.IO as IO
import Semantic.Log
import Semantic.Stat as Stat
import Semantic.Queue
data TaskF output where
ReadBlobs :: Either Handle [(FilePath, Maybe Language)] -> TaskF [Blob]
ReadBlobPairs :: Either Handle [Both (FilePath, Maybe Language)] -> TaskF [BlobPair]
WriteToOutput :: Either Handle FilePath -> B.ByteString -> TaskF ()
WriteLog :: Level -> String -> [(String, String)] -> TaskF ()
WriteStat :: Stat -> TaskF ()
Time :: String -> [(String, String)] -> Task output -> TaskF output
Parse :: Parser term -> Blob -> TaskF term
Decorate :: Functor f => RAlgebra (TermF f (Record fields)) (Term f (Record fields)) field -> Term f (Record fields) -> TaskF (Term f (Record (field ': fields)))
Diff :: Differ syntax ann1 ann2 -> Term syntax ann1 -> Term syntax ann2 -> TaskF (Diff syntax ann1 ann2)
Render :: Renderer input output -> input -> TaskF output
Distribute :: Traversable t => t (Task output) -> TaskF (t output)
Bidistribute :: Bitraversable t => t (Task output1) (Task output2) -> TaskF (t output1 output2)
-- | For MonadIO.
LiftIO :: IO a -> TaskF a
-- | For MonadError.
Throw :: SomeException -> TaskF a
Catch :: Task a -> (SomeException -> Task a) -> TaskF a
import Semantic.Stat as Stat
import Semantic.Telemetry
import System.Exit (die)
import System.IO (stderr)
-- | A high-level task producing some result, e.g. parsing, diffing, rendering. 'Task's can also specify explicit concurrency via 'distribute', 'distributeFor', and 'distributeFoldMap'
type Task = Freer TaskF
type TaskEff = Eff '[Distribute WrappedTask, Task, IO.Files, Reader Options, Telemetry, Exc SomeException, IO]
-- | A wrapper for a 'Task', to embed in other effects.
newtype WrappedTask a = WrapTask { unwrapTask :: TaskEff a }
deriving (Applicative, Functor, Monad)
-- | A function to compute the 'Diff' for a pair of 'Term's with arbitrary syntax functor & annotation types.
type Differ syntax ann1 ann2 = Term syntax ann1 -> Term syntax ann2 -> Diff syntax ann1 ann2
@ -84,177 +95,139 @@ type Differ syntax ann1 ann2 = Term syntax ann1 -> Term syntax ann2 -> Diff synt
-- | A function to render terms or diffs.
type Renderer i o = i -> o
-- | A 'Task' which reads a list of 'Blob's from a 'Handle' or a list of 'FilePath's optionally paired with 'Language's.
readBlobs :: Either Handle [(FilePath, Maybe Language)] -> Task [Blob]
readBlobs from = ReadBlobs from `Then` return
-- | A task which parses a 'Blob' with the given 'Parser'.
parse :: Member Task effs => Parser term -> Blob -> Eff effs term
parse parser = send . Parse parser
-- | A 'Task' which reads a list of pairs of 'Blob's from a 'Handle' or a list of pairs of 'FilePath's optionally paired with 'Language's.
readBlobPairs :: Either Handle [Both (FilePath, Maybe Language)] -> Task [BlobPair]
readBlobPairs from = ReadBlobPairs from `Then` return
-- | Parse a file into a 'Module'.
parseModule :: Members '[IO.Files, Task] effs => Parser term -> Maybe FilePath -> FilePath -> Eff effs (Module term)
parseModule parser rootDir path = do
blob <- head <$> IO.readBlobs (Right [(path, IO.languageForFilePath path)])
moduleForBlob rootDir blob <$> parse parser blob
-- | A 'Task' which writes a 'B.ByteString' to a 'Handle' or a 'FilePath'.
writeToOutput :: Either Handle FilePath -> B.ByteString -> Task ()
writeToOutput path contents = WriteToOutput path contents `Then` return
-- | Parse a list of files into 'Module's.
parseModules :: Members '[IO.Files, Task] effs => Parser term -> FilePath -> [FilePath] -> Eff effs [Module term]
parseModules parser rootDir = traverse (parseModule parser (Just rootDir))
-- | A 'Task' which logs a message at a specific log level to stderr.
writeLog :: Level -> String -> [(String, String)] -> Task ()
writeLog level message pairs = WriteLog level message pairs `Then` return
-- | Parse a list of files into a 'Package'.
parsePackage :: Members '[IO.Files, Task] effs => PackageName -> Parser term -> FilePath -> [FilePath] -> Eff effs (Package term)
parsePackage name parser rootDir paths = Package (PackageInfo name Nothing) . Package.fromModules <$> parseModules parser rootDir paths
-- | A 'Task' which writes a stat.
writeStat :: Stat -> Task ()
writeStat stat = WriteStat stat `Then` return
-- | A 'Task' which measures and stats the timing of another 'Task'.
time :: String -> [(String, String)] -> Task output -> Task output
time statName tags task = Time statName tags task `Then` return
-- | A task running some 'Analysis.MonadAnalysis' to completion.
analyze :: Member Task effs => Analysis.SomeAnalysis m result -> Eff effs result
analyze = send . Analyze
-- | A 'Task' which parses a 'Blob' with the given 'Parser'.
parse :: Parser term -> Blob -> Task term
parse parser blob = Parse parser blob `Then` return
-- | A task which decorates a 'Term' with values computed using the supplied 'RAlgebra' function.
decorate :: (Functor f, Member Task effs) => RAlgebra (TermF f (Record fields)) (Term f (Record fields)) field -> Term f (Record fields) -> Eff effs (Term f (Record (field ': fields)))
decorate algebra = send . Decorate algebra
-- | A 'Task' which decorates a 'Term' with values computed using the supplied 'RAlgebra' function.
decorate :: Functor f => RAlgebra (TermF f (Record fields)) (Term f (Record fields)) field -> Term f (Record fields) -> Task (Term f (Record (field ': fields)))
decorate algebra term = Decorate algebra term `Then` return
-- | A task which diffs a pair of terms using the supplied 'Differ' function.
diff :: Member Task effs => Differ syntax ann1 ann2 -> Term syntax ann1 -> Term syntax ann2 -> Eff effs (Diff syntax ann1 ann2)
diff differ term1 term2 = send (Semantic.Task.Diff differ term1 term2)
-- | A 'Task' which diffs a pair of terms using the supplied 'Differ' function.
diff :: Differ syntax ann1 ann2 -> Term syntax ann1 -> Term syntax ann2 -> Task (Diff syntax ann1 ann2)
diff differ term1 term2 = Semantic.Task.Diff differ term1 term2 `Then` return
-- | A task which renders some input using the supplied 'Renderer' function.
render :: Member Task effs => Renderer input output -> input -> Eff effs output
render renderer = send . Render renderer
-- | A 'Task' which renders some input using the supplied 'Renderer' function.
render :: Renderer input output -> input -> Task output
render renderer input = Render renderer input `Then` return
-- | Distribute a 'Traversable' container of 'Task's over the available cores (i.e. execute them concurrently), collecting their results.
--
-- This is a concurrent analogue of 'sequenceA'.
distribute :: Traversable t => t (Task output) -> Task (t output)
distribute tasks = Distribute tasks `Then` return
-- | Render and serialize the import graph for a given 'Package'.
graphImports :: (Apply Eq1 syntax, Apply Analysis.Evaluatable syntax, Apply FreeVariables1 syntax, Apply Functor syntax, Apply Ord1 syntax, Apply Show1 syntax, Member Syntax.Identifier syntax, Members '[Exc SomeException, Task] effs, Ord ann, Show ann) => Package (Term (Union syntax) ann) -> Eff effs B.ByteString
graphImports package = analyze (Analysis.SomeAnalysis (Analysis.evaluatePackage package `asAnalysisForTypeOfPackage` package)) >>= renderGraph
where asAnalysisForTypeOfPackage :: Abstract.ImportGraphing (Evaluating (Located Precise term) term (Value (Located Precise term))) effects value -> Package term -> Abstract.ImportGraphing (Evaluating (Located Precise term) term (Value (Located Precise term))) effects value
asAnalysisForTypeOfPackage = const
-- | Distribute a 'Bitraversable' container of 'Task's over the available cores (i.e. execute them concurrently), collecting their results.
--
-- This is a concurrent analogue of 'bisequenceA'.
bidistribute :: Bitraversable t => t (Task output1) (Task output2) -> Task (t output1 output2)
bidistribute tasks = Bidistribute tasks `Then` return
renderGraph result = case result of
(Right (Right (Right (Right (Right (_, graph))))), _) -> pure $! Abstract.renderImportGraph graph
_ -> throwError (toException (Exc.ErrorCall "graphImports: import graph rendering failed"))
-- | Distribute the application of a function to each element of a 'Traversable' container of inputs over the available cores (i.e. perform the function concurrently for each element), collecting the results.
--
-- This is a concurrent analogue of 'for' or 'traverse' (with the arguments flipped).
distributeFor :: Traversable t => t a -> (a -> Task output) -> Task (t output)
distributeFor inputs toTask = distribute (fmap toTask inputs)
-- | Distribute the application of a function to each element of a 'Bitraversable' container of inputs over the available cores (i.e. perform the functions concurrently for each element), collecting the results.
--
-- This is a concurrent analogue of 'bifor' or 'bitraverse' (with the arguments flipped).
bidistributeFor :: Bitraversable t => t a b -> (a -> Task output1) -> (b -> Task output2) -> Task (t output1 output2)
bidistributeFor inputs toTask1 toTask2 = bidistribute (bimap toTask1 toTask2 inputs)
-- | Distribute the application of a function to each element of a 'Traversable' container of inputs over the available cores (i.e. perform the function concurrently for each element), combining the results 'Monoid'ally into a final value.
--
-- This is a concurrent analogue of 'foldMap'.
distributeFoldMap :: (Traversable t, Monoid output) => (a -> Task output) -> t a -> Task output
distributeFoldMap toTask inputs = fmap fold (distribute (fmap toTask inputs))
-- | Execute a 'Task' with the 'defaultOptions', yielding its result value in 'IO'.
--
-- > runTask = runTaskWithOptions defaultOptions
runTask :: Task a -> IO a
runTask :: TaskEff a -> IO a
runTask = runTaskWithOptions defaultOptions
-- | Execute a 'Task' with the passed 'Options', yielding its result value in 'IO'.
runTaskWithOptions :: Options -> Task a -> IO a
-- | Execute a 'TaskEff' with the passed 'Options', yielding its result value in 'IO'.
runTaskWithOptions :: Options -> TaskEff a -> IO a
runTaskWithOptions options task = do
options <- configureOptionsForHandle stderr options
statter <- defaultStatsClient >>= newQueue sendStat
logger <- newQueue logMessage options
result <- withTiming (queue statter) "run" [] $
run options logger statter task
(result, stat) <- withTiming "run" [] $ do
let run :: TaskEff a -> IO (Either SomeException a)
run task = Run.run task (Action (run . unwrapTask)) options (Queues logger statter)
run task
queue statter stat
closeQueue statter
closeStatClient (asyncQueueExtra statter)
closeQueue logger
either (die . displayException) pure result
where
run :: Options
-> AsyncQueue Message Options
-> AsyncQueue Stat StatsClient
-> Task a
-> IO (Either SomeException a)
run options logger statter = go
where
go :: Task a -> IO (Either SomeException a)
go = iterFreerA (\ yield task -> case task of
ReadBlobs (Left handle) -> (IO.readBlobsFromHandle handle >>= yield) `catchError` (pure . Left . toException)
ReadBlobs (Right paths@[(path, Nothing)]) -> (IO.isDirectory path >>= bool (IO.readBlobsFromPaths paths) (IO.readBlobsFromDir path) >>= yield) `catchError` (pure . Left . toException)
ReadBlobs (Right paths) -> (IO.readBlobsFromPaths paths >>= yield) `catchError` (pure . Left . toException)
ReadBlobPairs source -> (either IO.readBlobPairsFromHandle (traverse (runBothWith IO.readFilePair)) source >>= yield) `catchError` (pure . Left . toException)
WriteToOutput destination contents -> either B.hPutStr B.writeFile destination contents >>= yield
WriteLog level message pairs -> queueLogMessage logger level message pairs >>= yield
WriteStat stat -> queue statter stat >>= yield
Time statName tags task -> withTiming (queue statter) statName tags (go task) >>= either (pure . Left) yield
Parse parser blob -> go (runParser options blob parser) >>= either (pure . Left) yield
Decorate algebra term -> pure (decoratorWithAlgebra algebra term) >>= yield
Semantic.Task.Diff differ term1 term2 -> pure (differ term1 term2) >>= yield
Render renderer input -> pure (renderer input) >>= yield
Distribute tasks -> Async.mapConcurrently go tasks >>= either (pure . Left) yield . sequenceA . withStrategy (parTraversable (parTraversable rseq))
Bidistribute tasks -> Async.runConcurrently (bitraverse (Async.Concurrently . go) (Async.Concurrently . go) tasks) >>= either (pure . Left) yield . bisequenceA . withStrategy (parBitraversable (parTraversable rseq) (parTraversable rseq))
LiftIO action -> action >>= yield
Throw err -> pure (Left err)
Catch during handler -> do
result <- go during
case result of
Left err -> go (handler err) >>= either (pure . Left) yield
Right a -> yield a) . fmap Right
parBitraversable :: Bitraversable t => Strategy a -> Strategy b -> Strategy (t a b)
parBitraversable strat1 strat2 = bitraverse (rparWith strat1) (rparWith strat2)
runParser :: Options -> Blob -> Parser term -> Task term
runParser Options{..} blob@Blob{..} = go
where
go :: Parser term -> Task term
go parser = case parser of
-- | An effect describing high-level tasks to be performed.
data Task output where
Parse :: Parser term -> Blob -> Task term
Analyze :: Analysis.SomeAnalysis m result -> Task result
Decorate :: Functor f => RAlgebra (TermF f (Record fields)) (Term f (Record fields)) field -> Term f (Record fields) -> Task (Term f (Record (field ': fields)))
Diff :: Differ syntax ann1 ann2 -> Term syntax ann1 -> Term syntax ann2 -> Task (Diff syntax ann1 ann2)
Render :: Renderer input output -> input -> Task output
-- | Run a 'Task' effect by performing the actions in 'IO'.
runTaskF :: Members '[Reader Options, Telemetry, Exc SomeException, IO] effs => Eff (Task ': effs) a -> Eff effs a
runTaskF = interpret $ \ task -> case task of
Parse parser blob -> runParser blob parser
Analyze analysis -> pure (Analysis.runSomeAnalysis analysis)
Decorate algebra term -> pure (decoratorWithAlgebra algebra term)
Semantic.Task.Diff differ term1 term2 -> pure (differ term1 term2)
Render renderer input -> pure (renderer input)
-- | Log an 'Error.Error' at the specified 'Level'.
logError :: Member Telemetry effs => Options -> Level -> Blob -> Error.Error String -> [(String, String)] -> Eff effs ()
logError Options{..} level blob err = writeLog level (Error.formatError optionsPrintSource (optionsIsTerminal && optionsEnableColour) blob err)
-- | Parse a 'Blob' in 'IO'.
runParser :: Members '[Reader Options, Telemetry, Exc SomeException, IO] effs => Blob -> Parser term -> Eff effs term
runParser blob@Blob{..} parser = case parser of
ASTParser language ->
time "parse.tree_sitter_ast_parse" languageTag $
liftIO ((Right <$> parseToAST language blob) `catchError` (pure . Left . toException)) >>= either throwError pure
IO.rethrowing (parseToAST language blob)
AssignmentParser parser assignment -> do
ast <- go parser `catchError` \ err -> do
ast <- runParser blob parser `catchError` \ (SomeException err) -> do
writeStat (Stat.increment "parse.parse_failures" languageTag)
writeLog Error "failed parsing" (("task", "parse") : blobFields)
throwError err
throwError (toException err)
options <- ask
time "parse.assign" languageTag $
case Assignment.assign blobSource assignment ast of
Left err -> do
writeStat (Stat.increment "parse.assign_errors" languageTag)
writeLog Error (Error.formatError optionsPrintSource (optionsIsTerminal && optionsEnableColour) blob err) (("task", "assign") : blobFields)
logError options Error blob err (("task", "assign") : blobFields)
throwError (toException err)
Right term -> do
for_ (errors term) $ \ err -> case Error.errorActual err of
(Just "ParseError") -> do
Just "ParseError" -> do
writeStat (Stat.increment "parse.parse_errors" languageTag)
writeLog Warning (Error.formatError optionsPrintSource (optionsIsTerminal && optionsEnableColour) blob err) (("task", "parse") : blobFields)
logError options Warning blob err (("task", "parse") : blobFields)
_ -> do
writeStat (Stat.increment "parse.assign_warnings" languageTag)
writeLog Warning (Error.formatError optionsPrintSource (optionsIsTerminal && optionsEnableColour) blob err) (("task", "assign") : blobFields)
logError options Warning blob err (("task", "assign") : blobFields)
writeStat (Stat.count "parse.nodes" (length term) languageTag)
pure term
MarkdownParser ->
time "parse.cmark_parse" languageTag $
let term = cmarkParser blobSource
in length term `seq` pure term
blobFields = ("path", blobPath) : languageTag
where blobFields = ("path", blobPath) : languageTag
languageTag = maybe [] (pure . (,) ("language" :: String) . show) blobLanguage
errors :: (Syntax.Error :< fs, Apply Foldable fs, Apply Functor fs) => Term (Union fs) (Record Assignment.Location) -> [Error.Error String]
errors = cata $ \ (In a syntax) -> case syntax of
_ | Just err@Syntax.Error{} <- prj syntax -> [Syntax.unError (getField a) err]
_ -> fold syntax
instance MonadIO Task where
liftIO action = LiftIO action `Then` return
instance MonadError SomeException Task where
throwError error = Throw error `Then` return
catchError during handler = Catch during handler `Then` return
{-# ANN module ("HLint: ignore Avoid return" :: String) #-}
instance (Members '[Reader Options, Telemetry, Exc SomeException, IO] effects, Run effects result rest) => Run (Task ': effects) result rest where
run = run . runTaskF

69
src/Semantic/Telemetry.hs Normal file
View File

@ -0,0 +1,69 @@
{-# LANGUAGE GADTs, RankNTypes, TypeOperators, UndecidableInstances #-}
module Semantic.Telemetry
( writeLog
, writeStat
, time
, Telemetry
, Queues(..)
, runTelemetry
, ignoreTelemetry
) where
import Control.Monad.Effect.Internal hiding (run)
import Control.Monad.Effect.Reader
import Control.Monad.Effect.Run
import Control.Monad.IO.Class
import Prologue
import Semantic.Log
import Semantic.Queue
import Semantic.Stat
-- | A task which logs a message at a specific log level to stderr.
writeLog :: Member Telemetry effs => Level -> String -> [(String, String)] -> Eff effs ()
writeLog level message pairs = send (WriteLog level message pairs)
-- | A task which writes a stat.
writeStat :: Member Telemetry effs => Stat -> Eff effs ()
writeStat stat = send (WriteStat stat)
-- | A task which measures and stats the timing of another task.
time :: Members '[Telemetry, IO] effs => String -> [(String, String)] -> Eff effs output -> Eff effs output
time statName tags task = do
(a, stat) <- withTiming statName tags task
a <$ writeStat stat
-- | Statting and logging effects.
data Telemetry output where
WriteStat :: Stat -> Telemetry ()
WriteLog :: Level -> String -> [(String, String)] -> Telemetry ()
-- | Queues for logging and statting.
data Queues = Queues { logger :: AsyncQueue Message Options, statter :: AsyncQueue Stat StatsClient }
-- | Run a 'Telemetry' effect by expecting a 'Reader' of 'Queue's to write stats and logs to.
runTelemetry :: Member IO (Reader Queues ': effs) => Eff (Telemetry ': effs) a -> Eff (Reader Queues ': effs) a
runTelemetry = reinterpret (\ t -> case t of
WriteStat stat -> asks statter >>= \ statter -> liftIO (queue statter stat)
WriteLog level message pairs -> asks logger >>= \ logger -> queueLogMessage logger level message pairs)
-- | Run a 'Telemetry' effect by ignoring statting/logging.
ignoreTelemetry :: Eff (Telemetry ': effs) a -> Eff effs a
ignoreTelemetry = interpret (\ t -> case t of
WriteStat{} -> pure ()
WriteLog{} -> pure ())
-- | Interpret an effect by replacing it with another effect.
reinterpret :: (forall x. effect x -> Eff (newEffect ': effs) x)
-> Eff (effect ': effs) a
-> Eff (newEffect ': effs) a
reinterpret handle = loop
where loop (Val x) = pure x
loop (E u' q) = case decompose u' of
Right eff -> handle eff >>= q >>> loop
Left u -> E (weaken u) (tsingleton (q >>> loop))
instance (Member IO (Reader Queues ': effects), Run (Reader Queues ': effects) result rest) => Run (Telemetry ': effects) result rest where
run = run . runTelemetry

View File

@ -33,9 +33,10 @@ import qualified GHC.TypeLits as TypeLevel
import Language.Preluded
import Parsing.Parser
import Prologue
import Semantic
import Semantic.Diff (diffTermPair)
import Semantic.IO as IO
import Semantic.Task
import Semantic.Task hiding (parsePackage)
import qualified Semantic.Task as Task
import System.FilePath.Posix
import qualified Language.Go.Assignment as Go
@ -227,7 +228,7 @@ parseFiles :: Parser term -> FilePath -> [FilePath] -> IO [Module term]
parseFiles parser rootDir = traverse (parseFile parser (Just rootDir))
parsePackage :: PackageName -> Parser term -> FilePath -> [FilePath] -> IO (Package term)
parsePackage name parser rootDir files = Package (PackageInfo name Nothing) . Package.fromModules <$> parseFiles parser rootDir files
parsePackage name parser rootDir = runTask . Task.parsePackage name parser rootDir
-- Read a file from the filesystem into a Blob.
@ -235,8 +236,7 @@ file :: MonadIO m => FilePath -> m Blob
file path = fromJust <$> IO.readFile path (languageForFilePath path)
-- Diff helpers
diffWithParser ::
( HasField fields Data.Span.Span
diffWithParser :: ( HasField fields Data.Span.Span
, HasField fields Range
, Eq1 syntax
, Show1 syntax
@ -244,23 +244,9 @@ diffWithParser ::
, Diffable syntax
, GAlign syntax
, HasDeclaration syntax
, Members '[Distribute WrappedTask, Task] effs
)
=> Parser (Term syntax (Record fields))
-> BlobPair
-> Task (Diff syntax (Record (Maybe Declaration ': fields)) (Record (Maybe Declaration ': fields)))
diffWithParser parser = run (\ blob -> parse parser blob >>= decorate (declarationAlgebra blob))
where
run parse blobs = bidistributeFor (runJoin blobs) parse parse >>= diffTermPair diffTerms
diffBlobWithParser ::
( HasField fields Data.Span.Span
, HasField fields Range
, Traversable syntax
, HasDeclaration syntax
)
=> Parser (Term syntax (Record fields))
-> Blob
-> Task (Term syntax (Record (Maybe Declaration : fields)))
diffBlobWithParser parser = run (\ blob -> parse parser blob >>= decorate (declarationAlgebra blob))
where
run parse = parse
-> Eff effs (Diff syntax (Record (Maybe Declaration ': fields)) (Record (Maybe Declaration ': fields)))
diffWithParser parser blobs = distributeFor blobs (\ blob -> WrapTask $ parse parser blob >>= decorate (declarationAlgebra blob)) >>= diffTermPair diffTerms . runJoin

View File

@ -27,8 +27,9 @@ import Data.Span as X
import Data.Term as X
import Parsing.Parser as X
import Rendering.Renderer as X
import Semantic as X
import Semantic.Task as X
import Semantic.Diff as X
import Semantic.Parse as X
import Semantic.Task as X hiding (parsePackage)
import Semantic.Util as X
import System.FilePath as X

2
vendor/effects vendored

@ -1 +1 @@
Subproject commit 215ac5be57258a786959dac391db6bef83a70f28
Subproject commit c316bd2d25fc562bbd49baf844d6587c497ede19