diff --git a/booster/library/Booster/JsonRpc.hs b/booster/library/Booster/JsonRpc.hs
index 11a0399f2c..3620a12bf0 100644
--- a/booster/library/Booster/JsonRpc.hs
+++ b/booster/library/Booster/JsonRpc.hs
@@ -155,33 +155,51 @@ respond stateVar request =
solver <- maybe (SMT.noSolver) (SMT.initSolver def) mSMTOptions
- logger <- getLogger
- prettyModifiers <- getPrettyModifiers
- let rewriteConfig =
- RewriteConfig
- { definition = def
- , llvmApi = mLlvmLibrary
- , smtSolver = solver
- , varsToAvoid = substVars
- , doTracing
- , logger
- , prettyModifiers
- , mbMaxDepth = mbDepth
- , mbSimplify = rewriteOpts.interimSimplification
- , cutLabels = cutPoints
- , terminalLabels = terminals
- }
- result <-
- performRewrite rewriteConfig substPat
- SMT.finaliseSolver solver
- stop <- liftIO $ getTime Monotonic
- let duration =
- if fromMaybe False req.logTiming
- then
- Just $
- fromIntegral (toNanoSecs (diffTimeSpec stop start)) / 1e9
- else Nothing
- pure $ execResponse duration req result substitution unsupported
+ -- check input pattern's consistency before starting rewriting
+ evaluatedInitialPattern <-
+ ApplyEquations.evaluatePattern
+ def
+ mLlvmLibrary
+ solver
+ mempty
+ substPat
+
+ case evaluatedInitialPattern of
+ (Left ApplyEquations.SideConditionFalse{}, _) -> do
+ -- input pattern's constraints are Bottom, return Vacuous
+ stop <- liftIO $ getTime Monotonic
+ pure $
+ execResponse
+ (duration req.logTiming start stop)
+ req
+ (0, mempty, RewriteTrivial substPat)
+ substitution
+ unsupported
+ (Left other, _) ->
+ pure . Left . RpcError.backendError $ RpcError.Aborted (Text.pack . constructorName $ other)
+ (Right newPattern, simplifierCache) -> do
+ logger <- getLogger
+ prettyModifiers <- getPrettyModifiers
+ let rewriteConfig =
+ RewriteConfig
+ { definition = def
+ , llvmApi = mLlvmLibrary
+ , smtSolver = solver
+ , varsToAvoid = substVars
+ , doTracing
+ , logger
+ , prettyModifiers
+ , mbMaxDepth = mbDepth
+ , mbSimplify = rewriteOpts.interimSimplification
+ , cutLabels = cutPoints
+ , terminalLabels = terminals
+ }
+
+ result <-
+ performRewrite rewriteConfig simplifierCache newPattern
+ SMT.finaliseSolver solver
+ stop <- liftIO $ getTime Monotonic
+ pure $ execResponse (duration req.logTiming start stop) req result substitution unsupported
RpcTypes.AddModule RpcTypes.AddModuleRequest{_module, nameAsId = nameAsId'} -> Booster.Log.withContext CtxAddModule $ runExceptT $ do
-- block other request executions while modifying the server state
state <- liftIO $ takeMVar stateVar
@@ -247,9 +265,9 @@ respond stateVar request =
start <- liftIO $ getTime Monotonic
let internalised =
runExcept $ internaliseTermOrPredicate DisallowAlias CheckSubsorts Nothing def req.state.term
- let mkTraces duration
+ let mkTraces durationLog
| Just True <- req.logTiming =
- Just [ProcessingTime (Just Booster) duration]
+ Just [ProcessingTime (Just Booster) durationLog]
| otherwise =
Nothing
@@ -327,11 +345,11 @@ respond stateVar request =
SMT.finaliseSolver solver
stop <- liftIO $ getTime Monotonic
- let duration =
+ let durationLog =
fromIntegral (toNanoSecs (diffTimeSpec stop start)) / 1e9
mkSimplifyResponse state =
RpcTypes.Simplify
- RpcTypes.SimplifyResult{state, logs = mkTraces duration}
+ RpcTypes.SimplifyResult{state, logs = mkTraces durationLog}
pure $ second mkSimplifyResponse result
RpcTypes.GetModel req -> withModule req._module $ \case
(_, _, Nothing, _) -> do
@@ -567,6 +585,13 @@ respond stateVar request =
, logs = Nothing
}
+ duration mLogTiming start stop =
+ if fromMaybe False mLogTiming
+ then
+ Just $
+ fromIntegral (toNanoSecs (diffTimeSpec stop start)) / 1e9
+ else Nothing
+
handleSmtError :: JsonRpcHandler
handleSmtError = JsonRpcHandler $ \case
SMT.GeneralSMTError err -> runtimeError "problem" err
diff --git a/booster/library/Booster/Pattern/ApplyEquations.hs b/booster/library/Booster/Pattern/ApplyEquations.hs
index ca5fee67b1..a2da5ce771 100644
--- a/booster/library/Booster/Pattern/ApplyEquations.hs
+++ b/booster/library/Booster/Pattern/ApplyEquations.hs
@@ -439,7 +439,11 @@ evaluateTerm' ::
evaluateTerm' direction = iterateEquations direction PreferFunctions
{- | Simplify a Pattern, processing its constraints independently.
- Returns either the first failure or the new pattern if no failure was encountered
+
+ Before evaluating the term of the pattern,
+ the constraints of the pattern are checked for consistency with an SMT solver.
+
+ Returns either the first failure or the new pattern if no failure was encountered.
-}
evaluatePattern ::
LoggerMIO io =>
@@ -457,10 +461,36 @@ evaluatePattern' ::
LoggerMIO io =>
Pattern ->
EquationT io Pattern
-evaluatePattern' pat@Pattern{term, ceilConditions} = withPatternContext pat $ do
+evaluatePattern' pat@Pattern{term, constraints, ceilConditions} = withPatternContext pat $ do
+ solver <- (.smtSolver) <$> getConfig
+ -- check the pattern's constraints for satisfiability to ensure they are consistent
+ consistent <-
+ withContext CtxConstraint $ do
+ withContext CtxDetail . withTermContext (coerce $ collapseAndBools constraints) $ pure ()
+ consistent <- SMT.isSat solver constraints
+ withContext CtxConstraint $
+ logMessage $
+ "Constraints consistency check returns: " <> show consistent
+ pure consistent
+ case consistent of
+ Right False -> do
+ -- the constraints are unsatisfiable, which means that the patten is Bottom
+ throw . SideConditionFalse . collapseAndBools $ constraints
+ Left SMT.SMTSolverUnknown{} -> do
+ -- unlikely case of an Unknown response to a consistency check.
+ -- continue to preserver the old behaviour.
+ withContext CtxConstraint . logWarn . Text.pack $
+ "Constraints consistency UNKNOWN: " <> show consistent
+ pure ()
+ Left other ->
+ -- fail hard on SMT error other than @SMT.SMTSolverUnknown@
+ liftIO $ Exception.throw other
+ Right True ->
+ -- constraints are consistent, continue
+ pure ()
+
newTerm <- withTermContext term $ evaluateTerm' BottomUp term `catch_` keepTopLevelResults
- -- after evaluating the term, evaluate all (existing and
- -- newly-acquired) constraints, once
+ -- after evaluating the term, evaluate all (existing and newly-acquired) constraints, once
traverse_ simplifyAssumedPredicate . predicates =<< getState
-- this may yield additional new constraints, left unevaluated
evaluatedConstraints <- predicates <$> getState
@@ -477,6 +507,9 @@ evaluatePattern' pat@Pattern{term, ceilConditions} = withPatternContext pat $ do
pure partialResult
err -> throw err
+ collapseAndBools :: Set Predicate -> Predicate
+ collapseAndBools = coerce . foldAndBool . map coerce . Set.toList
+
-- evaluate the given predicate assuming all others
simplifyAssumedPredicate :: LoggerMIO io => Predicate -> EquationT io ()
simplifyAssumedPredicate p = do
diff --git a/booster/library/Booster/Pattern/Rewrite.hs b/booster/library/Booster/Pattern/Rewrite.hs
index 8e30383f97..c162bd7676 100644
--- a/booster/library/Booster/Pattern/Rewrite.hs
+++ b/booster/library/Booster/Pattern/Rewrite.hs
@@ -704,9 +704,10 @@ performRewrite ::
forall io.
LoggerMIO io =>
RewriteConfig ->
+ SimplifierCache ->
Pattern ->
io (Natural, Seq (RewriteTrace ()), RewriteResult Pattern)
-performRewrite rewriteConfig pat = do
+performRewrite rewriteConfig initialCache pat = do
(rr, RewriteStepsState{counter, traces}) <-
flip runStateT rewriteStart $ doSteps False pat
pure (counter, traces, rr)
@@ -722,6 +723,14 @@ performRewrite rewriteConfig pat = do
, terminalLabels
} = rewriteConfig
+ rewriteStart :: RewriteStepsState
+ rewriteStart =
+ RewriteStepsState
+ { counter = 0
+ , traces = mempty
+ , simplifierCache = initialCache
+ }
+
logDepth = withContext CtxDepth . logMessage
depthReached n = maybe False (n >=) mbMaxDepth
@@ -919,11 +928,3 @@ data RewriteStepsState = RewriteStepsState
, traces :: !(Seq (RewriteTrace ()))
, simplifierCache :: SimplifierCache
}
-
-rewriteStart :: RewriteStepsState
-rewriteStart =
- RewriteStepsState
- { counter = 0
- , traces = mempty
- , simplifierCache = mempty
- }
diff --git a/booster/library/Booster/SMT/Interface.hs b/booster/library/Booster/SMT/Interface.hs
index d88a8b06f5..a80523c155 100644
--- a/booster/library/Booster/SMT/Interface.hs
+++ b/booster/library/Booster/SMT/Interface.hs
@@ -12,9 +12,11 @@ module Booster.SMT.Interface (
SMTError (..),
initSolver,
noSolver,
+ isNoSolver,
finaliseSolver,
getModelFor,
checkPredicates,
+ isSat,
hardResetSolver,
) where
@@ -33,7 +35,7 @@ import Data.Either.Extra (fromLeft', fromRight')
import Data.IORef
import Data.Map (Map)
import Data.Map qualified as Map
-import Data.Maybe (fromMaybe)
+import Data.Maybe (fromMaybe, isNothing)
import Data.Set (Set)
import Data.Set qualified as Set
import Data.Text as Text (Text, pack, unlines, unwords)
@@ -120,6 +122,10 @@ noSolver = do
, options = defaultSMTOptions{retryLimit = Just 0}
}
+-- | Detect of the @SMTContext@ does not have a solver
+isNoSolver :: SMT.SMTContext -> Bool
+isNoSolver SMTContext{mbSolver} = isNothing mbSolver
+
-- | Hot-swap @SMTOptions@ in the active @SMTContext@, update the query timeout
swapSmtOptions :: forall io. Log.LoggerMIO io => SMTOptions -> SMT io ()
swapSmtOptions smtOptions = do
@@ -128,6 +134,10 @@ swapSmtOptions smtOptions = do
SMT $ put ctxt{options = smtOptions}
runCmd_ $ SetTimeout smtOptions.timeout
+-- | This function defines the strategy to increment the timeout when retrying a solver query
+updateOptionsOnRetry :: SMTOptions -> SMTOptions
+updateOptionsOnRetry opts = opts{timeout = 2 * opts.timeout, retryLimit = ((-) 1) <$> opts.retryLimit}
+
-- | Stop the solver, initialise a new one and put in the @SMTContext@
hardResetSolver :: forall io. Log.LoggerMIO io => SMTOptions -> SMT io ()
hardResetSolver smtOptions = do
@@ -163,9 +173,13 @@ checkPrelude = do
Sat -> pure ()
other -> do
Log.logMessage $ "Initial SMT definition check returned " <> pack (show other)
- SMT get >>= closeContext
- throwSMT' $
- "Aborting due to potentially-inconsistent SMT setup: Initial check returned " <> show other
+ ctxt <- SMT get
+ if isNoSolver ctxt
+ then -- when running with a dummy solver, ignore the Unknown prelude check
+ pure ()
+ else
+ throwSMT' $
+ "Aborting due to potentially-inconsistent SMT setup: Initial check returned " <> show other
-- | Send the commands from the definition's SMT prelude
runPrelude :: Log.LoggerMIO io => SMT io ()
@@ -178,6 +192,23 @@ finaliseSolver ctxt = do
Log.logMessage ("Closing SMT solver" :: Text)
destroyContext ctxt
+failBecauseUnknown ::
+ forall io.
+ Log.LoggerMIO io =>
+ Set Predicate ->
+ ExceptT SMTError (SMT io) Bool
+failBecauseUnknown psToCheck =
+ smtRun GetReasonUnknown >>= \case
+ Unknown reason -> do
+ Log.withContext Log.CtxAbort $
+ Log.logMessage $
+ "Returned Unknown. Reason: " <> fromMaybe "UNKNOWN" reason
+ throwE $ SMTSolverUnknown reason mempty psToCheck
+ other -> do
+ let msg = "Unexpected result while calling ':reason-unknown': " <> show other
+ Log.withContext Log.CtxAbort $ Log.logMessage $ Text.pack msg
+ throwSMT' msg
+
{- |
Implementation of get-model request
@@ -224,8 +255,7 @@ getModelFor ctxt ps subst
Unknown{} -> do
case opts.retryLimit of
Just x | x > 0 -> do
- let newOpts = opts{timeout = 2 * opts.timeout, retryLimit = Just $ x - 1}
- lift $ hardResetSolver newOpts
+ lift $ hardResetSolver (updateOptionsOnRetry opts)
solve smtAsserts transState
_ -> pure . Left $ response
_ -> pure . Left $ response
@@ -322,9 +352,9 @@ getModelFor ctxt ps subst
mkComment :: Pretty (PrettyWithModifiers '[Decoded] a) => a -> BS.ByteString
mkComment = BS.pack . Pretty.renderDefault . pretty' @'[Decoded]
-{- | Check a predicates, given a set of predicates as known truth.
+{- | Check validity of a predicate, given a set of predicates as known truth.
-Simplest version:
+The set of input predicates @psToCheck@ is interpreted as a conjunction.
Given K as known truth and predicates P to check, check whether K => P
or K => !P, or neither of those implications holds. The check is done
@@ -392,8 +422,8 @@ checkPredicates ctxt givenPs givenSubst psToCheck
pure Nothing
(Sat, Unsat) -> pure . Just $ True
(Unsat, Sat) -> pure . Just $ False
- (Unknown reason, _) -> retry smtGiven sexprsToCheck transState reason
- (_, Unknown reason) -> retry smtGiven sexprsToCheck transState reason
+ (Unknown _, _) -> retry smtGiven sexprsToCheck transState
+ (_, Unknown _) -> retry smtGiven sexprsToCheck transState
other ->
throwE . GeneralSMTError $
("Unexpected result while checking a condition: " :: Text) <> Text.pack (show other)
@@ -402,17 +432,14 @@ checkPredicates ctxt givenPs givenSubst psToCheck
[DeclareCommand] ->
[SExpr] ->
TranslationState ->
- Maybe Text ->
ExceptT SMTError (SMT io) (Maybe Bool)
- retry smtGiven sexprsToCheck transState reasonUnknown = do
+ retry smtGiven sexprsToCheck transState = do
opts <- lift . SMT $ gets (.options)
case opts.retryLimit of
Just x | x > 0 -> do
- let newOpts = opts{timeout = 2 * opts.timeout, retryLimit = Just $ x - 1}
- lift $ hardResetSolver newOpts
+ lift $ hardResetSolver (updateOptionsOnRetry opts)
solve smtGiven sexprsToCheck transState
- _ -> failBecauseUnknown reasonUnknown
-
+ _ -> failBecauseUnknown psToCheck >> pure Nothing -- Nothing is unreachable and is here to make the type checker happy
translated :: Either Text (([DeclareCommand], [SExpr]), TranslationState)
translated = SMT.runTranslator $ do
let mkSMTEquation v t =
@@ -425,13 +452,6 @@ checkPredicates ctxt givenPs givenSubst psToCheck
mapM (SMT.translateTerm . coerce) $ Set.toList psToCheck
pure (smtSubst <> smtPs, toCheck)
- failBecauseUnknown :: Maybe Text -> ExceptT SMTError (SMT io) (Maybe Bool)
- failBecauseUnknown reason = do
- Log.withContext Log.CtxAbort $
- Log.logMessage $
- "Returned Unknown. Reason: " <> fromMaybe "UNKNOWN" reason
- throwE $ SMTSolverUnknown reason givenPs psToCheck
-
-- Given the known truth and the expressions to check,
-- interact with the solver to establish the validity of the expressions.
--
@@ -493,3 +513,55 @@ checkPredicates ctxt givenPs givenSubst psToCheck
"Given ∧ P and Given ∧ !P interpreted as "
<> pack (show (positive', negative'))
pure (positive', negative')
+
+{- | Check satisfiability of a predicate.
+ The set of input predicates @psToCheck@ is interpreted as a conjunction.
+-}
+isSat ::
+ forall io.
+ Log.LoggerMIO io =>
+ SMT.SMTContext ->
+ Set Predicate ->
+ io (Either SMTError Bool)
+isSat ctxt psToCheck
+ | null psToCheck = pure . Right $ True
+ | Left errMsg <- translated = Log.withContext Log.CtxSMT $ do
+ Log.withContext Log.CtxAbort $ Log.logMessage $ "SMT translation error: " <> errMsg
+ pure . Left . SMTTranslationError $ errMsg
+ | Right (smtToCheck, transState) <- translated = Log.withContext Log.CtxSMT $ do
+ evalSMT ctxt . runExceptT $ do
+ lift $ hardResetSolver ctxt.options
+ solve smtToCheck transState
+ where
+ translated :: Either Text ([DeclareCommand], TranslationState)
+ translated =
+ SMT.runTranslator $
+ mapM (\(Predicate p) -> Assert (mkComment p) <$> SMT.translateTerm p) $
+ Set.toList psToCheck
+
+ solve smtToCheck transState = solve'
+ where
+ solve' = do
+ Log.getPrettyModifiers >>= \case
+ ModifiersRep (_ :: FromModifiersT mods => Proxy mods) ->
+ Log.logMessage . Pretty.renderOneLineText $
+ hsep ("Predicates to check:" : map (pretty' @mods) (Set.toList psToCheck))
+ lift $ declareVariables transState
+ mapM_ smtRun smtToCheck
+ smtRun CheckSat >>= \case
+ Sat -> pure True
+ Unsat -> pure False
+ Unknown _ -> retry
+ other -> do
+ let msg = "Unexpected result while calling 'check-sat': " <> show other
+ Log.withContext Log.CtxAbort $ Log.logMessage $ Text.pack msg
+ throwSMT' msg
+
+ retry = do
+ opts <- lift . SMT $ gets (.options)
+ case opts.retryLimit of
+ Just x | x > 0 -> do
+ lift $ hardResetSolver (updateOptionsOnRetry opts)
+ Log.logMessage ("Retrying with higher timeout" :: Text)
+ solve'
+ _ -> failBecauseUnknown psToCheck
diff --git a/booster/test/rpc-integration/test-log-simplify-json/simplify-log.txt.golden b/booster/test/rpc-integration/test-log-simplify-json/simplify-log.txt.golden
index e7d03842a4..dc3c511503 100644
--- a/booster/test/rpc-integration/test-log-simplify-json/simplify-log.txt.golden
+++ b/booster/test/rpc-integration/test-log-simplify-json/simplify-log.txt.golden
@@ -1,6 +1,7 @@
{"context":["proxy"],"message":"Loading definition from resources/log-simplify-json.kore, main module \"IMP-VERIFICATION\""}
{"context":["proxy"],"message":"Starting RPC server"}
{"context":["proxy"],"message":"Processing request 4"}
+{"context":[{"request":"4"},"booster","execute",{"term":"bd7c50d"},{"term":"4a36bb8"},{"hook":"INT.le"},"failure"],"message":"Hook returned no result"}
{"context":[{"request":"4"},"booster","execute",{"term":"4b03f8b"},{"rewrite":"2821768ca76231d9d23da884f160f8a8a67b03f3575f41bd4fc76649c39a94fb"},"constraint",{"term":"a2a070a"},{"function":"f4c2469bcff9527515b6d36f16040307917bda61067d10b85fb531e142483bee"},"detail"],"message":"UNKNOWN"}
{"context":[{"request":"4"},"booster","execute",{"term":"4b03f8b"},{"rewrite":"2821768ca76231d9d23da884f160f8a8a67b03f3575f41bd4fc76649c39a94fb"},"constraint",{"term":"a2a070a"},{"function":"f4c2469bcff9527515b6d36f16040307917bda61067d10b85fb531e142483bee"},"match","failure","continue"],"message":"Sorts differ: Eq#VarKResult:SortKResult{} =/= !__EXPRESSIONS-SYNTAX_Expr_Expr(_<=__EXPRESSIONS-SYNTAX_Expr_Expr_Expr(\"$n\", \"0\"))"}
{"context":[{"request":"4"},"booster","execute",{"term":"4b03f8b"},{"rewrite":"2821768ca76231d9d23da884f160f8a8a67b03f3575f41bd4fc76649c39a94fb"},"constraint",{"term":"a2a070a"},{"function":"afefecb36598372bc1ba6e5d0b24a00b91796244dc3bd7435e40ca6e9ab33d4b"},"detail"],"message":"UNKNOWN"}
diff --git a/booster/test/rpc-integration/test-substitutions/README.md b/booster/test/rpc-integration/test-substitutions/README.md
index 1c7a208f58..2311202566 100644
--- a/booster/test/rpc-integration/test-substitutions/README.md
+++ b/booster/test/rpc-integration/test-substitutions/README.md
@@ -31,10 +31,9 @@ NB: Booster applies the given substitution before performing any action.
- with an additional constraint `Y = 1 +Int X` (internalised as a substitution)
- leading to a contradictory constraint `X = 1 +Int X` after
rewriting and adding `Y =Int X` from the `ensures`-clause
- - `kore-rpc-booster` returns `vacuous` after 1 step
- - `kore-rpc-dev` returns `vacuous` after 0 steps (detects the contradiction earlier)
- - `kore-rpc-dev` reproduces the exact input as `state` while
- `kore-rpc-booster` splits off `substitution` (from input) and `predicate` (from the rule)
+ - `kore-rpc-booster` and `booster-dev` return `vacuous` after 0 step, substitution `Y` for `X +Int 1` in the state. However, `kore-rpc-booster` and `booster-dev` disagree a little on the value in the substitution, hence the two responses.
+ - `kore-rpc-dev` returns `vacuous` after 0 steps and reproduces the exact input as `state`
+
* `state-circular-equations.execute`
- starts from `concrete-subst`
- with two equations that have circular dependencies: `Y = 1 +Int X`, `X = Y -Int 1`
@@ -42,6 +41,6 @@ NB: Booster applies the given substitution before performing any action.
* `state-symbolic-bottom-predicate.execute`
- starts from `symbolic-subst`
- with an equation that is instantly false: `X = 1 +Int X`
- - leading to a vacuous state in `kore-rpc-booster` after rewriting once,
+ - leading to a vacuous state in `booster-dev` and `kore-rpc-booster`,
- while `kore-rpc-dev` returns `stuck` instantly after 0 steps,
returning the exact input as `state`.
diff --git a/booster/test/rpc-integration/test-substitutions/response-circular-equations.booster-dev b/booster/test/rpc-integration/test-substitutions/response-circular-equations.booster-dev
index b012148190..259a441aeb 100644
--- a/booster/test/rpc-integration/test-substitutions/response-circular-equations.booster-dev
+++ b/booster/test/rpc-integration/test-substitutions/response-circular-equations.booster-dev
@@ -225,61 +225,13 @@
}
},
{
- "tag": "App",
- "name": "Lbl'UndsPlus'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "App",
- "name": "Lbl'UndsPlus'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "1"
- }
- ]
- },
- {
- "tag": "App",
- "name": "Lbl'Unds'-Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "0"
- },
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "1"
- }
- ]
- }
- ]
+ "tag": "EVar",
+ "name": "X",
+ "sort": {
+ "tag": "SortApp",
+ "name": "SortInt",
+ "args": []
+ }
}
]
}
diff --git a/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.booster-dev b/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.booster-dev
deleted file mode 100644
index 9aec4d5289..0000000000
--- a/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.booster-dev
+++ /dev/null
@@ -1,240 +0,0 @@
-{
- "jsonrpc": "2.0",
- "id": 1,
- "result": {
- "reason": "stuck",
- "depth": 1,
- "state": {
- "term": {
- "format": "KORE",
- "version": 1,
- "term": {
- "tag": "App",
- "name": "Lbl'-LT-'generatedTop'-GT-'",
- "sorts": [],
- "args": [
- {
- "tag": "App",
- "name": "Lbl'-LT-'k'-GT-'",
- "sorts": [],
- "args": [
- {
- "tag": "App",
- "name": "kseq",
- "sorts": [],
- "args": [
- {
- "tag": "App",
- "name": "inj",
- "sorts": [
- {
- "tag": "SortApp",
- "name": "SortState",
- "args": []
- },
- {
- "tag": "SortApp",
- "name": "SortKItem",
- "args": []
- }
- ],
- "args": [
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortState",
- "args": []
- },
- "value": "a"
- }
- ]
- },
- {
- "tag": "App",
- "name": "dotk",
- "sorts": [],
- "args": []
- }
- ]
- }
- ]
- },
- {
- "tag": "App",
- "name": "Lbl'-LT-'int'-GT-'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- }
- ]
- },
- {
- "tag": "App",
- "name": "Lbl'-LT-'jnt'-GT-'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "Y",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- }
- ]
- },
- {
- "tag": "App",
- "name": "Lbl'-LT-'generatedCounter'-GT-'",
- "sorts": [],
- "args": [
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "0"
- }
- ]
- }
- ]
- }
- },
- "predicate": {
- "format": "KORE",
- "version": 1,
- "term": {
- "tag": "And",
- "sort": {
- "tag": "SortApp",
- "name": "SortGeneratedTopCell",
- "args": []
- },
- "patterns": [
- {
- "tag": "Equals",
- "argSort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "sort": {
- "tag": "SortApp",
- "name": "SortGeneratedTopCell",
- "args": []
- },
- "first": {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "value": "true"
- },
- "second": {
- "tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "App",
- "name": "Lbl'UndsPlus'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "1"
- }
- ]
- }
- ]
- }
- },
- {
- "tag": "Equals",
- "argSort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "sort": {
- "tag": "SortApp",
- "name": "SortGeneratedTopCell",
- "args": []
- },
- "first": {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "value": "true"
- },
- "second": {
- "tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "EVar",
- "name": "Y",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- }
- ]
- }
- }
- ]
- }
- }
- }
- }
-}
\ No newline at end of file
diff --git a/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.json b/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.json
index be972500f9..8196839aaa 100644
--- a/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.json
+++ b/booster/test/rpc-integration/test-substitutions/response-symbolic-bottom-predicate.json
@@ -3,7 +3,7 @@
"id": 1,
"result": {
"reason": "vacuous",
- "depth": 1,
+ "depth": 0,
"state": {
"term": {
"format": "KORE",
@@ -46,7 +46,7 @@
"name": "SortState",
"args": []
},
- "value": "a"
+ "value": "symbolic-subst"
}
]
},
@@ -115,114 +115,57 @@
"format": "KORE",
"version": 1,
"term": {
- "tag": "And",
+ "tag": "Equals",
+ "argSort": {
+ "tag": "SortApp",
+ "name": "SortBool",
+ "args": []
+ },
"sort": {
"tag": "SortApp",
"name": "SortGeneratedTopCell",
"args": []
},
- "patterns": [
- {
- "tag": "Equals",
- "argSort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "sort": {
- "tag": "SortApp",
- "name": "SortGeneratedTopCell",
- "args": []
- },
- "first": {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "value": "true"
- },
- "second": {
- "tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "App",
- "name": "Lbl'UndsPlus'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "X",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "1"
- }
- ]
- }
- ]
- }
+ "first": {
+ "tag": "DV",
+ "sort": {
+ "tag": "SortApp",
+ "name": "SortBool",
+ "args": []
},
- {
- "tag": "Equals",
- "argSort": {
- "tag": "SortApp",
- "name": "SortBool",
- "args": []
- },
- "sort": {
- "tag": "SortApp",
- "name": "SortGeneratedTopCell",
- "args": []
- },
- "first": {
- "tag": "DV",
+ "value": "true"
+ },
+ "second": {
+ "tag": "App",
+ "name": "Lbl'UndsEqlsEqls'Int'Unds'",
+ "sorts": [],
+ "args": [
+ {
+ "tag": "EVar",
+ "name": "X",
"sort": {
"tag": "SortApp",
- "name": "SortBool",
+ "name": "SortInt",
"args": []
- },
- "value": "true"
+ }
},
- "second": {
+ {
"tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
+ "name": "Lbl'UndsPlus'Int'Unds'",
"sorts": [],
"args": [
{
- "tag": "EVar",
- "name": "X",
+ "tag": "DV",
"sort": {
"tag": "SortApp",
"name": "SortInt",
"args": []
- }
+ },
+ "value": "1"
},
{
"tag": "EVar",
- "name": "Y",
+ "name": "X",
"sort": {
"tag": "SortApp",
"name": "SortInt",
@@ -231,8 +174,8 @@
}
]
}
- }
- ]
+ ]
+ }
}
}
}
diff --git a/booster/test/rpc-integration/test-substitutions/response-symbolic-two-substitutions.booster-dev b/booster/test/rpc-integration/test-substitutions/response-symbolic-two-substitutions.booster-dev
index 40c6e95a24..58a6ab33aa 100644
--- a/booster/test/rpc-integration/test-substitutions/response-symbolic-two-substitutions.booster-dev
+++ b/booster/test/rpc-integration/test-substitutions/response-symbolic-two-substitutions.booster-dev
@@ -87,22 +87,22 @@
"sorts": [],
"args": [
{
- "tag": "DV",
+ "tag": "EVar",
+ "name": "X",
"sort": {
"tag": "SortApp",
"name": "SortInt",
"args": []
- },
- "value": "1"
+ }
},
{
- "tag": "EVar",
- "name": "X",
+ "tag": "DV",
"sort": {
"tag": "SortApp",
"name": "SortInt",
"args": []
- }
+ },
+ "value": "1"
}
]
}
diff --git a/booster/test/rpc-integration/test-vacuous/README.md b/booster/test/rpc-integration/test-vacuous/README.md
index 00c6823dc4..9464e3a6de 100644
--- a/booster/test/rpc-integration/test-vacuous/README.md
+++ b/booster/test/rpc-integration/test-vacuous/README.md
@@ -38,7 +38,7 @@ Rules `init` and `AC` introduce constraints on this variable:
_Expected:_
- The rewrite is stuck with `dN \and...(contradiction)`
- The result is simplified and discovered to be `vacuous` (with state `d`).
-1) _vacuous-but-rewritten_
+1) _vacuous-not-rewritten_
_Input:_
- `execute` request with initial state `bN \and N
diff --git a/booster/test/rpc-integration/test-vacuous/response-vacuous-but-rewritten.json b/booster/test/rpc-integration/test-vacuous/response-vacuous-not-rewritten.json
similarity index 84%
rename from booster/test/rpc-integration/test-vacuous/response-vacuous-but-rewritten.json
rename to booster/test/rpc-integration/test-vacuous/response-vacuous-not-rewritten.json
index b7cf387833..3d880fa432 100644
--- a/booster/test/rpc-integration/test-vacuous/response-vacuous-but-rewritten.json
+++ b/booster/test/rpc-integration/test-vacuous/response-vacuous-not-rewritten.json
@@ -3,7 +3,7 @@
"id": 1,
"result": {
"reason": "vacuous",
- "depth": 1,
+ "depth": 0,
"state": {
"term": {
"format": "KORE",
@@ -46,7 +46,7 @@
"name": "SortState",
"args": []
},
- "value": "d"
+ "value": "b"
}
]
},
@@ -129,7 +129,7 @@
},
"second": {
"tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
+ "name": "Lbl'UndsEqlsSlshEqls'Int'Unds'",
"sorts": [],
"args": [
{
@@ -176,33 +176,26 @@
},
"second": {
"tag": "App",
- "name": "LblnotBool'Unds'",
+ "name": "Lbl'UndsEqlsEqls'Int'Unds'",
"sorts": [],
"args": [
{
- "tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "N",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "0"
- }
- ]
+ "tag": "EVar",
+ "name": "N",
+ "sort": {
+ "tag": "SortApp",
+ "name": "SortInt",
+ "args": []
+ }
+ },
+ {
+ "tag": "DV",
+ "sort": {
+ "tag": "SortApp",
+ "name": "SortInt",
+ "args": []
+ },
+ "value": "0"
}
]
}
diff --git a/booster/test/rpc-integration/test-vacuous/response-vacuous-but-rewritten.kore-rpc-dev b/booster/test/rpc-integration/test-vacuous/response-vacuous-not-rewritten.kore-rpc-dev
similarity index 100%
rename from booster/test/rpc-integration/test-vacuous/response-vacuous-but-rewritten.kore-rpc-dev
rename to booster/test/rpc-integration/test-vacuous/response-vacuous-not-rewritten.kore-rpc-dev
diff --git a/booster/test/rpc-integration/test-vacuous/response-vacuous-without-rewrite.json b/booster/test/rpc-integration/test-vacuous/response-vacuous-without-rewrite.json
index d3b14044a8..9d4198621d 100644
--- a/booster/test/rpc-integration/test-vacuous/response-vacuous-without-rewrite.json
+++ b/booster/test/rpc-integration/test-vacuous/response-vacuous-without-rewrite.json
@@ -129,7 +129,7 @@
},
"second": {
"tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
+ "name": "Lbl'UndsEqlsSlshEqls'Int'Unds'",
"sorts": [],
"args": [
{
@@ -176,33 +176,26 @@
},
"second": {
"tag": "App",
- "name": "LblnotBool'Unds'",
+ "name": "Lbl'UndsEqlsEqls'Int'Unds'",
"sorts": [],
"args": [
{
- "tag": "App",
- "name": "Lbl'UndsEqlsEqls'Int'Unds'",
- "sorts": [],
- "args": [
- {
- "tag": "EVar",
- "name": "N",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- }
- },
- {
- "tag": "DV",
- "sort": {
- "tag": "SortApp",
- "name": "SortInt",
- "args": []
- },
- "value": "0"
- }
- ]
+ "tag": "EVar",
+ "name": "N",
+ "sort": {
+ "tag": "SortApp",
+ "name": "SortInt",
+ "args": []
+ }
+ },
+ {
+ "tag": "DV",
+ "sort": {
+ "tag": "SortApp",
+ "name": "SortInt",
+ "args": []
+ },
+ "value": "0"
}
]
}
diff --git a/booster/test/rpc-integration/test-vacuous/state-vacuous-but-rewritten.execute b/booster/test/rpc-integration/test-vacuous/state-vacuous-not-rewritten.execute
similarity index 100%
rename from booster/test/rpc-integration/test-vacuous/state-vacuous-but-rewritten.execute
rename to booster/test/rpc-integration/test-vacuous/state-vacuous-not-rewritten.execute
diff --git a/booster/unit-tests/Test/Booster/Pattern/Rewrite.hs b/booster/unit-tests/Test/Booster/Pattern/Rewrite.hs
index 3c295c7ebe..63d9b70f7b 100644
--- a/booster/unit-tests/Test/Booster/Pattern/Rewrite.hs
+++ b/booster/unit-tests/Test/Booster/Pattern/Rewrite.hs
@@ -294,7 +294,7 @@ runRewrite t = do
conf <- testConf
(counter, _, res) <-
runNoLoggingT $
- performRewrite conf $
+ performRewrite conf mempty $
Pattern_ t
pure (counter, fmap (.term) res)
@@ -438,7 +438,7 @@ supportsDepthControl =
rewritesToDepth (MaxDepth depth) (Steps n) t t' f = do
conf <- testConf
(counter, _, res) <-
- runNoLoggingT $ performRewrite conf{mbMaxDepth = Just depth} $ Pattern_ t
+ runNoLoggingT $ performRewrite conf{mbMaxDepth = Just depth} mempty $ Pattern_ t
(counter, fmap (.term) res) @?= (n, f t')
supportsCutPoints :: TestTree
@@ -492,7 +492,7 @@ supportsCutPoints =
conf <- testConf
(counter, _, res) <-
runNoLoggingT $
- performRewrite conf{cutLabels = [lbl]} $
+ performRewrite conf{cutLabels = [lbl]} mempty $
Pattern_ t
(counter, fmap (.term) res) @?= (n, f t')
@@ -524,5 +524,5 @@ supportsTerminalRules =
rewritesToTerminal lbl (Steps n) t t' f = do
conf <- testConf
(counter, _, res) <-
- runNoLoggingT $ performRewrite conf{terminalLabels = [lbl]} $ Pattern_ t
+ runNoLoggingT $ performRewrite conf{terminalLabels = [lbl]} mempty $ Pattern_ t
(counter, fmap (.term) res) @?= (n, f t')
diff --git a/dev-tools/package.yaml b/dev-tools/package.yaml
index 3ab1e66bc3..cf002942bd 100644
--- a/dev-tools/package.yaml
+++ b/dev-tools/package.yaml
@@ -157,6 +157,7 @@ executables:
- base
- aeson
- bytestring
+ - containers
- hs-backend-booster
- prettyprinter
- text
diff --git a/dev-tools/pretty/Pretty.hs b/dev-tools/pretty/Pretty.hs
index e7a2400bc9..0dafebdf71 100644
--- a/dev-tools/pretty/Pretty.hs
+++ b/dev-tools/pretty/Pretty.hs
@@ -9,6 +9,15 @@ module Main (
main,
) where
+import Control.Monad.Trans.Except
+import Data.Aeson (eitherDecode)
+import Data.ByteString.Lazy qualified as BS
+import Data.Map qualified as Map
+import Data.Text.IO qualified as Text
+import Prettyprinter
+import System.Environment (getArgs)
+
+import Booster.Pattern.Base (Term, Variable (..))
import Booster.Pattern.Pretty
import Booster.Prettyprinter (renderDefault)
import Booster.Syntax.Json (KoreJson (..))
@@ -21,12 +30,6 @@ import Booster.Syntax.Json.Internalise (
pattern DisallowAlias,
)
import Booster.Syntax.ParsedKore (internalise, parseKoreDefinition)
-import Control.Monad.Trans.Except
-import Data.Aeson (eitherDecode)
-import Data.ByteString.Lazy qualified as BS
-import Data.Text.IO qualified as Text
-import Prettyprinter
-import System.Environment (getArgs)
main :: IO ()
main = do
@@ -40,9 +43,11 @@ main = do
Left err -> putStrLn $ "Error: " ++ err
Right KoreJson{term} -> do
case runExcept $ internalisePattern DisallowAlias CheckSubsorts Nothing internalDef term of
- Right (trm, _subst, _unsupported) -> do
+ Right (trm, subst, _unsupported) -> do
putStrLn "Pretty-printing pattern: "
putStrLn $ renderDefault $ pretty' @'[Decoded] trm
+ putStrLn "Substitution: "
+ mapM_ (putStrLn . prettyPrintSubstItem) (Map.toList subst)
Left (NoTermFound _) ->
case runExcept $ internalisePredicates DisallowAlias CheckSubsorts Nothing internalDef [term] of
Left es -> error (show es)
@@ -53,7 +58,10 @@ main = do
putStrLn "Ceil predicates: "
mapM_ (putStrLn . renderDefault . pretty' @'[Decoded]) ts.ceilPredicates
putStrLn "Substitution: "
- mapM_ (putStrLn . renderDefault . pretty' @'[Decoded]) ts.substitution
+ mapM_ (putStrLn . prettyPrintSubstItem) (Map.toList ts.substitution)
putStrLn "Unsupported predicates: "
mapM_ print ts.unsupported
Left err -> error (show err)
+
+prettyPrintSubstItem :: (Variable, Term) -> String
+prettyPrintSubstItem (v, t) = show v.variableName <> " |-> " <> (renderDefault . pretty' @'[Decoded] $ t)
diff --git a/scripts/booster-integration-tests.sh b/scripts/booster-integration-tests.sh
index 3a24d3db64..bde164c195 100755
--- a/scripts/booster-integration-tests.sh
+++ b/scripts/booster-integration-tests.sh
@@ -28,12 +28,12 @@ for dir in $(ls -d test-*); do
name=${dir##test-}
echo "Running $name..."
case "$name" in
- "a-to-f" | "diamond")
+ "a-to-f" | "diamond" | "substitutions")
SERVER=$BOOSTER_DEV ./runDirectoryTest.sh test-$name $@
SERVER=$KORE_RPC_DEV ./runDirectoryTest.sh test-$name $@
SERVER=$KORE_RPC_BOOSTER ./runDirectoryTest.sh test-$name $@
;;
- "substitutions" | "vacuous" | "pathological-add-module")
+ "vacuous" | "pathological-add-module")
SERVER=$KORE_RPC_DEV ./runDirectoryTest.sh test-$name $@
SERVER=$KORE_RPC_BOOSTER ./runDirectoryTest.sh test-$name $@
;;
diff --git a/scripts/performance-tests-kevm.sh b/scripts/performance-tests-kevm.sh
index 2ca4ec2df3..e0f2c27563 100755
--- a/scripts/performance-tests-kevm.sh
+++ b/scripts/performance-tests-kevm.sh
@@ -107,6 +107,7 @@ feature_shell "cd kevm-pyk && poetry run pytest src/tests/integration/test_prove
mkdir -p $SCRIPT_DIR/logs
# use special options if given, but restore KORE_RPC_OPTS afterwards
+FEATURE_SERVER_OPTS=${FEATURE_SERVER_OPTS:-''}
if [ ! -z "${FEATURE_SERVER_OPTS}" ]; then
echo "Using special options '${FEATURE_SERVER_OPTS}' via KORE_RPC_OPTS"
if [ ! -z "${KORE_RPC_OPTS:-}" ]; then