{- Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved. This source code is licensed under the BSD-style license found in the LICENSE file in the root directory of this source tree. -} {-# LANGUAGE RecursiveDo #-} module Glean.Query.Codegen ( compileQuery , compileQueryFacts , Boundaries , flatBoundaries , stackedBoundaries , skipTrusted , buildTerm ) where import Control.Exception import Control.Monad import Control.Monad.Extra (whenJust) import Control.Monad.State import Data.Bifunctor (bimap) import Data.ByteString (ByteString) import Data.Coerce import Data.IntSet (IntSet) import qualified Data.IntSet as IntSet import Data.List (find,genericLength) import Data.Maybe import qualified Data.Text as Text import qualified Data.Vector as Vector import Data.Vector (Vector, (!), (//)) import Data.Word import Foreign.Ptr hiding (WordPtr) import System.IO.Unsafe import qualified Util.FFI as FFI import qualified Util.Log as StringLog import qualified Util.Log.Text as TextLog import Glean.Query.Transform ( skipTrusted , isWordTy , isByteTy , buildTerm , transformType , transformBytes , defaultValue) import Glean.Angle.Types (IsWild(..), tempPredicateId) import qualified Glean.Angle.Types as Angle import Glean.Bytecode.Types import Glean.Display import qualified Glean.FFI as FFI import Glean.Query.Codegen.QueryRegs import Glean.Query.Codegen.Types import Glean.Database.Types (EnableRecursion(..)) import Glean.Database.Schema.Types ( Bytes(..) , PredicateTransformation(..) , QueryTransformations , IsPointQuery , lookupTransformation) import Glean.RTS import Glean.RTS.Builder import Glean.RTS.Bytecode.Code import Glean.RTS.Bytecode.Disassemble import Glean.RTS.Bytecode.Gen.Issue import Glean.RTS.Foreign.Bytecode import Glean.RTS.Foreign.Lookup (Lookup) import qualified Glean.RTS.Foreign.Lookup as Lookup import Glean.RTS.Foreign.Query import Glean.RTS.Traverse import Glean.RTS.Types import Glean.RTS.Term import Glean.Typed.Binary (buildRtsValue) import Glean.Types hiding (Nat, Byte) import Glean.Bytecode.SysCalls (userQuerySysCalls) {- Debugging the bytecode query backend: Use glog verbosity levels to enable logging. For example, to enable verbosity level 2, you can do GLOG_v=2 (The environment variable GLOG_v is often more reliable than passing -v 3 on the command line, since it bypasses the program's own command-line parser). Verbosity levels are as follows: 2: log the query at different stages (typechecking, expansion, ...) 3: disassembles the generated bytecode 5: tracing during bytecode execution (see rts/query.cpp) -} {- Note [pausing/resuming queries] To support paging queries we need to be able to pause and resume a query in progress. This is achieved by capturing the state of the bytecode evaluator during the query, and serializing it so that it can be returned to the user. Later the user sends back the serialized query state, and we deserialize it and continue executing it. Roughly speaking this is achieved as follows: * The saved state of the evaluator is described by the Thrift type QueryCont in glean/if/internal.thrift. We use Thrift for serializing and deserializing the state. * The state includes: - the code, instruction pointer and local registers - the state of any fact iterators generated by 'seek' - the contents of binary::Output registers created by 'output' * The bytecode instruction Suspend allows the subroutine to be suspended by updating the activation record such that execution can be resumed and then returning an appropriate status from the evaluator. To make this work we have to obey a rule in the bytecode program: Don't keep any pointers in local registers across Suspend because they won't be valid when we resume. Pointers in input registers are fine; the contents of these will be saved and restored as necessary (this includes binary::Output registers). -} -- | Find all variables that need to be bound to a binary::Output findOutputs :: [CgStatement] -> IntSet findOutputs stmts = findOutputsStmts stmts IntSet.empty where findOutputsStmts :: [CgStatement] -> IntSet -> IntSet findOutputsStmts stmts r = foldr findOutputsStmt r stmts findOutputsStmt :: CgStatement -> IntSet -> IntSet findOutputsStmt (CgStatement lhs gen) r = findOutputsGen gen (foldr findOutputsMatch r lhs) findOutputsStmt (CgAllStatement (Var _ var _) expr stmts) r = IntSet.insert var $ findOutputsPat expr $ foldr findOutputsStmt r stmts findOutputsStmt (CgNegation stmts) r = foldr findOutputsStmt r stmts findOutputsStmt (CgDisjunction stmtss) r = foldr (flip (foldr findOutputsStmt)) r stmtss findOutputsStmt (CgConditional cond then_ else_) r = foldr (flip (foldr findOutputsStmt)) r [cond, then_, else_] findOutputsGen :: Generator -> IntSet -> IntSet findOutputsGen (FactGenerator _ kpat vpat _) r = foldr findOutputsMatch (foldr findOutputsMatch r vpat) kpat findOutputsGen (TermGenerator t) r = findOutputsPat t r findOutputsGen (DerivedFactGenerator _ k v) r = findOutputsPat k (findOutputsPat v r) findOutputsGen (ArrayElementGenerator _ _) r = r findOutputsGen (SetElementGenerator _ _) r = r findOutputsGen (PrimCall _ args _) r = foldr findOutputsPat r args findOutputsPat pat r = foldr findOutputsMatch r pat findOutputsMatch :: Match () Var -> IntSet -> IntSet findOutputsMatch (MatchPrefix _ rest) r = foldr findOutputsMatch r rest findOutputsMatch (MatchBind (Var ty var _)) r | not (isWordTy ty) = IntSet.insert var r findOutputsMatch (MatchAnd a b) r = findOutputsPat a (findOutputsPat b r) findOutputsMatch (MatchArrayPrefix _ exps all) r = foldr findOutputsPat r (all:exps) findOutputsMatch _ r = r -- | The database boundaries we are interested in to implement -- sectioned 'seek' calls. data Boundaries = FlatBoundaries SectionBoundaries | StackedBoundaries SectionBoundaries SectionBoundaries -- | Limits of a section of a database. Represents the range [start, end). data SectionBoundaries = SectionBoundaries { _section_start :: Fid , _section_end :: Fid } flatBoundaries :: Lookup -> IO Boundaries flatBoundaries lookup = FlatBoundaries <$> sectionBounds lookup stackedBoundaries :: Lookup -> Lookup -> IO Boundaries stackedBoundaries base added = StackedBoundaries <$> sectionBounds base <*> sectionBounds added sectionBounds :: Lookup -> IO SectionBoundaries sectionBounds lookup = SectionBoundaries <$> Lookup.startingId lookup <*> Lookup.firstFreeId lookup compileQuery :: EnableRecursion -> QueryTransformations -> Boundaries -> CodegenQuery -- ^ The query to compile. NB. no type checking or validation is -- done on this; we assume that earlier phases have done this. A -- malformed query can cause a crash. -> IO CompiledQuery compileQuery r qtrans bounds (QueryWithInfo query numVars lookup ty) = do StringLog.vlog 2 $ show (displayDefault query) (idTerm, resultKey, resultValue, stmts) <- if | Just gen@(FactGenerator _ keyTerm valTerm _) <- lookup, CgQuery idTerm stmts <- query -> do let lookup = CgStatement idTerm gen let unBind (Ref (MatchBind x)) = Ref (MatchVar x) unBind (Ref (MatchWild (Angle.RecordTy []))) = Tuple [] unBind _ = error "unBind" return (idTerm, unBind keyTerm, unBind valTerm, stmts <> [lookup]) | CgQuery (Tuple [idTerm, resultKey, resultValue]) stmts <- query -> return (idTerm, resultKey, resultValue, stmts) | otherwise -> throwIO $ BadQuery "unsupported query" (Meta{..}, sub) <- generateQueryCode $ \ regs@QueryRegs{..} -> do let outputVars = IntSet.toList $ findOutputs stmts outputUninitialized $ Many (length outputVars) $ \outputRegs -> do let outputRegAssocs :: [(Int, Register 'BinaryOutputPtr)] outputRegAssocs = zip outputVars outputRegs local $ Many (numVars - length outputVars) $ \localRegs -> do let localRegAssocs :: [(Int, Register 'Word)] localRegAssocs = zip (filter (`notElem` outputVars) [0..]) localRegs -- vector of variables corresponding to the vars in the original query let vars = Vector.replicate numVars (error "compileQuery") // (localRegAssocs ++ coerce outputRegAssocs) -- resultKeyReg/resultValueReg is where we build up result values outputUninitialized $ \resultKeyOutput resultValueOutput -> let code :: forall a. Code a -> Code a code = compileStatements regs qtrans bounds regs stmts vars queryStmts :: forall a. Code a -> Code a queryStmts = case r of EnableRecursion -> recursive regs code code DisableRecursion -> code in queryStmts $ mdo -- If the result term is a variable, avoid unnecessarily -- copying it into resultOutput and just use it directly. resultKeyReg <- case resultKey of Ref (MatchVar (Var ty v _)) | not (isWordTy ty) -> return (castRegister (vars ! v)) _other -> do resetOutput resultKeyOutput buildTerm resultKeyOutput vars resultKey return resultKeyOutput resultValReg <- case resultValue of Ref (MatchVar (Var ty v _)) | not (isWordTy ty) -> return (Just (castRegister (vars ! v))) Tuple [] -> return Nothing _other -> do resetOutput resultValueOutput buildTerm resultValueOutput vars resultValue return (Just resultValueOutput) idReg <- case idTerm of Ref (MatchVar (Var ty idVar _)) | isWordTy ty -> return (vars ! idVar) Ref (MatchFid fid) -> constant (fromIntegral (fromFid fid)) _ -> error "unsupported result type" local $ \len -> mdo val <- case resultValReg of Nothing -> castRegister <$> constant 0 Just reg -> return reg result idReg resultKeyReg val len jumpIf0 len continue decrAndJumpIf0 maxResults pause -- check whether we have exceeded maxBytes. Note that we -- will return more than max_bytes, but this way we don't -- have to backtrack and regenerate the most recent result -- again in the continuation. jumpIfGt len maxBytes pause sub len maxBytes jump continue pause <- label suspend saveState continue -- see Note [pausing/resuming queries] continue <- label return () ret TextLog.vlog 3 $ Text.unlines $ disassemble "Query" userQuerySysCalls sub -- Tell the query engine how to traverse results for expanding -- nested facts. (pid, traverse) <- case derefType ty of Angle.RecordTy [ pidfield, key, val ] | Angle.PredicateTy _ (PidRef pid ref) <- derefType (Angle.fieldDefType pidfield) -> do traverse <- if ref == tempPredicateId then Just <$> -- detect temporary predicates, see Glean.Query.Flatten.captureKey -- TODO: matching like this is a bit janky. genTraversal -- If the key and value have been transformed they will have -- the correct shape but the facts they refer to will have the -- types available in the db and not the type requested in the -- query. Therefore we must transform the type of those -- references for the traversal. (transformType qtrans $ Angle.fieldDefType key) (transformType qtrans $ Angle.fieldDefType val) else return Nothing return (Just pid, traverse) _other -> throwIO $ ErrorCall "unrecognised query return type" return (CompiledQuery sub pid traverse meta_fullScans) isEmptyTy :: Type -> Bool isEmptyTy ty | TupleRep [] <- repType ty = True | otherwise = False patIsExactFid :: Vector (Register 'Word) -> Term (Match () Var) -> Maybe (Label -> Code (Register 'Word)) patIsExactFid vars pat = case pat of Ref (MatchFid fid) -> Just $ \_ -> constant (fromIntegral (fromFid fid)) Ref (MatchVar (Var ty v _)) | PredicateRep{} <- repType ty -> Just $ \_ -> return (vars ! v) Nat n -> Just $ \_ -> constant n -- if the typechecker allows it Ref (MatchAnd p q) -> case (patIsExactFid vars p, patIsExactFid vars q) of -- the MatchAnd cases are a bit silly, but who knows what earlier -- transformations might generate. (Just l, Just r) -> Just $ \fail -> do r1 <- l fail r2 <- r fail jumpIfNe r1 r2 fail -- patterns better be equal return r1 (Just l, Nothing) | Just cmpR <- cmpWordPat vars q -> Just $ \fail -> do reg <- l fail cmpR reg fail return reg (Nothing, Just r) | Just cmpR <- cmpWordPat vars p -> Just $ \fail -> do reg <- r fail cmpR reg fail return reg _other -> Nothing _other -> Nothing -- | if the given pattern is represented by a word register, then -- return the code to compare a value in the given register against -- the pattern. -- -- A @Term (Match Var)@ doesn't have enough information to reconstruct -- its 'Type', because in particular we can't reconstruct a 'SumTy' -- from an 'Alt' value. But it does have enough information to derive -- its representation, so we can tell by looking at a pattern whether -- it is represented as a word. cmpWordPat :: Vector (Register 'Word) -> Term (Match () Var) -> Maybe (Register 'Word -> Label -> Code ()) cmpWordPat vars pat = case pat of Byte b -> Just $ \reg fail -> do r <- constant (fromIntegral b) jumpIfNe reg r fail Nat n -> Just $ \reg fail -> do r <- constant n jumpIfNe reg r fail Ref (MatchFid fid) -> Just $ \reg fail -> do r <- constant (fromIntegral (fromFid fid)) jumpIfNe reg r fail Ref (MatchWild ty) | isWordTy ty -> Just $ \_ _ -> return () Ref (MatchNever ty) | isWordTy ty -> Just $ \_ fail -> jump fail Ref (MatchVar (Var ty v _)) | isWordTy ty -> Just $ \reg fail -> jumpIfNe reg (vars ! v) fail Ref (MatchAnd a b) -> case (cmpWordPat vars a, cmpWordPat vars b) of (Nothing, Nothing) -> Nothing (Just f, Nothing) -> Just f (Nothing, Just f) -> Just f (Just f, Just g) -> Just $ \reg fail -> f reg fail >> g reg fail Ref (MatchBind (Var ty v _)) | isWordTy ty -> Just $ \reg _ -> move reg (vars ! v) _otherwise -> Nothing -- | Compare a value in an output register with a pattern. If the -- pattern matches, fall through, otherwise jump to the given label. cmpOutputPat :: QueryRegs -> Register 'BinaryOutputPtr -- ^ register containing the value -> [QueryChunk Output] -- ^ pattern to match against -> Label -- ^ jump here on match failure -> Code () cmpOutputPat syscalls reg pat fail = local $ \ptr begin -> do getOutput reg begin ptr matchPat syscalls (Bytes begin ptr) fail pat compileTermGen :: Expr -> Vector (Register 'Word) -- ^ registers for variables -> Maybe (Register 'BinaryOutputPtr) -- ^ where to put the output -> Code a -- ^ @andThen@: code to insert after -- the result is constructed. -> Code a compileTermGen term vars maybeReg andThen = do forM_ maybeReg $ \resultOutput -> case term of Nat n -> loadConst (fromIntegral n) (castRegister resultOutput) Byte n -> loadConst (fromIntegral n) (castRegister resultOutput) Ref (MatchFid f) -> loadConst (fromIntegral (fromFid f)) (castRegister resultOutput) Ref (MatchVar (Var ty v _)) | isWordTy ty -> move (vars ! v) (castRegister resultOutput) _other -> do resetOutput resultOutput buildTerm resultOutput vars term andThen compileStatements :: forall a . QueryRegs -> QueryTransformations -> Boundaries -> QueryRegs -> [CgStatement] -> Vector (Register 'Word) -- ^ registers for variables -> Code a -- ^ @andThen@: code to insert after -- the result is constructed. -> Code a compileStatements syscalls qtrans bounds regs@QueryRegs{..} stmts vars andThen = compile stmts where compile [] = andThen compile (CgConditional cond then_ else_: rest) = local $ \failed innerRet -> mdo let compileBranch stmts = compileStatements syscalls qtrans bounds regs stmts vars $ mdo site <- callSite loadLabel ret innerRet jump doInner ret <- label return site -- if loadConst 1 failed thenSite <- compileStatements syscalls qtrans bounds regs cond vars $ do -- then loadConst 0 failed compileBranch then_ -- else jumpIf0 failed done elseSite <- compileBranch else_ jump done doInner <- label a <- calledFrom [thenSite, elseSite] $ compile rest jumpReg innerRet done <- label return a compile (CgStatement pat gen : rest) = if isExistenceCheck then singleResult (compileStmt pat gen) $ compile rest else compileStmt pat gen $ compile rest where -- An existence check is a statement whose sole purpose is to assert -- whether there is any entry in the database that satisfies it. -- Variables bound by an existencial check are not used by subsequent -- statements. -- -- Given the environment is not meaningfully changed, all executions -- of the code after it will be identical. This means we can stop -- after going through the first result. -- -- For now instead of checking the use of bound variables we will -- just say that a statement is an existence check if it doesn't bind -- any variables at all. isExistenceCheck = case gen of FactGenerator _ k v _ -> let bindsAnywhere = bindsIn pat || bindsIn k || bindsIn v in not bindsAnywhere _ -> False -- does the pattern bind a variable? bindsIn pat = flip any (matches pat) $ \case MatchBind _ -> True MatchWild _ -> False MatchNever _ -> False MatchFid _ -> False MatchVar _ -> False MatchAnd l r -> bindsIn l || bindsIn r MatchPrefix _ exp -> bindsIn exp MatchArrayPrefix _ exps all -> any bindsIn exps || bindsIn all MatchExt _ -> False matches :: Term (Match e v) -> [Match e v] matches p = foldMap pure p compile (CgAllStatement (Var (Angle.SetTy ty) v _) expr stmts : rest) | isWordTy ty = do local $ \setReg -> do let set = castRegister setReg newWordSet set compileStatements syscalls qtrans bounds regs stmts vars $ local $ \reg -> do compileTermGen expr vars (Just reg) $ insertWordSet set (castRegister reg) resetOutput (castRegister (vars!v)) if isByteTy ty then byteSetToByteArray set (castRegister (vars!v)) else wordSetToArray set (castRegister (vars!v)) freeWordSet set compile rest compile (CgAllStatement (Var _ v _) expr stmts : rest) = do local $ \setReg -> do let set = castRegister setReg newSet set compileStatements syscalls qtrans bounds regs stmts vars $ outputUninitialized $ \out -> do compileTermGen expr vars (Just out) $ insertOutputSet set out setToArray set (castRegister (vars!v)) freeSet set compile rest compile (CgNegation stmts : rest) = mdo singleResult (compileStatements syscalls qtrans bounds regs stmts vars) (jump fail) a <- compile rest fail <- label return a -- an empty list of generators should fall through without -- executing inner, but we have to compile inner because we need -- its result. compile (CgDisjunction [] : rest) = mdo jump fail a <- compile rest fail <- label return a compile (CgDisjunction [one] : rest) = compile (one ++ rest) -- To compile sequential generators, we simply compile each generator, -- and when it produces a result we "call" the continuation to -- handle the rest of the query. As each generator completes, -- it falls through to the next one. -- -- "calling" the continuation is done by explicitly passing the -- return address, because we don't have a stack (yet). Some -- minor shenanigans are involved to ensure that the "called" -- routine's local registers don't clash with its "call site". compile (CgDisjunction stmtss : rest) = local $ \innerRet -> mdo sites <- forM stmtss $ \stmts -> do compileStatements syscalls qtrans bounds regs stmts vars $ mdo site <- callSite loadLabel ret_ innerRet jump doInner ret_ <- label return site jump done doInner <- label a <- calledFrom sites $ compile rest jumpReg innerRet done <- label return a compileStmt :: forall a. Pat -> Generator -> Code a -> Code a compileStmt pat gen continue = case (pat, gen) of (Ref (MatchWild _), gen) -> compileGen gen Nothing continue (Ref (MatchBind (Var _ var _)), gen) -> compileGen gen (Just (vars ! var)) continue -- = is a lookup if is known (pat, FactGenerator pref@(PidRef pid _) kpat vpat _) | Just load <- patIsExactFid vars pat -> mdo let patOutput :: forall a . [QueryChunk Output] -> ( Register 'BinaryOutputPtr -- load bytes here -> (Label -> Code ()) -- test the loaded bytes -> Code a) -> Code a patOutput chunks cont | all isWild chunks = do reg <- constant 0 -- null means "don't copy the key/value" cont (castRegister reg) (\_ -> return ()) | [QueryBind (Typed ty out)] <- chunks , not (isWordTy ty) = -- bind into the output directly cont out (\_ -> return ()) | otherwise = outputUninitialized $ \reg -> cont reg (\fail -> cmpOutputPat syscalls reg chunks fail) mtrans :: Maybe PredicateTransformation mtrans = lookupTransformation pid qtrans -- The pid we expect to retrieve from the database PidRef expected _ = maybe pref tAvailable mtrans noTrans _ v f = f v transKeyPat = fromMaybe noTrans $ (transformKeyPattern =<< mtrans) <*> Just syscalls transValPat = fromMaybe noTrans $ (transformValuePattern =<< mtrans) <*> Just syscalls a <- transKeyPat (matchDef syscalls fail) (inlineVars vars kpat) $ \kpat' -> transValPat (matchDef syscalls fail) (inlineVars vars vpat) $ \vpat' -> do patOutput (preProcessPat kpat') $ \kout kcmp -> do patOutput (preProcessPat vpat') $ \vout vcmp -> do reg <- load fail local $ \pidReg -> mdo lookupKeyValue reg kout vout pidReg -- TODO: if this is a trusted fact ID (i.e. not supplied by -- the user) then we could skip this test. expectedReg <- constant (fromIntegral (fromPid expected)) jumpIfEq pidReg expectedReg ok raise "fact has the wrong type" ok <- label return () kcmp fail vcmp fail continue fail <- label return a -- ToDO: push the pat into compileGen and match it eagerly, save -- some copying. (pat, gen) -> let outReg | Just{} <- maybeWordFilter = local | otherwise = \f -> outputUninitialized $ \r -> f $ castRegister (r :: Register 'BinaryOutputPtr) maybeWordFilter = cmpWordPat vars pat filterPat reg pat fail | Just cmp <- maybeWordFilter = cmp reg fail | otherwise = cmpOutputPat syscalls (castRegister reg) chunks fail where chunks = preProcessPat $ inlineVars vars pat in outReg $ \reg -> compileGen gen (Just reg) $ mdo filterPat reg pat fail a <- continue fail <- label return a -- Helper function for processing numeric primitive operations compileGenNumericPrim :: forall a . Expr -> Expr -> Maybe (Register 'Word) -> Code a -> (Register 'Word -> Register 'Word -> Label -> Code ()) -> Code a compileGenNumericPrim p q maybeReg inner failOp = withNatTerm vars p $ \a -> withNatTerm vars q $ \b -> mdo failOp (castRegister a) (castRegister b) fail whenJust maybeReg (resetOutput . castRegister) r <- inner fail <- label return r compileGen :: forall a . Generator -> Maybe (Register 'Word) -> Code a -> Code a compileGen (TermGenerator term) maybeReg inner = do compileTermGen term vars (coerce maybeReg) inner compileGen (PrimCall PrimOpNeNat [p, q] _) maybeReg inner = compileGenNumericPrim p q maybeReg inner $ \a b fail -> jumpIfEq a b fail compileGen (PrimCall PrimOpGtNat [p, q] _) maybeReg inner = compileGenNumericPrim p q maybeReg inner $ \a b fail -> jumpIfLe a b fail compileGen (PrimCall PrimOpGeNat [p, q] _) maybeReg inner = compileGenNumericPrim p q maybeReg inner $ \a b fail -> jumpIfLt a b fail compileGen (PrimCall PrimOpLtNat [p, q] _) maybeReg inner = compileGenNumericPrim p q maybeReg inner $ \a b fail -> jumpIfGe a b fail compileGen (PrimCall PrimOpLeNat [p, q] _) maybeReg inner = compileGenNumericPrim p q maybeReg inner $ \a b fail -> jumpIfGt a b fail compileGen (PrimCall PrimOpNeExpr [p, q] _) maybeReg inner = mdo case cmpWordPat vars p of Just cmp -> local $ \q' -> compileTermGen q vars (Just q') $ cmp (castRegister q') ok Nothing -> withTerm vars q $ \q' -> cmpOutputPat syscalls q' (preProcessPat $ inlineVars vars p) ok jump fail ok <- label whenJust maybeReg (resetOutput . castRegister) r <- inner fail <- label return r compileGen (PrimCall _prim _args _) Nothing inner = inner compileGen (PrimCall PrimOpAddNat [p, q] _) (Just reg) inner = withNatTerm vars p $ \a -> withNatTerm vars q $ \b -> do move a reg add b reg inner compileGen (PrimCall PrimOpToLower [arg] _) (Just reg) inner = withTerm vars arg $ \str -> do local $ \ptr end -> do getOutput str ptr end resetOutput (castRegister reg) outputStringToLower ptr end (castRegister reg) inner compileGen (PrimCall PrimOpLength [arg] _) (Just reg) inner = withTerm vars arg $ \array -> do local $ \ptr end -> do getOutput array ptr end inputNat ptr end reg inner compileGen (PrimCall PrimOpSize [arg] _) (Just reg) inner = withTerm vars arg $ \set -> do local $ \ptr end -> do getOutput set ptr end inputNat ptr end reg inner compileGen (PrimCall PrimOpZip [arg1,arg2] (Angle.ArrayTy (Angle.RecordTy fields))) (Just reg) inner | Just (Angle.FieldDef _ ty1) <- find (("tuplefield0" ==) . Angle.fieldDefName) fields , Just (Angle.FieldDef _ ty2) <- find (("tuplefield1" ==) . Angle.fieldDefName) fields = withTerm vars arg1 $ \array1 -> withTerm vars arg2 $ \array2 -> do local $ \ptr1 ptr2 end1 end2 len1 len2 len -> mdo getOutput array1 ptr1 end1 getOutput array2 ptr2 end2 inputNat ptr1 end1 len1 inputNat ptr2 end2 len2 -- Take the mininum of the two lengths jumpIfGe len1 len2 minLabel move len1 len jump cont minLabel <- label move len2 len cont <- label resetOutput (castRegister reg) outputNat len (castRegister reg) jumpIf0 len done loop <- label local $ \start1 start2 -> do move ptr1 start1 skipTrusted ptr1 end1 ty1 outputBytes start1 ptr1 (castRegister reg) move ptr2 start2 skipTrusted ptr2 end2 ty2 outputBytes start2 ptr2 (castRegister reg) jumpIfEq ptr1 end1 done jumpIfEq ptr2 end2 done jump loop done <- label return () inner compileGen (PrimCall PrimOpConcat [arg1,arg2] (Angle.ArrayTy ty)) (Just reg) inner = withTerm vars arg1 $ \array1 -> withTerm vars arg2 $ \array2 -> do local $ \ptr1 ptr2 end1 end2 -> mdo getOutput array1 ptr1 end1 getOutput array2 ptr2 end2 local $ \len1 len2 len -> do inputNat ptr1 end1 len1 inputNat ptr2 end2 len2 move len1 len add len2 len resetOutput (castRegister reg) outputNat len (castRegister reg) loop1 <- label concatLoop ptr1 end1 loop1 loop2 (castRegister reg) ty loop2 <- label concatLoop ptr2 end2 loop2 done (castRegister reg) ty done <- label return () inner where concatLoop arrPtr arrEnd loopLabel doneLabel out ty = do jumpIfEq arrPtr arrEnd doneLabel local $ \localPtr -> do move arrPtr localPtr skipTrusted arrPtr arrEnd ty outputBytes localPtr arrPtr out jump loopLabel compileGen (PrimCall PrimOpRelToAbsByteSpans [arg] _) (Just reg) inner = withTerm vars arg $ \array -> do local $ \ptr end -> do getOutput array ptr end resetOutput (castRegister reg) outputRelToAbsByteSpans ptr end (castRegister reg) inner compileGen (PrimCall PrimOpReverse [arg] _) (Just reg) inner = withTerm vars arg $ \str -> do local $ \ptr end -> do getOutput str ptr end resetOutput (castRegister reg) outputStringReverse ptr end (castRegister reg) inner compileGen (PrimCall PrimOpUnpackByteSpans [arg] _) (Just reg) inner = withTerm vars arg $ \array -> do local $ \ptr end -> do getOutput array ptr end resetOutput (castRegister reg) outputUnpackByteSpans ptr end (castRegister reg) inner compileGen PrimCall{} _ _ = error "compileGen: unknown PrimCall" compileGen (ArrayElementGenerator _ _) Nothing inner = inner compileGen (ArrayElementGenerator eltTy term) (Just reg) inner = withTerm vars term $ \array -> do local $ \len off -> mdo local $ \ptr end -> do getOutput array ptr end ptrDiff ptr end len local $ \start -> do move ptr start -- the number of elements is useless to us, since we need -- to traverse the array linearly. We'll use the length -- to know when we've reached the end. inputSkipNat ptr end ptrDiff start ptr off jumpIfEq off len done loop <- label -- Note that ptr/len are not valid here, because we might -- have suspended and resumed during the previous -- iteration of the loop, so we must getOutput again. -- We're maintaining the current offset into the array -- value in the 'off' register. local $ \ptr end -> do getOutput array ptr end ptrDiff ptr end len add off ptr local $ \start size -> do -- really want to just matchPat here move ptr start if | isByteTy eltTy -> inputByte ptr end reg | isWordTy eltTy -> inputNat ptr end reg | otherwise -> do skipTrusted ptr end eltTy resetOutput (castRegister reg) outputBytes start ptr (castRegister reg) ptrDiff start ptr size add size off a <- inner jumpIfLt off len loop done <- label return a compileGen (SetElementGenerator _ _) Nothing inner = inner compileGen (SetElementGenerator eltTy term) r@(Just _reg) inner = compileGen (ArrayElementGenerator eltTy term) r inner -- derived fact where we don't bind the Fid: no need to store it compileGen DerivedFactGenerator{} Nothing inner = inner -- derived fact where we bind the Fid: generate the key and -- store the fact in the environment. compileGen (DerivedFactGenerator (PidRef pid _) key val) (Just resultReg) inner = do rpid <- constant (fromIntegral (fromPid pid)) let isEmpty (Ref (MatchVar (Var ty _ _))) = isEmptyTy ty isEmpty (Tuple []) = True isEmpty _ = False local $ \size -> do if | isEmpty val -> withTerm vars key $ \out -> do getOutputSize out size newDerivedFact rpid out size resultReg | isEmpty key -> withTerm vars val $ \out -> do getOutputSize out size newDerivedFact rpid out size resultReg | otherwise -> output $ \out -> do buildTerm out vars key getOutputSize out size buildTerm out vars val newDerivedFact rpid out size resultReg inner compileGen (FactGenerator (PidRef pid _) kpat vpat range) maybeReg inner = do let mtrans = lookupTransformation pid qtrans compileFactGenerator mtrans bounds regs vars pid kpat vpat range maybeReg inner -- Perform an action but interrupt it and clean-up after the first result. singleResult :: (forall a. Code a -> Code a) -> Code b -> Code b singleResult action continue = local $ \seekLevel -> mdo currentSeek seekLevel action $ mdo endSeek seekLevel jump success jump fail success <- label a <- continue fail <- label return a -- | Match term against the default value for its type matchDef :: QueryRegs -> Label -> Type -> Term (Match TransformAndBind Output) -> Code () matchDef syscalls fail ty pat = output $ \out -> do buildTerm out mempty (defaultValue ty) local $ \start end -> do getOutput out start end matchPat syscalls (Bytes start end) fail (preProcessPat pat) compileFactGenerator :: forall a . Maybe PredicateTransformation -> Boundaries -> QueryRegs -> Vector (Register 'Word) -- ^ registers for variables -> Pid -> Pat -> Pat -> SeekSection -> Maybe (Register 'Word) -> Code a -> Code a compileFactGenerator mtrans bounds qregs@QueryRegs{..} vars pid kpat vpat section maybeReg inner = mdo let etrans = maybe (Left pid) Right mtrans withPatterns qregs etrans vars kpat vpat $ \availablePid isPointQuery prefix matchKey matchValue -> do typ <- constant $ fromIntegral $ fromPid availablePid local $ \seekTok -> mdo let Bytes pstart pend = prefix seek' typ pstart pend seekTok -- for each fact... loop <- label local $ \clause keyend clauseend -> mdo let need_value = isJust matchValue local $ \ignore ok -> do next seekTok need_value ok clause keyend clauseend (fromMaybe ignore maybeReg) select ok [ end -- 0 -> no match , continue -- 1 -> match ] suspend saveState loop -- 2 -> timeout / interrupted continue <- label return () -- check that the key matches the pattern matchKey (Bytes clause keyend) loop -- match the value forM_ matchValue $ \match -> match (Bytes keyend clauseend) loop a <- inner -- loop to find more facts, unless this is a point query which could -- only have a single result. unless isPointQuery $ jump loop end <- label endSeek seekTok return a where seek' typ ptr end tok = case (section, bounds) of (SeekOnAllFacts, _) -> seek typ ptr end tok (SeekOnBase, StackedBoundaries (SectionBoundaries from to) _) -> seekBetween from to (SeekOnStacked, StackedBoundaries _ (SectionBoundaries from to)) -> seekBetween from to _ -> error "unexpected section seek on non-stacked db" where seekBetween from to = do pfrom <- constant $ fromIntegral $ fromFid from pto <- constant $ fromIntegral $ fromFid to seekWithinSection typ ptr end pfrom pto tok -- ^ Extract a prefix to be searched and create code to match on the key and -- value of facts searched. -- -- If the transformation determines that the modified pattern will never match -- the callback code will be skipped at runtime. withPatterns :: QueryRegs -> Either Pid PredicateTransformation -> Vector (Register 'Word) -- ^ registers for variables -> Pat -- ^ key pattern -> Pat -- ^ value pattern -> ( Pid -- predicate to search for -> IsPointQuery -> Bytes -- prefix -> (Bytes -> Label -> Code ()) -- match key -> Maybe (Bytes -> Label -> Code ()) -- match value -> Code a ) -> Code a withPatterns syscalls etrans vars kpat vpat act = mdo a <- transKeyPat (matchDef syscalls fail) (inlineVars vars kpat) $ \kpat' -> do transValPat (matchDef syscalls fail) (inlineVars vars vpat) $ \vpat' -> do let kchunks = preProcessPat kpat' vchunks = preProcessPat vpat' when (emptyPrefix kchunks) (fullScan pid) withPrefix kchunks $ \isPointQuery prefix remaining -> do let matchKey bytes fail = matchPat syscalls bytes fail remaining matchVal bytes fail = matchPat syscalls bytes fail vchunks needs_value = not (all isWild vchunks) act pid isPointQuery prefix matchKey (if needs_value then Just matchVal else Nothing) fail <- label return a where (pid, transKeyPat, transValPat) = case etrans of Right PredicateTransformation{..} -> let PidRef pid _ = tAvailable in ( pid , fromMaybe noTrans (transformKeyPattern <*> Just syscalls) , fromMaybe noTrans (transformValuePattern <*> Just syscalls) ) Left pid -> ( pid , noTrans , noTrans ) noTrans _ bytes f = f bytes -- Extract a query prefix and a pattern which should match the entire -- structure withPrefix :: [QueryChunk Output] -> ( IsPointQuery -> Bytes -- prefix bytes -> [QueryChunk Output] -- match against results from the beginning -> Code a) -> Code a withPrefix chunks fun = case chunks of -- Special case for a QueryBind that covers the whole -- pattern. This is used to capture the key of a fact so -- that we can return it, but we don't want it to interfere -- with prefix lookup. [QueryAnd left right] | emptyPrefix left && not (emptyPrefix right) -> withPrefix [QueryAnd right left] fun | not $ emptyPrefix left -> withPrefix left $ \isPointQuery bytes filters -> fun isPointQuery bytes $ if null filters then right else [QueryAnd filters right] -- Just a fixed ByteString: use it directly (QueryPrefix bs : rest) | emptyPrefix rest -> local $ \ptr end -> do loadLiteral bs ptr end wildPrefixFor rest (Bytes ptr end) fun -- A variable: use it directly (QueryVar (Typed ty out) : rest) | emptyPrefix rest, not (isWordTy ty) -> local $ \ptr end -> do getOutput out ptr end wildPrefixFor rest (Bytes ptr end) fun -- Otherwise: build up the prefix in a binary::Output -- (register 'prefixOut') _otherwise -> buildPrefix chunks $ \prefixOut chunks' -> local $ \ptr end -> do getOutput prefixOut ptr end wildPrefixFor chunks' (Bytes ptr end) fun where wildPrefixFor :: [QueryChunk Output] -- ^ non-prefix filters -> Bytes -- ^ prefix bytes -> ( Bool -> Bytes -> [QueryChunk Output] -> Code a) -- ^ send prefix-adjusted filters -> Code a wildPrefixFor chunks prefix@(Bytes ptr end) fun = if null chunks then fun True prefix [] else local $ \prefix_size -> do ptrDiff ptr end prefix_size fun False prefix (QueryPrefixWild prefix_size : chunks) -- ---------------------------------------------------------------------------- -- | A pre-processed pattern, ready for compilation to bytecode data QueryChunk var = -- | A literal sequence of bytes to match, corresponding to a -- fragment of the serialized pattern QueryPrefix ByteString -- | a sequence of bytes to skip over | QueryPrefixWild (Register 'Word) -- | An array prefix of length N | QueryArrayPrefix Word64 -- ^ Length Type -- ^ Element type [QueryChunk var] -- ^ chunks for the prefix -- | A wildcard, represented as the type of the fragment to skip over | QueryWild Type -- | Never matches | QueryNever -- | A binder: like a wildcard, but we capture the fragment and bind -- it to a variable | QueryBind var -- | Transform a fragment into a different type and bind it to a variable | QueryTransformAndBind Type var -- | A variable: match this fragment against the specified variable | QueryVar var -- | Match two patterns simultaneously. This is the preprocessed -- equivalent of 'MatchAnd'. | QueryAnd [QueryChunk var] [QueryChunk var] -- | A sum type, with different matchers for each of the alternatives. -- If an alternative is Nothing, then the pattern does not match -- that alternative. | QuerySum [Maybe [QueryChunk var]] deriving Show type M a = StateT (Builder, Bool, [QueryChunk Output]) IO a -- @Bool@ is whether the builder is empty (True for empty, False for non-empty) instance IsWild (QueryChunk var) where isWild QueryWild{} = True isWild QueryPrefixWild{} = True isWild _ = False inlineVars :: Vector (Register 'Word) -> Term (Match () Var) -> Term (Match TransformAndBind Output) inlineVars vars term = fmap (bimap (error "inlineVars") toOutput) term where toOutput :: Var -> Output toOutput (Var ty v _) = Typed ty (castRegister (vars ! v)) -- -- | Process a query into a [QueryChunk], which enables the compiler to -- generate efficient code for a pattern match. -- preProcessPat :: Term (Match TransformAndBind Output) -> [QueryChunk Output] preProcessPat pat = unsafePerformIO $ withBuilder $ \builder -> do (_, (_, _, chunks)) <- flip runStateT (builder, True, []) $ build pat >> endOfChunk return (reverse chunks) where builder :: M Builder builder = do (b, empty, chunks) <- get if empty then do put (b, False, chunks) return b else return b endOfChunk :: M () endOfChunk= do (b,empty,chunks) <- get if empty then return () else do bs <- lift $ finishBuilder b lift $ resetBuilder b put (b, True, QueryPrefix bs : chunks) getChunks :: M () -> M [QueryChunk Output] getChunks m = do (b, empty, saveChunks) <- get put (b, empty, []) m endOfChunk (b, empty, chunks) <- get put (b, empty, saveChunks) return (reverse chunks) chunk c = do endOfChunk modify $ \(b, empty, chunks) -> (b, empty, c : chunks) prefixString s = do b <- builder lift $ FFI.unsafeWithBytes s $ \p n -> FFI.call $ glean_push_value_bytes b (castPtr p) n -- NB. Use glean_push_value_bytes to omit the NUL NUL suffix. -- Thus the remaining input is a valid string, and we can continue -- by matching it as a StringTy with QueryWild or QueryBind. build :: Term (Match TransformAndBind Output) -> M () build pat = case pat of Array terms -> do b <- builder lift $ FFI.call $ glean_push_value_array b $ fromIntegral $ length terms mapM_ build terms Tuple terms -> mapM_ build terms (Alt ix term) -> do b <- builder lift $ FFI.call $ glean_push_value_selector b $ fromIntegral ix build term Ref (MatchVar out) -> chunk $ QueryVar out Ref (MatchBind out) -> chunk $ QueryBind out Ref (MatchWild ty) -> chunk (QueryWild ty) Ref (MatchNever _) -> chunk QueryNever Ref (MatchFid fid) -> do b <- builder lift $ FFI.call $ glean_push_value_fact b fid -- optimise MatchAnd where one side is a wildcard: Ref (MatchAnd (Ref MatchWild{}) b) -> build b Ref (MatchAnd a (Ref MatchWild{})) -> build a Ref (MatchAnd a b) -> do endOfChunk chunk =<< QueryAnd <$> getChunks (build a) <*> getChunks (build b) Ref (MatchPrefix txt rest) -> do prefixString txt build rest Ref (MatchArrayPrefix ty prefix all) -> do endOfChunk let arr = do chunks <- getChunks $ mapM_ build prefix chunk $ QueryArrayPrefix (genericLength chunks) ty chunks chunk =<< QueryAnd <$> getChunks arr <*> getChunks (build all) Ref (MatchExt (TransformAndBind src dst)) -> chunk $ QueryTransformAndBind src dst Byte w -> do b <- builder lift $ FFI.call $ glean_push_value_byte b w Nat w -> do b <- builder lift $ FFI.call $ glean_push_value_nat b w String s -> do b <- builder lift $ FFI.unsafeWithBytes s $ \p n -> FFI.call $ glean_push_value_string b (castPtr p) n ByteArray bs -> do b <- builder lift $ encodeByteArray b bs -- | True if the prefix of this query is empty emptyPrefix :: [QueryChunk var] -> Bool emptyPrefix (QueryPrefix{} : _) = False emptyPrefix (QueryVar{} : _) = False emptyPrefix [QueryAnd [QueryBind{}] x] = emptyPrefix x emptyPrefix [QueryAnd [QueryTransformAndBind{}] x] = emptyPrefix x emptyPrefix _ = True -- | Serialize the prefix of a query into an output register, and return -- the remaining non-prefix part of the query. buildPrefix :: [QueryChunk Output] -> (Register 'BinaryOutputPtr -> [QueryChunk Output] -> Code a) -> Code a buildPrefix chunks cont = output $ \out -> do go out cont chunks where go out cont [] = cont out [] go out cont (chunk:rest) = case chunk of QueryPrefix bs -> do local $ \ptr end -> do loadLiteral bs ptr end outputBytes ptr end out go out cont rest QueryVar (Typed ty o) | isByteTy ty -> do outputByte (castRegister o) out go out cont rest QueryVar (Typed ty o) | isWordTy ty -> do outputNat (castRegister o) out go out cont rest -- every other type is currently represented as a binary::Output QueryVar (Typed _ o) -> do local $ \ptr end -> do getOutput o ptr end outputBytes ptr end out go out cont rest _ -> cont out (chunk:rest) -- We might want to consider handling QuerySum here: if an -- alt can be serialized to a prefix, it might be better -- to seek to it directly. But then we would possibly have -- to do multiple seeks. -- | Load a term that is expected to be word-typed into a register. withNatTerm :: Vector (Register 'Word) -- ^ registers for variables -> Expr -> (Register 'Word -> Code a) -> Code a withNatTerm vars term andThen = do case term of Nat n -> local $ \r -> do loadConst (fromIntegral n) r; andThen r Byte n -> local $ \r -> do loadConst (fromIntegral n) r; andThen r Ref (MatchFid f) -> local $ \r -> do loadConst (fromIntegral (fromFid f)) r; andThen r Ref (MatchVar (Var ty v _)) | isWordTy ty -> andThen (vars ! v) _other -> error "withNatTerm: shouldn't happen" -- | Serialize a term into an output register. A copy is avoided if -- the term is already represented by an output register. withTerm :: Vector (Register 'Word) -> Term (Match () Var) -> (Register 'BinaryOutputPtr -> Code a) -> Code a withTerm vars (Ref (MatchVar (Var ty v _))) action | not (isWordTy ty) = action (castRegister (vars ! v)) withTerm vars term action = do output $ \reg -> do buildTerm reg vars term action reg -- | Execute a piece of code repeatedly for as long as it keeps producing -- new facts. -- -- The first argument is executed once first. -- If it defines new facts then the second argument is executed repeatedly -- for as long as it keeps adding facts to the Define. -- recursive :: QueryRegs -> (forall a. Code a -> Code a) -- ^ code for first run -> (forall a. Code a -> Code a) -- ^ code to evaluate repeatedly -> Code b -- ^ code to insert after -> Code b recursive QueryRegs{..} before after andThen = local $ \innerRet startId deltaId -> mdo firstFreeId startId siteBefore <- before $ mdo site <- callSite loadLabel ret_ innerRet jump doInner ret_ <- label return site firstFreeId deltaId -- skip to end if there were no new facts produced local $ \difference -> mdo move deltaId difference sub startId difference jumpIf0 difference done recurse <- label move deltaId startId siteAfter <- after $ mdo site <- callSite loadLabel ret_ innerRet jump doInner ret_ <- label return site firstFreeId deltaId -- execute 'after' again if there are new facts local $ \difference -> mdo move deltaId difference sub startId difference jumpIfNot0 difference recurse jump done doInner <- label a <- calledFrom [siteBefore, siteAfter] andThen jumpReg innerRet done <- label return a -- | check that a value matches a pattern, and bind variables as -- necessary. The pattern is assumed to cover the *whole* of the -- input. matchPat :: QueryRegs -> Bytes -> Label -> [QueryChunk Output] -> Code () matchPat syscalls (Bytes input inputend) fail chunks = match True chunks where match :: Bool -- ^ whether the query chunks match until the end of the input. -> [QueryChunk Output] -> Code () match _ [] = return () match tillEnd (x:rest) = let isLastPattern = tillEnd && null rest in case x of QueryPrefix bs -> local $ \ok -> do inputShiftLit input inputend bs ok jumpIf0 ok fail -- chunk didn't match match tillEnd rest QueryPrefixWild prefix_size -> do add prefix_size input match tillEnd rest QueryWild ty -- skip trailing wildcards | tillEnd && all isWild rest -> return () | otherwise -> do skipTrusted input inputend ty match tillEnd rest QueryNever -> jump fail QueryVar (Typed ty out) | isEmptyTy ty -> match tillEnd rest | isWordTy ty -> do local $ \id -> do inputNat input inputend id jumpIfNe id (castRegister out) fail match tillEnd rest -- the empty tuple could be represented by a null pointer, so it's -- not safe to do inputShiftBytes anyway. | otherwise -> do local $ \ptr end ok -> do getOutput out ptr end inputShiftBytes input inputend ptr end ok jumpIf0 ok fail match tillEnd rest QueryAnd a b | all isWild a -> match tillEnd (b ++ rest) | otherwise -> do local $ \start -> do move input start match isLastPattern a move start input match isLastPattern b match tillEnd rest QuerySum alts -> mdo local $ \sel -> do inputNat input inputend sel -- an unknown alternative always fails to match. let unknown = [fail] select sel (lbls ++ unknown) raise "selector out of range" lbls <- forM alts $ \mb -> do case mb of Nothing -> return fail Just chunks -> do lbl <- label match isLastPattern chunks jump end return lbl end <- label match tillEnd rest QueryBind (Typed ty out) | isWordTy ty -> do inputNat input inputend (castRegister out) match tillEnd rest | otherwise -> do let outReg = castRegister out resetOutput outReg if isLastPattern -- we don't need to traverse the data, just copy the bytes. then outputBytes input inputend outReg else local $ \start -> do move input start skipTrusted input inputend ty outputBytes start input outReg match tillEnd rest QueryTransformAndBind from (Typed to out) -> case transformBytes syscalls from to of Nothing -> match tillEnd (QueryBind (Typed to out) : rest) Just transform -> do let bytes = Bytes input inputend resetOutput out transform bytes out match tillEnd rest QueryArrayPrefix npatterns ty patterns -> do local $ \size patternsLen -> do loadConst npatterns patternsLen inputNat input inputend size jumpIfLt size patternsLen fail match isLastPattern patterns -- skip to the end unless we are done matching this branch, in which -- case we don't need to leave the 'input' pointer in the right -- place. unless isLastPattern $ mdo sub patternsLen size jumpIf0 size done skip <- label skipTrusted input inputend ty decrAndJumpIfNot0 size skip done <- label match tillEnd rest ----------------------------------------------------------------------------- -- | Compile a query for some facts, possibly with recursive expansion. compileQueryFacts :: [FactQuery] -> IO CompiledQuery compileQueryFacts facts = do input <- withBuilder $ \builder -> do buildRtsValue builder [ (fromIntegral factQuery_id :: Word64, if factQuery_recursive then 1 else 0 :: Word64) | FactQuery{..} <- facts ] finishBuilder builder (Meta{..}, sub) <- generateQueryCode $ \ QueryRegs{..} -> outputUninitialized $ \kout vout -> local $ \fid pid rec_ ptr end -> do loadLiteral input ptr end local $ inputNat ptr end -- ignore the size loop <- label inputNat ptr end fid inputNat ptr end rec_ lookupKeyValue fid kout vout pid resultWithPid fid kout vout pid rec_ jumpIfLt ptr end loop ret return (CompiledQuery sub Nothing Nothing meta_fullScans) -- ----------------------------------------------------------------------------- -- The FFI layer for query bytecode subroutines -- -- IF YOU BREAK BACKWARD COMPATIBILITY HERE, BUMP version IN -- Glean.Bytecode.Generate.Instruction -- -- IF YOU ALSO BREAK FORWARD COMPATIBILITY, BUMP latestSupportedVersion AS WELL -- generateQueryCode :: (QueryRegs -> Code ()) -> IO (Meta, Subroutine CompiledQuery) generateQueryCode f = generate Optimised $ \ seek_ seekWithinSection_ currentSeek_ endSeek_ next_ lookupKey_ result_ resultWithPid_ newDerivedFact_ firstFreeId_ newSet_ insertOutputSet_ setToArray_ freeSet_ newWordSet_ insertWordSet_ wordSetToArray_ byteSetToByteArray_ freeWordSet_ saveState maxResults maxBytes -> let seek typ ptr end tok = callFun_3_1 seek_ typ (castRegister ptr) (castRegister end) tok seekWithinSection typ ptr end pfrom pto tok = callFun_5_1 seekWithinSection_ typ (castRegister ptr) (castRegister end) (castRegister pfrom) (castRegister pto) tok currentSeek tok = callFun_0_1 currentSeek_ tok endSeek tok = callFun_1_0 endSeek_ tok next tok need_value ok clause keyend clauseend id = do demand <- constant $ if need_value then 1 else 0 callFun_2_5 next_ tok demand ok (castRegister clause) (castRegister keyend) (castRegister clauseend) id lookupKeyValue id kout vout pid = callFun_3_1 lookupKey_ id (castRegister kout) (castRegister vout) pid result id key val new = callFun_3_1 result_ id (castRegister key) (castRegister val) new resultWithPid id key val pid recexp = callFun_5_0 resultWithPid_ id (castRegister key) (castRegister val) pid recexp newDerivedFact ty key val id = callFun_3_1 newDerivedFact_ ty (castRegister key) (castRegister val) id firstFreeId fid = callFun_0_1 firstFreeId_ fid newSet setToken = callFun_0_1 newSet_ setToken insertOutputSet setToken outputPtr = callFun_2_0 insertOutputSet_ setToken (castRegister outputPtr) setToArray setToken outputPtr = callFun_2_0 setToArray_ setToken (castRegister outputPtr) freeSet setToken = callFun_1_0 freeSet_ setToken newWordSet setToken = callFun_0_1 newWordSet_ setToken insertWordSet setToken outputPtr = callFun_1_1 insertWordSet_ setToken (castRegister outputPtr) wordSetToArray setToken outputPtr = callFun_1_1 wordSetToArray_ setToken (castRegister outputPtr) byteSetToByteArray setToken outputPtr = callFun_1_1 byteSetToByteArray_ setToken (castRegister outputPtr) freeWordSet setToken = callFun_1_0 freeWordSet_ setToken in f QueryRegs{..}