diff --git a/glean/client/hs/Glean.hs b/glean/client/hs/Glean.hs index bd8b7b7b2..fb570a0f8 100644 --- a/glean/client/hs/Glean.hs +++ b/glean/client/hs/Glean.hs @@ -149,6 +149,7 @@ module Glean , fillDatabase , finalize , completePredicates + , CompletePredicates(..) , basicWriter , FactBuilder , makeFact diff --git a/glean/client/hs/Glean/Remote.hs b/glean/client/hs/Glean/Remote.hs index aaf761a98..19d8c2544 100644 --- a/glean/client/hs/Glean/Remote.hs +++ b/glean/client/hs/Glean/Remote.hs @@ -30,6 +30,12 @@ module Glean.Remote , dbShard , dbShardWord + -- * Retries + , RetryBackend(..) + , backendRetryWrites + , RetryPolicy(..) + , defaultRetryPolicy + -- * More operations , SchemaPredicates , loadPredicates @@ -60,6 +66,7 @@ import Util.Log import Util.STM import Glean.Backend.Types +import Glean.Backend.Retry import Glean.BuildInfo (buildRule) import Glean.ClientConfig.Types (UseShards(..), ClientConfig(..)) import Glean.DefaultConfigs @@ -69,6 +76,7 @@ import qualified Glean.Types as Thrift import Glean.Query.Thrift import Glean.Query.Thrift.Internal import Glean.Util.ConfigProvider +import Glean.Util.RetryChannelException import Glean.Util.Service import Glean.Username (getUsername) import Glean.Util.ThriftSource as ThriftSource @@ -236,8 +244,8 @@ instance Backend ThriftBackend where withShard t (Thrift.work_repo $ Thrift.workFinished_work rq) $ GleanService.workFinished rq - completePredicates_ t repo = withShard t repo $ - GleanService.completePredicates repo + completePredicates_ t repo preds = withShard t repo $ + GleanService.completePredicates repo preds restoreDatabase t loc = withoutShard t $ GleanService.restore loc diff --git a/glean/db/Glean/Backend/Logging.hs b/glean/db/Glean/Backend/Logging.hs index 624e418f2..462b5f45c 100644 --- a/glean/db/Glean/Backend/Logging.hs +++ b/glean/db/Glean/Backend/Logging.hs @@ -128,11 +128,11 @@ instance Backend LoggingBackend where (const mempty) $ workFinished env rq - completePredicates_ (LoggingBackend env) repo = + completePredicates_ (LoggingBackend env) repo preds = loggingAction (runLogRepo "completePredicates" env repo) (const mempty) $ - completePredicates_ env repo + completePredicates_ env repo preds restoreDatabase (LoggingBackend env) loc = loggingAction (runLogCmd "restoreDatabase" env) (const mempty) $ diff --git a/glean/db/Glean/Database/CompletePredicates.hs b/glean/db/Glean/Database/CompletePredicates.hs index 7d8cccecd..8105dd89f 100644 --- a/glean/db/Glean/Database/CompletePredicates.hs +++ b/glean/db/Glean/Database/CompletePredicates.hs @@ -27,15 +27,35 @@ import Util.STM import qualified Glean.Database.Catalog as Catalog import Glean.Database.Open import Glean.Database.Schema +import Glean.Database.Schema.Types + ( lookupPredicateSourceRef + , SchemaSelector(..) + ) import qualified Glean.Database.Storage as Storage import Glean.Database.Types import Glean.Internal.Types as Thrift import Glean.Logger import Glean.RTS.Foreign.Ownership +import Glean.RTS.Types (Pid) +import Glean.Schema.Util (convertRef) import Glean.Types as Thrift import qualified Glean.Util.Warden as Warden import Glean.Util.Mutex + +-- | Client API: kick off the completion process for a DB asynchronously +completePredicates + :: Env + -> Repo + -> Thrift.CompletePredicates + -> IO CompletePredicatesResponse +completePredicates env repo complete = + case complete of + Thrift.CompletePredicates_axiom _ -> + completeAxiomPredicates env repo + Thrift.CompletePredicates_derived (Thrift.CompleteDerivedPredicate pred) -> + completeDerivedPredicate env repo pred + -- For internal use: actually perform completion for a DB syncCompletePredicates :: Env -> Repo -> IO () syncCompletePredicates env repo = @@ -64,16 +84,16 @@ syncCompletePredicates env repo = withWriteLock (Just writing) f = withMutex (wrLock writing) $ const f - --- | Client API: kick off the completion process for a DB asynchronously -completePredicates :: Env -> Repo -> IO CompletePredicatesResponse -completePredicates env@Env{..} repo = do +completeAxiomPredicates :: Env -> Repo -> IO CompletePredicatesResponse +completeAxiomPredicates env@Env{..} repo = do let doCompletion = -- we are masked in here (`finally` deregister) $ do r <- tryAll $ syncCompletePredicates env repo case r of - Left (e :: SomeException) -> do setBroken e; throwIO e + Left (e :: SomeException) -> do + setBroken repo envCatalog "completePredicates" e + throwIO e Right{} -> setComplete deregister = @@ -82,21 +102,6 @@ completePredicates env@Env{..} repo = do setComplete = void $ atomically $ Catalog.modifyMeta envCatalog repo $ \meta -> return meta { metaAxiomComplete = True } - - -- If a synchronous exception is thrown during completion, this is - -- not recoverable, so mark the DB as broken. The exception is also - -- propagated to the caller of completePredicates. - setBroken e = void $ atomically $ - Catalog.modifyMeta envCatalog repo $ \meta -> - return meta { metaCompleteness = - Broken (DatabaseBroken "completePredicates" (Text.pack (show e))) } - - waitFor async = do - r <- timeout 1000000 $ wait async - case r of - Nothing -> throwIO $ Retry 1 - Just _ -> return CompletePredicatesResponse{} - mask_ $ do -- speculatively spawn a thread to do the completion, we'll cancel -- this if we don't need it. This is so that we can atomically @@ -121,8 +126,106 @@ completePredicates env@Env{..} repo = do case HashMap.lookup repo completing of Just existingAsync -> do -- in progress later $ cancel async - return $ waitFor existingAsync + return $ waitFor existingAsync CompletePredicatesResponse{} Nothing -> now $ do -- start async completion void $ tryPutTMVar tmvar () modifyTVar envCompleting (HashMap.insert repo async) - return $ waitFor async + return $ waitFor async CompletePredicatesResponse{} + + +-- | Propagate ownership information for an externally derived predicate. +syncCompleteDerivedPredicate :: Env -> Repo -> Pid -> IO () +syncCompleteDerivedPredicate env repo pid = + withOpenDatabase env repo $ \OpenDB{..} -> do + maybeOwnership <- readTVarIO odbOwnership + forM_ maybeOwnership $ \ownership -> do + maybeBase <- repoParent env repo + let withBase repo f = readDatabase env repo $ \_ lookup -> f (Just lookup) + maybe ($ Nothing) withBase maybeBase $ \base -> + withWriteLock odbWriting $ do + computed <- Storage.computeDerivedOwnership odbHandle ownership base pid + Storage.storeOwnership odbHandle computed + where + withWriteLock Nothing f = f + -- if there's no write lock, we must be in the finalization + -- phase and the DB has already been marked read-only. We have + -- exclusive write access at this point so it's safe to + -- continue. + withWriteLock (Just writing) f = + withMutex (wrLock writing) $ const f + +-- | Kick off completion of externally derived predicate asynchronously +completeDerivedPredicate + :: Env + -> Repo + -> PredicateRef + -> IO CompletePredicatesResponse +completeDerivedPredicate env@Env{..} repo pred = do + details <- withOpenDatabase env repo $ \odb -> + predicateDetails (odbSchema odb) pred + let + doCompletion = do -- we are masked in here + r <- tryAll $ + syncCompleteDerivedPredicate env repo (predicatePid details) + case r of + Left (e :: SomeException) -> do + setBroken repo envCatalog "completeDerivedPredicate" e + throwIO e + Right{} -> return () + + mask_ $ do + -- speculatively spawn a thread to do the completion, we'll cancel + -- this if we don't need it. This is so that we can atomically + -- start the job and update the Env state together. + tmvar <- newEmptyTMVarIO + async <- Warden.spawn envWarden $ + atomically (takeTMVar tmvar) >> doCompletion + join $ immediately $ do + meta <- now $ Catalog.readMeta envCatalog repo + if + | not $ metaAxiomComplete meta -> do + now $ throwSTM $ Thrift.Exception "DB is not complete." + | Broken b <- metaCompleteness meta -> do + later $ cancel async + now $ throwSTM $ Exception $ databaseBroken_reason b + | envReadOnly -> do + later $ cancel async + now $ throwSTM $ Exception "DB is read-only" + | otherwise -> do + completing <- now $ readTVar envCompletingDerived + let derivations = HashMap.lookupDefault mempty repo completing + predId = predicateId details + case HashMap.lookup predId derivations of + Just existingAsync -> do -- in progress + later $ cancel async + return $ waitFor existingAsync CompletePredicatesResponse{} + Nothing -> now $ do -- start async completion + void $ tryPutTMVar tmvar () + modifyTVar envCompletingDerived $ + HashMap.insert repo $ + HashMap.insert predId async derivations + return $ waitFor async CompletePredicatesResponse{} + where + predicateDetails schema pred = + case lookupPredicateSourceRef (convertRef pred) LatestSchemaAll schema of + Right details -> return details + Left err -> + throwIO $ Thrift.Exception $ "completeDerivedPredicate: " <> err + +-- If a synchronous exception is thrown during completion, this is +-- not recoverable, so mark the DB as broken. The exception is also +-- propagated to the caller. +setBroken :: Repo -> Catalog.Catalog -> Text.Text -> SomeException -> IO () +setBroken repo catalog context e = void $ atomically $ + Catalog.modifyMeta catalog repo $ \meta -> + return meta { + metaCompleteness = Broken + (DatabaseBroken context (Text.pack (show e))) + } + +waitFor :: Async () -> b -> IO b +waitFor async result = do + r <- timeout 1000000 $ wait async + case r of + Nothing -> throwIO $ Retry 1 + Just _ -> return result diff --git a/glean/db/Glean/Database/Env.hs b/glean/db/Glean/Database/Env.hs index ab08f9c37..fbab99380 100644 --- a/glean/db/Glean/Database/Env.hs +++ b/glean/db/Glean/Database/Env.hs @@ -126,6 +126,8 @@ initEnv evb envStorage envCatalog shardManager cfg envCompleting <- newTVarIO HashMap.empty + envCompletingDerived <- newTVarIO HashMap.empty + envDbSchemaCache <- newMVar HashMap.empty return Env diff --git a/glean/db/Glean/Database/Open.hs b/glean/db/Glean/Database/Open.hs index 1fa4eea01..f0907e97d 100644 --- a/glean/db/Glean/Database/Open.hs +++ b/glean/db/Glean/Database/Open.hs @@ -553,17 +553,21 @@ baseSlices env deps stored_units = case deps of slice <- if exclude && Set.null units then return Nothing else do + let !n = Set.size units vlog 1 $ "computing slice for " <> showRepo repo <> " with " <> - show (Set.toList units) <> - if exclude then " excluded" else " included" + show n <> + (if exclude then " excluded: " else " included: ") <> + show (Set.toList units) unitIds <- fmap catMaybes $ forM (Set.toList units) $ \name -> do id <- Storage.getUnitId odbHandle name vlog 2 $ "unit: " <> show name <> " = " <> show id return id maybeOwnership <- readTVarIO odbOwnership - forM maybeOwnership $ \ownership -> + r <- forM maybeOwnership $ \ownership -> Ownership.slice ownership (catMaybes rest) unitIds exclude + logInfo $ "completed " <> show n <> " slices for " <> showRepo repo + return r return (slice : rest) -- | Create the slices for a stack of dependencies diff --git a/glean/db/Glean/Database/Types.hs b/glean/db/Glean/Database/Types.hs index 6a9e71c21..3bb46f767 100644 --- a/glean/db/Glean/Database/Types.hs +++ b/glean/db/Glean/Database/Types.hs @@ -240,6 +240,8 @@ data Env = forall storage. Storage storage => Env , envActive :: TVar (HashMap Thrift.Repo DB) , envDeleting :: TVar (HashMap Thrift.Repo (Async ())) , envCompleting :: TVar (HashMap Thrift.Repo (Async ())) + , envCompletingDerived :: + TVar (HashMap Thrift.Repo (HashMap PredicateId (Async ()))) , envReadOnly :: Bool , envMockWrites :: Bool , envStats :: Stats diff --git a/glean/db/Glean/Database/Write/Batch.hs b/glean/db/Glean/Database/Write/Batch.hs index 0a34093f5..243319fa9 100644 --- a/glean/db/Glean/Database/Write/Batch.hs +++ b/glean/db/Glean/Database/Write/Batch.hs @@ -15,8 +15,8 @@ module Glean.Database.Write.Batch import Control.Exception import Control.Monad.Extra import qualified Data.ByteString as BS -import Data.Map.Strict (Map) -import qualified Data.Map.Strict as Map +import qualified Data.HashMap.Strict as HashMap +import Data.HashMap.Strict (HashMap) import Data.Coerce import Data.Default import Data.Int (Int64) @@ -24,11 +24,11 @@ import Data.IORef import Data.Maybe import qualified Data.Text as Text import qualified Data.Vector.Storable as Vector -import Data.Vector.Storable (Vector) import Util.Control.Exception import Util.STM +import qualified Glean.Database.Catalog as Catalog import Glean.Database.Open import Glean.Database.Exception import Glean.Database.Repo @@ -36,6 +36,7 @@ import qualified Glean.Database.Storage as Storage import Glean.Database.Schema import Glean.Database.Types import Glean.FFI +import Glean.Internal.Types as Thrift import Glean.RTS.Foreign.FactSet (FactSet) import Glean.RTS.Foreign.Define (trustRefs) import qualified Glean.RTS.Foreign.FactSet as FactSet @@ -45,7 +46,7 @@ import qualified Glean.RTS.Foreign.LookupCache as LookupCache import Glean.RTS.Foreign.Ownership as Ownership import Glean.RTS.Foreign.Subst (Subst) import qualified Glean.RTS.Foreign.Subst as Subst -import Glean.RTS.Types (Pid(..), Fid(..)) +import Glean.RTS.Types (Pid(..)) import Glean.Types (Repo) import qualified Glean.Types as Thrift import Glean.Util.Metric @@ -70,19 +71,27 @@ syncWriteDatabase env repo batch = do makeDefineOwnership :: Env -> Repo - -> Map Int64 (Map Int64 (Vector Int64)) + -> HashMap Int64 [Thrift.FactDependencies] -> IO (Maybe DefineOwnership) makeDefineOwnership env repo deps - | Map.null deps = return Nothing + | HashMap.null deps = return Nothing | otherwise = do + ensureComplete readDatabase env repo $ \odb lookup -> do maybeOwnership <- readTVarIO (odbOwnership odb) forM maybeOwnership $ \ownership -> do nextId <- Lookup.firstFreeId lookup define <- Ownership.newDefineOwnership ownership nextId - forM_ (Map.toList deps) $ \(pid, ownerMap) -> + forM_ (HashMap.toList deps) $ \(pid, ownerMap) -> Ownership.addDerivedOwners lookup define (Pid pid) ownerMap return define + where + -- TODO: enforce that not only the axiom predicates be complete + -- but that every predicate depended upon be complete. + ensureComplete = atomically $ do + meta <- Catalog.readMeta (envCatalog env) repo + unless (metaAxiomComplete meta) $ + throwSTM $ Thrift.Exception "DB is not complete" writeDatabase :: Env @@ -132,7 +141,7 @@ writeDatabase env repo (WriteContent factBatch maybeOwn) latency = if | Just owners <- maybeOwn -> do Ownership.substDefineOwnership owners subst return $ Just owners - | not $ Map.null deps -> + | not $ HashMap.null deps -> makeDefineOwnership env repo deps | otherwise -> return Nothing @@ -207,15 +216,17 @@ writeDatabase env repo (WriteContent factBatch maybeOwn) latency = where batch_size = fromIntegral . BS.length . Thrift.batch_facts substDependencies - :: Subst - -> Map.Map Int64 (Vector.Vector Int64) - -> Map.Map Int64 (Vector.Vector Int64) - substDependencies subst dmap = Map.fromListWith (<>) $ zip keys vals + :: Subst + -> [Thrift.FactDependencies] + -> [Thrift.FactDependencies] + substDependencies subst dmap = map substFD dmap where - !keys = substFid <$> Map.keys dmap - substFid = fromFid . Subst.subst subst . Fid - !vals = under (Subst.substVector subst) <$> Map.elems dmap - under f = Vector.unsafeCast . f . Vector.unsafeCast + substFD (Thrift.FactDependencies facts deps) = + Thrift.FactDependencies facts' deps' + where + !facts' = under (Subst.substVector subst) facts + !deps' = under (Subst.substVector subst) deps + under f = Vector.unsafeCast . f . Vector.unsafeCast withLookupCache :: Repo diff --git a/glean/glass/Glean/Glass/Handler.hs b/glean/glass/Glean/Glass/Handler.hs index dc7976950..4ef3e0544 100644 --- a/glean/glass/Glean/Glass/Handler.hs +++ b/glean/glass/Glean/Glass/Handler.hs @@ -304,6 +304,8 @@ combineDescriptions y x = symbolDescription_annotations y -- <> Maybe [a] , symbolDescription_comments = symbolDescription_comments x <> symbolDescription_comments y + , symbolDescription_pretty_comments = symbolDescription_pretty_comments x <> + symbolDescription_pretty_comments y , symbolDescription_sym_other_locations = filter (/= symbolDescription_sym_location x) $ uniq ( -- collapse duplicate syms diff --git a/glean/hs/Glean/Backend/Retry.hs b/glean/hs/Glean/Backend/Retry.hs index 9794836e0..d7f5eb24b 100644 --- a/glean/hs/Glean/Backend/Retry.hs +++ b/glean/hs/Glean/Backend/Retry.hs @@ -49,8 +49,8 @@ instance Backend RetryBackend where retryChannelExceptions policy $ workHeartbeat backend work workFinished (RetryBackend policy backend) work = retryChannelExceptions policy $ workFinished backend work - completePredicates_ (RetryBackend policy backend) repo = - retryChannelExceptions policy $ completePredicates_ backend repo + completePredicates_ (RetryBackend policy backend) repo preds = + retryChannelExceptions policy $ completePredicates_ backend repo preds restoreDatabase (RetryBackend _ backend) = restoreDatabase backend deleteDatabase (RetryBackend _ backend) = deleteDatabase backend diff --git a/glean/hs/Glean/Backend/Types.hs b/glean/hs/Glean/Backend/Types.hs index f9f72c8b6..fbd70f031 100644 --- a/glean/hs/Glean/Backend/Types.hs +++ b/glean/hs/Glean/Backend/Types.hs @@ -115,6 +115,7 @@ class Backend a where completePredicates_ :: a -> Thrift.Repo + -> Thrift.CompletePredicates -> IO Thrift.CompletePredicatesResponse -- | Request a backed up database (specified via its backup locator) to be @@ -277,9 +278,9 @@ finalize env repo = -- | Notify the server when non-derived predicates are complete. This -- must be called before derivedStored. -completePredicates :: Backend a => a -> Repo -> IO () -completePredicates env repo = - void $ untilDone $ completePredicates_ env repo +completePredicates :: Backend a => a -> Repo -> CompletePredicates -> IO () +completePredicates env repo preds = + void $ untilDone $ completePredicates_ env repo preds untilDone :: IO a -> IO a untilDone io = loop diff --git a/glean/hs/Glean/RTS/Foreign/Ownership.hsc b/glean/hs/Glean/RTS/Foreign/Ownership.hsc index cef48f141..6c0fa4490 100644 --- a/glean/hs/Glean/RTS/Foreign/Ownership.hsc +++ b/glean/hs/Glean/RTS/Foreign/Ownership.hsc @@ -37,10 +37,9 @@ module Glean.RTS.Foreign.Ownership import Control.Exception import Control.Monad import Data.Coerce -import qualified Data.Map.Strict as Map -import Data.Map.Strict (Map) -import qualified Data.Vector.Storable as VS +import Data.List (unzip4) import Data.Text (Text) +import qualified Data.Vector.Storable as VS import Foreign hiding (with) import Foreign.C import TextShow @@ -170,29 +169,35 @@ addDerivedOwners => base -> DefineOwnership -> Pid - -> Map Thrift.Id (VS.Vector Thrift.Id) + -> [Thrift.FactDependencies] -> IO () addDerivedOwners base define (Pid pid) deps = - when (not $ Map.null deps) $ + when (not $ null deps) $ with define $ \define_ptr -> withLookup base $ \base_lookup_ptr -> - withMany entry (Map.toList deps) $ \xs -> - let (fids, fids_ptrs, fids_sizes) = unzip3 xs in - withArray fids $ \p_fids -> - withArray fids_ptrs $ \p_fids_ptrs -> - withArray fids_sizes $ \p_fids_sizes -> + withMany entry deps $ \xs -> + let !(facts_ptrs, facts_sizes, deps_ptrs, deps_sizes) = unzip4 xs in + withArray facts_ptrs $ \p_facts_ptrs -> + withArray facts_sizes $ \p_facts_sizes -> + withArray deps_ptrs $ \p_deps_ptrs -> + withArray deps_sizes $ \p_deps_sizes -> invoke $ glean_define_ownership_add_derived base_lookup_ptr define_ptr (fromIntegral pid) - (fromIntegral $ Map.size deps) - p_fids - p_fids_ptrs - p_fids_sizes + (fromIntegral $ length deps) + p_facts_ptrs + p_facts_sizes + p_deps_ptrs + p_deps_sizes where - entry (fid, fids) f = - VS.unsafeWith (VS.map fromIntegral fids) $ \fids_ptr -> - f (fromIntegral fid, fids_ptr, fromIntegral $ VS.length fids) + entry (Thrift.FactDependencies facts deps) f = + VS.unsafeWith (coerce facts) $ \facts_ptr -> + VS.unsafeWith (coerce deps) $ \deps_ptr -> do + let + !num_facts = fromIntegral $ VS.length facts + !num_deps = fromIntegral $ VS.length deps + f (facts_ptr, num_facts, deps_ptr, num_deps) newtype DerivedFactOwnershipIterator = @@ -327,7 +332,8 @@ foreign import ccall unsafe glean_define_ownership_add_derived -> Ptr DefineOwnership -> Word64 -> CSize - -> Ptr Word64 + -> Ptr (Ptr Word64) + -> Ptr CSize -> Ptr (Ptr Word64) -> Ptr CSize -> IO CString diff --git a/glean/hs/Glean/Typed/BuildFact.hs b/glean/hs/Glean/Typed/BuildFact.hs index 0c717a6ac..c25e0048a 100644 --- a/glean/hs/Glean/Typed/BuildFact.hs +++ b/glean/hs/Glean/Typed/BuildFact.hs @@ -6,13 +6,24 @@ LICENSE file in the root directory of this source tree. -} -{-# LANGUAGE AllowAmbiguousTypes, TypeApplications, CPP #-} +{-# LANGUAGE AllowAmbiguousTypes, TypeApplications, CPP, InstanceSigs #-} module Glean.Typed.BuildFact - ( NewFact(newFact,withUnit), makeFact, makeFact_, makeFactV, makeFactV_ - , Facts, newFacts, serializeFacts, factsMemory - , FactBuilder, buildFacts, extendFacts, buildBatch + ( NewFact(newFact,withUnit, derivedFrom) + , makeFact + , makeFact_ + , makeFactV + , makeFactV_ + , Facts + , newFacts + , serializeFacts + , factsMemory + , FactBuilder + , buildFacts + , extendFacts + , buildBatch ) where +import Data.Coerce import Data.HashMap.Strict (HashMap) import qualified Data.HashMap.Strict as HashMap import Data.Int @@ -50,6 +61,9 @@ class (MonadFail m, Monad m) => NewFact m where -- | Create some facts owned by the given UnitName withUnit ::Thrift.UnitName -> m a -> m a + -- | Set dependencies of an externally derived fact + derivedFrom :: Predicate p => [Fid] -> [p] -> m () + -- | Create a new fact in a 'NewFact' monad and return the corresponding Thrift -- structure which will have 'Just' the passed key and value. makeFactV @@ -82,6 +96,7 @@ data Facts = Facts { factsPredicates :: Predicates , factsData :: FactSet , factsOwnership :: IORef (HashMap Thrift.UnitName [Int64]) + , factsDerivations :: IORef (HashMap Pid [([Fid],[Fid])]) } -- | Create a new empty collection of facts. New facts will be assigned @@ -96,13 +111,26 @@ newFacts ps start = Facts ps <$> FactSet.new (fromMaybe lowestFid start) <*> newIORef HashMap.empty + <*> newIORef HashMap.empty -- | Serialize the facts into a batch which can be sent via Thrift. serializeFacts :: Facts -> IO Thrift.Batch serializeFacts Facts{..} = do batch <- FactSet.serialize factsData ownership <- readIORef factsOwnership - return batch { Thrift.batch_owned = fmap Vector.fromList ownership } + derivations <- readIORef factsDerivations + return batch + { Thrift.batch_owned = fmap Vector.fromList ownership + , Thrift.batch_dependencies = HashMap.fromList + [ (fromPid pid, map toFactDependencies deps) + | (pid, deps) <- HashMap.toList derivations ] + } + where + toFactDependencies :: ([Fid],[Fid]) -> Thrift.FactDependencies + toFactDependencies (deps, facts) = + Thrift.FactDependencies + (Vector.fromList (coerce facts)) + (Vector.fromList (coerce deps)) -- | Return a rough estimate of how much memory is used by the facts. factsMemory :: Facts -> IO Int @@ -144,6 +172,12 @@ instance NewFact FactsM where [fromFid firstId, fromFid lastId - 1] return a + derivedFrom :: forall p. Predicate p => [Fid] -> [p] -> FactsM () + derivedFrom deps facts = FactsM $ do + Facts{..} <- ask + let pid = pidOf (getPid factsPredicates :: PidOf p) + liftIO $ modifyIORef' factsDerivations $ + HashMap.insertWith (<>) pid [(deps, map (idOf . getId) facts)] -- | A fact builder type FactBuilder = forall m. NewFact m => m () diff --git a/glean/if/glean.thrift b/glean/if/glean.thrift index aa80497f4..18c697731 100644 --- a/glean/if/glean.thrift +++ b/glean/if/glean.thrift @@ -27,6 +27,7 @@ namespace py3 glean // Uniquely identifies a fact in a database typedef i64 Id typedef list (hs.type = "VectorStorable") listOfIds +typedef map (hs.type = "HashMap") multimapOfIds const Id INVALID_ID = 0; const Id FIRST_FREE_ID = 1024; @@ -228,6 +229,11 @@ struct Fact { 3: Value value; // value decodes to a term that matches valueType } +struct FactDependencies { + 1: list (hs.type = "VectorStorable") facts; + 2: list (hs.type = "VectorStorable") dependencies; +} + // A collection of facts which can be written to a database. struct Batch { // Id of the first fact in the batch if ids isn't supplied and the boundary @@ -278,7 +284,7 @@ struct Batch { // derive that fact. // // The dependency relation is used to determine ownership. - 6: map> dependencies; + 6: map> (hs.type = "HashMap") dependencies; } struct Subst { @@ -1132,6 +1138,19 @@ struct PredicateStatsOpts { 1: bool excludeBase = true; } +// Complete all base predicates (i.e. non-derived). +struct CompleteAxiomPredicates {} + +// Complete externally derived predicates. +struct CompleteDerivedPredicate { + 1: PredicateRef predicate; +} + +union CompletePredicates { + 1: CompleteAxiomPredicates axiom; + 2: CompleteDerivedPredicate derived; +} (hs.nonempty) + struct CompletePredicatesResponse {} service GleanService extends fb303.FacebookService { @@ -1215,9 +1234,10 @@ service GleanService extends fb303.FacebookService { 6: WrongHandle h, ); - // Tell the server when non-derived predicates are complete. This - // call must be completed successfully before deriveStored() is - // called. + // Tell the server when predicates are complete. + // Axiom predicates must be completed first. Then externally + // derived predicates can be completed and the derivation of + // stored predicates can start. // // Note that the process of completing predicates may take some // time, and the call may return Retry multiple times. You can't @@ -1225,7 +1245,7 @@ service GleanService extends fb303.FacebookService { // successfully. CompletePredicatesResponse completePredicates( 1: Repo repo, - // later: 2: optional list predicates + 2: CompletePredicates predicates, ) throws (1: Exception e, 3: Retry r, 4: UnknownDatabase u); // Wait for a DB to be complete, after the last workFinished diff --git a/glean/rts/ffi.cpp b/glean/rts/ffi.cpp index 5a88628f4..2468c5293 100644 --- a/glean/rts/ffi.cpp +++ b/glean/rts/ffi.cpp @@ -1068,36 +1068,46 @@ namespace { struct DependencyIterator : DerivedDependencyIterator { DependencyIterator( size_t size_, - uint64_t* fids, - uint64_t** fids_lists, - size_t* fids_lists_sizes) - : size_(size_), fids(fids), fids_lists(fids_lists), - fids_lists_sizes(fids_lists_sizes), ix(0) {} - - folly::Optional>> get() override { + uint64_t** facts_lists, + size_t* facts_lists_sizes, + uint64_t** deps_lists, + size_t* deps_lists_sizes) + : size_(size_), facts_lists(facts_lists), facts_lists_sizes(facts_lists_sizes), + deps_lists(deps_lists), deps_lists_sizes(deps_lists_sizes), ix(0) {} + + folly::Optional, std::vector>> get() override { if (ix < size_) { - const auto id = Id::fromThrift(fids[ix]); + std::vector facts; + const auto facts_size = facts_lists_sizes[ix]; + facts.reserve(facts_size); + std::transform( + facts_lists[ix], + facts_lists[ix] + facts_size, + std::back_inserter(facts), + Id::fromThrift + ); std::vector deps; - const auto dep_size = fids_lists_sizes[ix]; + const auto dep_size = deps_lists_sizes[ix]; deps.reserve(dep_size); std::transform( - fids_lists[ix], - fids_lists[ix] + dep_size, + deps_lists[ix], + deps_lists[ix] + dep_size, std::back_inserter(deps), Id::fromThrift ); ix++; - return std::pair(id, std::move(deps)); + return std::pair(std::move(facts), std::move(deps)); } return folly::none; } private: size_t size_; - uint64_t* fids; - uint64_t** fids_lists; - size_t* fids_lists_sizes; + uint64_t** facts_lists; + size_t* facts_lists_sizes; + uint64_t** deps_lists; + size_t* deps_lists_sizes; size_t ix; }; @@ -1108,15 +1118,17 @@ const char *glean_define_ownership_add_derived( DefineOwnership *define, uint64_t pid_raw, size_t fids_size, - uint64_t* fids, - uint64_t** fids_lists, - size_t* fids_lists_sizes) { + uint64_t** facts_lists, + size_t* facts_lists_sizes, + uint64_t** deps_lists, + size_t* deps_lists_sizes) { return ffi::wrap([=] { auto it = DependencyIterator( fids_size, - fids, - fids_lists, - fids_lists_sizes + facts_lists, + facts_lists_sizes, + deps_lists, + deps_lists_sizes ); addDerived(lookup, define, Pid::fromThrift(pid_raw), &it); }); diff --git a/glean/rts/ffi.h b/glean/rts/ffi.h index 4006dc8ea..d92f52ec8 100644 --- a/glean/rts/ffi.h +++ b/glean/rts/ffi.h @@ -626,6 +626,17 @@ const char *glean_define_ownership_sort_by_owner( FactOrder *result ); +const char *glean_define_ownership_add_derived( + Lookup *lookup, + DefineOwnership *define, + uint64_t pid_, + size_t fids_size, + uint64_t** facts_lists, + size_t* facts_lists_sizes, + uint64_t** deps_lists, + size_t* deps_lists_sizes +); + void glean_define_ownership_free(DefineOwnership *def); const char *glean_derived_ownership_compute( diff --git a/glean/rts/ownership/derived.cpp b/glean/rts/ownership/derived.cpp index ac15e48e6..5176bd1c3 100644 --- a/glean/rts/ownership/derived.cpp +++ b/glean/rts/ownership/derived.cpp @@ -252,7 +252,7 @@ void addDerived( Pid pid, DerivedDependencyIterator* it) { while (auto v = it->get()) { - Id id = v->first; + auto& facts = v->first; auto& deps = v->second; std::set owners; for (auto dep : deps) { @@ -264,7 +264,9 @@ void addDerived( } } - define->derivedFrom(pid, id, owners); + for (auto id : facts) { + define->derivedFrom(pid, id, owners); + } } } diff --git a/glean/rts/ownership/derived.h b/glean/rts/ownership/derived.h index e5b37dc1e..4180a298d 100644 --- a/glean/rts/ownership/derived.h +++ b/glean/rts/ownership/derived.h @@ -81,7 +81,7 @@ std::unique_ptr computeDerivedOwnership( struct DerivedDependencyIterator { virtual ~DerivedDependencyIterator () {} - virtual folly::Optional>> get() = 0; + virtual folly::Optional,std::vector>> get() = 0; }; /// diff --git a/glean/schema/source/csharp.angle b/glean/schema/source/csharp.angle new file mode 100644 index 000000000..cc50c6e9a --- /dev/null +++ b/glean/schema/source/csharp.angle @@ -0,0 +1,41 @@ +schema csharp.1 { + import src.1 + +# AlphaNamespace and type names (§7.8) +# https://fburl.com/d94ubqay +predicate AlphaName : string + +# AlphaNamespace declarations (§13.3) +# https://fburl.com/gv0o62tb +predicate AlphaNamespace : + { + name : maybe AlphaName, + containing_namespace : maybe AlphaNamespace, + is_global : bool, + locations : [src.FileLocation], + } + +# Classes (§14) +# https://fburl.com/mb38zzjh +predicate AlphaClassDefinition : + { + name : AlphaName, + containing_namespace : maybe AlphaNamespace, + locations : [src.FileLocation], + } + +# Projects (.csproj) +# https://fburl.com/0axo3xpz +predicate AlphaProject : + { + solution : maybe AlphaSolution, + file : src.File, + } + +# Solutions (.sln) +# https://fburl.com/var17xpf +predicate AlphaSolution : + { + file : src.File, + } +} diff --git a/glean/server/Glean/Handler.hs b/glean/server/Glean/Handler.hs index 198bbd8c5..998f8212e 100644 --- a/glean/server/Glean/Handler.hs +++ b/glean/server/Glean/Handler.hs @@ -76,8 +76,8 @@ handler State{..} req = Service.UserQueryBatch repo queries -> do Backend.userQueryBatch backend repo queries - Service.CompletePredicates repo -> - Backend.completePredicates_ backend repo + Service.CompletePredicates repo preds -> + Backend.completePredicates_ backend repo preds Service.DeriveStored repo pred -> Backend.deriveStored backend (const mempty) repo pred diff --git a/glean/test/tests/DbDeriveTest.hs b/glean/test/tests/DbDeriveTest.hs index 35db2cb09..6cc38c61b 100644 --- a/glean/test/tests/DbDeriveTest.hs +++ b/glean/test/tests/DbDeriveTest.hs @@ -488,7 +488,7 @@ deriveIncrementalTest = TestLabel "incremental" $ TestList let base = Repo "base" "0" kickOffTestDB env base id void $ syncWriteJsonBatch env base baseFacts Nothing - void $ completePredicates env base + void $ completePredicates env base (CompletePredicates_axiom def) _ <- deriveStored' env base opts' pref completeTestDB env base diff --git a/glean/test/tests/GracefulShutdown.hs b/glean/test/tests/GracefulShutdown.hs index 8283e4c0d..a408d8648 100644 --- a/glean/test/tests/GracefulShutdown.hs +++ b/glean/test/tests/GracefulShutdown.hs @@ -64,6 +64,7 @@ import Glean.Remote ( withRemoteBackend, ) import Glean.Server.Spawn (withServer) +import qualified Glean.Types as Glean import Glean.Util.ConfigProvider ( ConfigProvider (defaultConfigOptions, withConfigProvider), ) @@ -128,7 +129,7 @@ waitForIncompleteDBs = setupOutOfProcessServer 10000 $ \ph err backend -> do _ <- async $ evaluate (rnf $ length err) -- finish the DB and check the server exits - Glean.completePredicates backend repo + Glean.completePredicates backend repo (Glean.CompletePredicates_axiom def) workFinished backend WorkFinished diff --git a/glean/test/tests/IncrementalTest.hs b/glean/test/tests/IncrementalTest.hs index d26ede86a..842f4bcb3 100644 --- a/glean/test/tests/IncrementalTest.hs +++ b/glean/test/tests/IncrementalTest.hs @@ -198,7 +198,7 @@ stackedIncrementalTest = TestCase $ let deriveAndFinish :: Env -> Repo -> IO () deriveAndFinish env repo = do - void $ completePredicates env repo + void $ completePredicates env repo (CompletePredicates_axiom def) derivePredicate env repo Nothing Nothing (parseRef "glean.test.RevEdge") Nothing derivePredicate env repo Nothing Nothing @@ -431,7 +431,7 @@ stackedIncrementalTest2 = TestCase $ let deriveAndFinish :: Env -> Repo -> IO () deriveAndFinish env repo = do - void $ completePredicates env repo + void $ completePredicates env repo (CompletePredicates_axiom def) derivePredicate env repo Nothing Nothing (parseRef "glean.test.NodePair") Nothing completeTestDB env repo @@ -607,13 +607,86 @@ orphanTest = TestCase $ (rec $ field @"child" (rec $ field @"label" (string "d") end) end) assertEqual "orphan 3" 1 (length results) +externalDerivationTest :: Test +externalDerivationTest = TestList + [ TestLabel "add fact with dependencies" $ TestCase $ + withDB $ \env repo -> do + writeFactsIntoDB env repo [ Glean.Test.allPredicates ] $ do + a <- makeFact @Glean.Test.Node (Glean.Test.Node_key "a") + b <- makeFact @Glean.Test.Node (Glean.Test.Node_key "b") + p <- makeFact @Glean.Test.StringPair (Glean.Test.StringPair_key "a" "b") + derivedFrom [idOf (getId a), idOf (getId b)] [p] + + void $ completePredicates env repo $ CompletePredicates_derived $ + CompleteDerivedPredicate $ + PredicateRef "glean.test.StringPair" 1 + + owners <- ownersOf env repo $ query $ + predicate @Glean.Test.StringPair wild + + assertEqual "owners" + (show [Just $ AndOwners + [ OrOwners [Unit "B"] + , OrOwners [Unit "A"] + ] ]) + (show owners) + + , TestLabel "multiple predicates in the same batch" $ TestCase $ + withDB $ \env repo -> do + writeFactsIntoDB env repo [ Glean.Test.allPredicates ] $ do + a <- makeFact @Glean.Test.Node (Glean.Test.Node_key "a") + p <- makeFact @Glean.Test.StringPair (Glean.Test.StringPair_key "a" "a") + derivedFrom [idOf (getId a)] [p] + + b <- makeFact @Glean.Test.Node (Glean.Test.Node_key "b") + n <- makeFact @Glean.Test.Name "b" + derivedFrom [idOf (getId b)] [n] + + void $ completePredicates env repo $ CompletePredicates_derived $ + CompleteDerivedPredicate $ + PredicateRef "glean.test.StringPair" 1 + + void $ completePredicates env repo $ CompletePredicates_derived $ + CompleteDerivedPredicate $ + PredicateRef "glean.test.Name" 1 + + owners <- ownersOf env repo $ query $ + predicate @Glean.Test.StringPair wild + assertEqual "StringPair owners" + (show [Just $ OrOwners [Unit "A"]]) + (show owners) + + owners <- ownersOf env repo $ query $ + predicate @Glean.Test.Name wild + assertEqual "Name owners" + (show [Just $ OrOwners [Unit "B"]]) + (show owners) + ] + where + ownersOf env repo q = do + results <- runQuery_ env repo q + let fids = map (idOf . getId) results + traverse (factOwnership env repo) fids + + withDB act = + withTestEnv [setCompactOnCompletion] $ \env -> do + let repo = Repo "base" "0" + kickOffTestDB env repo id + writeFactsIntoDB env repo [ Glean.Test.allPredicates ] $ do + void $ withUnit "A" $ + makeFact @Glean.Test.Node (Glean.Test.Node_key "a") + void $ withUnit "B" $ + makeFact @Glean.Test.Node (Glean.Test.Node_key "b") + void $ completePredicates env repo $ CompletePredicates_axiom def + act env repo + deriveTest :: Test deriveTest = TestCase $ withTestEnv [setCompactOnCompletion] $ \env -> do let base = Repo "base" "0" kickOffTestDB env base id mkGraph env base - void $ completePredicates env base + void $ completePredicates env base (CompletePredicates_axiom def) derivePredicate env base Nothing Nothing (parseRef "glean.test.RevEdge") Nothing @@ -723,6 +796,7 @@ main_ = withUnitTest $ testRunner $ TestList , TestLabel "dupSetTest" dupSetTest , TestLabel "orphanTest" orphanTest , TestLabel "deriveTest" deriveTest + , TestLabel "externalDerivationTest" externalDerivationTest , TestLabel "stackedIncrementalTest" stackedIncrementalTest , TestLabel "stackedIncrementalTest2" stackedIncrementalTest2 , TestLabel "restartIndexing" restartIndexing diff --git a/glean/tools/gleancli/GleanCLI/Complete.hs b/glean/tools/gleancli/GleanCLI/Complete.hs index d19e26f88..f497bbe10 100644 --- a/glean/tools/gleancli/GleanCLI/Complete.hs +++ b/glean/tools/gleancli/GleanCLI/Complete.hs @@ -10,6 +10,7 @@ module GleanCLI.Complete (CompleteCommand) where import Control.Monad +import Data.Default (def) import Options.Applicative import Util.IO @@ -20,6 +21,7 @@ import GleanCLI.Types import Glean import Glean.Schema.Util +import qualified Glean.Types as Thrift data CompleteCommand = Complete @@ -44,4 +46,5 @@ instance Plugin CompleteCommand where when (not (null completePredicates)) $ die 1 "completing individual predicates is not supported yet" - void $ Glean.completePredicates backend completeRepo + void $ Glean.completePredicates backend completeRepo $ + Thrift.CompletePredicates_axiom def diff --git a/thrift/annotation/cpp.thrift b/thrift/annotation/cpp.thrift index 4d1456bdc..777256837 100644 --- a/thrift/annotation/cpp.thrift +++ b/thrift/annotation/cpp.thrift @@ -286,3 +286,12 @@ struct Frozen2RequiresCompleteContainerParams {} @scope.Interaction @scope.Function struct GenerateTypedInterceptor {} + +/** + * Causes C++ handler code to run inline on the EventBase thread. + * Disables overload protection, use with caution. + * Cannot be applied to individual functions in interactions. + */ +@scope.Function +@scope.Interaction +struct ProcessInEbThreadUnsafe {} diff --git a/thrift/annotation/thrift.thrift b/thrift/annotation/thrift.thrift index fe9b09af1..d03cab14d 100644 --- a/thrift/annotation/thrift.thrift +++ b/thrift/annotation/thrift.thrift @@ -128,17 +128,10 @@ struct SerializeInFieldIdOrder {} struct BitmaskEnum {} /** - * Specifies the field where the exception message is stored. The field - * is used to generate an additional method to get it. + * Specifies the field where the exception message is stored. */ -// TODO(afuller): Consider allowing this annotation to be placed on the field -// itself, so the annotation can be used (transitively), without specifying an -// explicit name. Also, maybe move to api.thrift. -@scope.Exception -@Experimental // TODO: Support in C++, Python, Java. -struct ExceptionMessage { - 1: string field; -} +@scope.Field +struct ExceptionMessage {} /** * Generates a const of type schema. Struct containing the schema of the @@ -162,3 +155,9 @@ struct GenerateRuntimeSchema { @scope.Field @Experimental struct InternBox {} + +/** + * Indicates that an interaction's methods should be processed sequentially. + */ +@scope.Interaction +struct Serial {}