diff --git a/benchmark/src/test/scala/com/wavesplatform/state/StateSyntheticBenchmark.scala b/benchmark/src/test/scala/com/wavesplatform/state/StateSyntheticBenchmark.scala index 88bba795815..ec4989cb30b 100644 --- a/benchmark/src/test/scala/com/wavesplatform/state/StateSyntheticBenchmark.scala +++ b/benchmark/src/test/scala/com/wavesplatform/state/StateSyntheticBenchmark.scala @@ -67,7 +67,7 @@ object StateSyntheticBenchmark { val textScript = "sigVerify(tx.bodyBytes,tx.proofs[0],tx.senderPublicKey)" val untypedScript = Parser.parseExpr(textScript).get.value - val typedScript = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untypedScript).explicitGet()._1 + val typedScript = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untypedScript).explicitGet()._1 val setScriptBlock = nextBlock( Seq( diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationError.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationError.scala index 08e377eb8f4..243fa22889e 100644 --- a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationError.scala +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationError.scala @@ -1,7 +1,5 @@ package com.wavesplatform.lang.v1.compiler -import java.nio.charset.StandardCharsets - import cats.Show import com.wavesplatform.lang.v1.ContractLimits import com.wavesplatform.lang.v1.compiler.Types.* @@ -9,6 +7,8 @@ import com.wavesplatform.lang.v1.evaluator.ctx.FunctionTypeSignature import com.wavesplatform.lang.v1.parser.Expressions import com.wavesplatform.lang.v1.parser.Expressions.{Declaration, PART} +import java.nio.charset.StandardCharsets + sealed trait CompilationError { def start: Int def end: Int @@ -52,18 +52,11 @@ object CompilationError { s"but ${names.map(n => s"`$n`").mkString(", ")} found" } - final case class UnusedCaseVariables(start: Int, end: Int, names: List[String]) extends CompilationError { - val message = s"Unused case variable(s) ${names.map(n => s"`$n`").mkString(", ")}" - } - final case class AlreadyDefined(start: Int, end: Int, name: String, isFunction: Boolean) extends CompilationError { val message = if (isFunction) s"Value '$name' can't be defined because function with this name is already defined" else s"Value '$name' already defined in the scope" } - final case class NonExistingType(start: Int, end: Int, name: String, existing: List[String]) extends CompilationError { - val message = s"Value '$name' declared as non-existing type, while all possible types are $existing" - } final case class BadFunctionSignatureSameArgNames(start: Int, end: Int, name: String) extends CompilationError { val message = s"Function '$name' declared with duplicating argument names" diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationStepResultDec.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationStepResultDec.scala new file mode 100644 index 00000000000..aa0f6f7537f --- /dev/null +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationStepResultDec.scala @@ -0,0 +1,12 @@ +package com.wavesplatform.lang.v1.compiler + +import com.wavesplatform.lang.v1.compiler.Types.FINAL +import com.wavesplatform.lang.v1.parser.Expressions + +case class CompilationStepResultDec( + ctx: CompilerContext, + dec: Terms.DECLARATION, + t: FINAL, + parseNodeExpr: Expressions.Declaration, + errors: Iterable[CompilationError] = Iterable.empty +) diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationStepResultExpr.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationStepResultExpr.scala new file mode 100644 index 00000000000..bf8eb9fd0c5 --- /dev/null +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/CompilationStepResultExpr.scala @@ -0,0 +1,12 @@ +package com.wavesplatform.lang.v1.compiler + +import com.wavesplatform.lang.v1.compiler.Types.FINAL +import com.wavesplatform.lang.v1.parser.Expressions + +case class CompilationStepResultExpr( + ctx: CompilerContext, + expr: Terms.EXPR, + t: FINAL, + parseNodeExpr: Expressions.EXPR, + errors: Iterable[CompilationError] = Iterable.empty +) diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ContractCompiler.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ContractCompiler.scala index a8647939e8c..1240d5aae79 100644 --- a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ContractCompiler.scala +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ContractCompiler.scala @@ -13,7 +13,7 @@ import com.wavesplatform.lang.contract.meta.{MetaMapper, V1 as MetaV1, V2 as Met import com.wavesplatform.lang.directives.values.{StdLibVersion, V3, V6} import com.wavesplatform.lang.v1.compiler.CompilationError.{AlreadyDefined, Generic, UnionNotAllowedForCallableArgs, WrongArgumentType} import com.wavesplatform.lang.v1.compiler.CompilerContext.{VariableInfo, vars} -import com.wavesplatform.lang.v1.compiler.ExpressionCompiler.* +import com.wavesplatform.lang.v1.compiler.ContractCompiler.* import com.wavesplatform.lang.v1.compiler.ScriptResultSource.FreeCall import com.wavesplatform.lang.v1.compiler.Terms.EXPR import com.wavesplatform.lang.v1.compiler.Types.{BOOLEAN, BYTESTR, LONG, STRING} @@ -25,16 +25,13 @@ import com.wavesplatform.lang.v1.parser.Expressions.{FUNC, PART, Type} import com.wavesplatform.lang.v1.parser.Parser.LibrariesOffset import com.wavesplatform.lang.v1.parser.{Expressions, Parser} import com.wavesplatform.lang.v1.task.imports.* -import com.wavesplatform.lang.v1.{ContractLimits, FunctionHeader, compiler} +import com.wavesplatform.lang.v1.{ContractLimits, FunctionHeader} import scala.annotation.tailrec -object ContractCompiler { - val FreeCallInvocationArg = "i" - +class ContractCompiler(version: StdLibVersion) extends ExpressionCompiler(version) { private def compileAnnotatedFunc( af: Expressions.ANNOTATEDFUNC, - version: StdLibVersion, saveExprContext: Boolean, allowIllFormedStrings: Boolean, source: ScriptResultSource @@ -88,10 +85,10 @@ object ContractCompiler { _.flatMap(_.dic(version).toList).map(nameAndType => (nameAndType._1, VariableInfo(AnyPos, nameAndType._2))) ) .getOrElse(List.empty) - unionInCallableErrs <- checkCallableUnions(af, annotationsWithErr._1.toList.flatten, version) + unionInCallableErrs <- checkCallableUnions(af, annotationsWithErr._1.toList.flatten) compiledBody <- local { modify[Id, CompilerContext, CompilationError](vars.modify(_)(_ ++ annotationBindings)).flatMap(_ => - compiler.ExpressionCompiler.compileFunc(af.f.position, af.f, saveExprContext, annotationBindings.map(_._1), allowIllFormedStrings) + compileFunc(af.f.position, af.f, saveExprContext, annotationBindings.map(_._1), allowIllFormedStrings) ) } annotatedFuncWithErr <- getCompiledAnnotatedFunc(annotationsWithErr, compiledBody._1).handleError() @@ -132,7 +129,6 @@ object ContractCompiler { private def compileContract( parsedDapp: Expressions.DAPP, - version: StdLibVersion, needCompaction: Boolean, removeUnusedCode: Boolean, source: ScriptResultSource, @@ -149,7 +145,7 @@ object ContractCompiler { annFuncArgTypesErr <- validateAnnotatedFuncsArgTypes(parsedDapp).handleError() compiledAnnFuncsWithErr <- parsedDapp.fs .traverse[CompileM, (Option[AnnotatedFunction], List[(String, Types.FINAL)], Expressions.ANNOTATEDFUNC, Iterable[CompilationError])](af => - local(compileAnnotatedFunc(af, version, saveExprContext, allowIllFormedStrings, source)) + local(compileAnnotatedFunc(af, saveExprContext, allowIllFormedStrings, source)) ) annotatedFuncs = compiledAnnFuncsWithErr.filter(_._1.nonEmpty).map(_._1.get) parsedNodeAFuncs = compiledAnnFuncsWithErr.map(_._3) @@ -236,7 +232,7 @@ object ContractCompiler { } yield result } - def handleValid[T](part: PART[T]): CompileM[PART.VALID[T]] = part match { + private def handleValid[T](part: PART[T]): CompileM[PART.VALID[T]] = part match { case x: PART.VALID[T] => x.pure[CompileM] case PART.INVALID(p, message) => raiseError(Generic(p.start, p.end, message)) } @@ -305,13 +301,7 @@ object ContractCompiler { } } - val primitiveCallableTypes: Set[String] = - Set(LONG, BYTESTR, BOOLEAN, STRING).map(_.name) - - val allowedCallableTypesV4: Set[String] = - primitiveCallableTypes + "List[]" - - private def validateDuplicateVarsInContract(contract: Expressions.DAPP): CompileM[Any] = { + private def validateDuplicateVarsInContract(contract: Expressions.DAPP): CompileM[Any] = for { ctx <- get[Id, CompilerContext, CompilationError] annotationVars = contract.fs.flatMap(_.anns.flatMap(_.args)).traverse[CompileM, PART.VALID[String]](handleValid) @@ -339,7 +329,52 @@ object ContractCompiler { } } } yield () + + private def checkCallableUnions( + func: Expressions.ANNOTATEDFUNC, + annotations: List[Annotation], + ): CompileM[Seq[UnionNotAllowedForCallableArgs]] = { + @tailrec + def containsUnion(tpe: Type): Boolean = + tpe match { + case Expressions.Union(types) if types.size > 1 => true + case Expressions.Single(PART.VALID(_, Type.ListTypeName), Some(PART.VALID(_, Expressions.Union(types)))) if types.size > 1 => true + case Expressions.Single( + PART.VALID(_, Type.ListTypeName), + Some(PART.VALID(_, inner @ Expressions.Single(PART.VALID(_, Type.ListTypeName), _))) + ) => + containsUnion(inner) + case _ => false + } + + val isCallable = annotations.exists { + case CallableAnnotation(_) => true + case _ => false + } + + if (version < V6 || !isCallable) { + Seq.empty[UnionNotAllowedForCallableArgs].pure[CompileM] + } else { + func.f.args + .filter { case (_, tpe) => + containsUnion(tpe) + } + .map { case (argName, _) => + UnionNotAllowedForCallableArgs(argName.position.start, argName.position.end) + } + .pure[CompileM] + } } +} + +object ContractCompiler { + val FreeCallInvocationArg = "i" + + val primitiveCallableTypes: Set[String] = + Set(LONG, BYTESTR, BOOLEAN, STRING).map(_.name) + + val allowedCallableTypesV4: Set[String] = + primitiveCallableTypes + "List[]" def apply( c: CompilerContext, @@ -350,7 +385,8 @@ object ContractCompiler { removeUnusedCode: Boolean = false, allowIllFormedStrings: Boolean = false ): Either[String, DApp] = { - compileContract(contract, version, needCompaction, removeUnusedCode, source, allowIllFormedStrings = allowIllFormedStrings) + new ContractCompiler(version) + .compileContract(contract, needCompaction, removeUnusedCode, source, allowIllFormedStrings = allowIllFormedStrings) .run(c) .map( _._2 @@ -375,7 +411,7 @@ object ContractCompiler { val parser = new Parser(version)(offset) parser.parseContract(input) match { case fastparse.Parsed.Success(xs, _) => - ContractCompiler(ctx, xs, version, source, needCompaction, removeUnusedCode, allowIllFormedStrings) match { + apply(ctx, xs, version, source, needCompaction, removeUnusedCode, allowIllFormedStrings) match { case Left(err) => Left(err) case Right(c) => Right(c) } @@ -396,7 +432,8 @@ object ContractCompiler { new Parser(version)(offset) .parseDAPPWithErrorRecovery(input) .flatMap { case (parseResult, removedCharPosOpt) => - compileContract(parseResult, version, needCompaction, removeUnusedCode, ScriptResultSource.CallableFunction, saveExprContext) + new ContractCompiler(version) + .compileContract(parseResult, needCompaction, removeUnusedCode, ScriptResultSource.CallableFunction, saveExprContext) .run(ctx) .map( _._2 @@ -437,41 +474,4 @@ object ContractCompiler { Left(parser.toString(input, f)) } } - - private def checkCallableUnions( - func: Expressions.ANNOTATEDFUNC, - annotations: List[Annotation], - version: StdLibVersion - ): CompileM[Seq[UnionNotAllowedForCallableArgs]] = { - @tailrec - def containsUnion(tpe: Type): Boolean = - tpe match { - case Expressions.Union(types) if types.size > 1 => true - case Expressions.Single(PART.VALID(_, Type.ListTypeName), Some(PART.VALID(_, Expressions.Union(types)))) if types.size > 1 => true - case Expressions.Single( - PART.VALID(_, Type.ListTypeName), - Some(PART.VALID(_, inner @ Expressions.Single(PART.VALID(_, Type.ListTypeName), _))) - ) => - containsUnion(inner) - case _ => false - } - - val isCallable = annotations.exists { - case CallableAnnotation(_) => true - case _ => false - } - - if (version < V6 || !isCallable) { - Seq.empty[UnionNotAllowedForCallableArgs].pure[CompileM] - } else { - func.f.args - .filter { case (_, tpe) => - containsUnion(tpe) - } - .map { case (argName, _) => - UnionNotAllowedForCallableArgs(argName.position.start, argName.position.end) - } - .pure[CompileM] - } - } } diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ExpressionCompiler.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ExpressionCompiler.scala index 8a45f5bfee0..9a9ae79997e 100644 --- a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ExpressionCompiler.scala +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/ExpressionCompiler.scala @@ -4,28 +4,17 @@ import cats.implicits.* import cats.{Id, Show} import com.wavesplatform.common.state.ByteStr import com.wavesplatform.lang.{CommonError, StringOps} -import com.wavesplatform.lang.directives.values.StdLibVersion +import com.wavesplatform.lang.directives.values.{StdLibVersion, V8} import com.wavesplatform.lang.v1.compiler.CompilationError.* import com.wavesplatform.lang.v1.compiler.CompilerContext.* +import com.wavesplatform.lang.v1.compiler.ExpressionCompiler.* import com.wavesplatform.lang.v1.compiler.Terms.* import com.wavesplatform.lang.v1.compiler.Types.* import com.wavesplatform.lang.v1.evaluator.EvaluatorV1.* import com.wavesplatform.lang.v1.evaluator.ctx.* import com.wavesplatform.lang.v1.evaluator.ctx.impl.GlobalValNames import com.wavesplatform.lang.v1.parser.BinaryOperation.* -import com.wavesplatform.lang.v1.parser.Expressions.{ - BINARY_OP, - CompositePattern, - ConstsPat, - MATCH_CASE, - ObjPat, - PART, - Pos, - Single, - TuplePat, - Type, - TypedVar -} +import com.wavesplatform.lang.v1.parser.Expressions.{BINARY_OP, CompositePattern, ConstsPat, MATCH_CASE, ObjPat, PART, Pos, Single, TuplePat, Type, TypedVar} import com.wavesplatform.lang.v1.parser.Parser.LibrariesOffset import com.wavesplatform.lang.v1.parser.{BinaryOperation, Expressions, Parser} import com.wavesplatform.lang.v1.task.imports.* @@ -34,90 +23,7 @@ import com.wavesplatform.lang.v1.{ContractLimits, FunctionHeader} import java.nio.charset.StandardCharsets import scala.util.Try -object ExpressionCompiler { - case class CompilationStepResultExpr( - ctx: CompilerContext, - expr: Terms.EXPR, - t: FINAL, - parseNodeExpr: Expressions.EXPR, - errors: Iterable[CompilationError] = Iterable.empty - ) - - case class CompilationStepResultDec( - ctx: CompilerContext, - dec: Terms.DECLARATION, - t: FINAL, - parseNodeExpr: Expressions.Declaration, - errors: Iterable[CompilationError] = Iterable.empty - ) - - def compile( - input: String, - offset: LibrariesOffset, - ctx: CompilerContext, - version: StdLibVersion, - allowIllFormedStrings: Boolean = false - ): Either[String, (EXPR, FINAL)] = { - val parser = new Parser(version)(offset) - parser.parseExpr(input) match { - case fastparse.Parsed.Success(xs, _) => ExpressionCompiler(ctx, xs, allowIllFormedStrings) - case f: fastparse.Parsed.Failure => Left(parser.toString(input, f)) - } - } - - def compileBoolean(input: String, offset: LibrariesOffset, ctx: CompilerContext, version: StdLibVersion): Either[String, EXPR] = { - compile(input, offset, ctx, version).flatMap { - case (expr, BOOLEAN) => Right(expr) - case _ => Left("Script should return boolean") - } - } - - def compileUntyped(input: String, offset: LibrariesOffset, ctx: CompilerContext, version: StdLibVersion): Either[String, EXPR] = { - compile(input, offset, ctx, version) - .map { case (expr, _) => expr } - } - - def compileWithParseResult( - input: String, - offset: LibrariesOffset, - ctx: CompilerContext, - version: StdLibVersion, - saveExprContext: Boolean = true - ): Either[(String, Int, Int), (EXPR, Expressions.SCRIPT, Iterable[CompilationError])] = - new Parser(version)(offset) - .parseExpressionWithErrorRecovery(input) - .flatMap { case (parseResult, removedCharPosOpt) => - compileExprWithCtx(parseResult.expr, saveExprContext, allowIllFormedStrings = false) - .run(ctx) - .value - ._2 - .map { compRes => - val errorList = - compRes.errors ++ - (if (compRes.t equivalent BOOLEAN) Nil else List(Generic(0, 0, "Script should return boolean"))) ++ - (if (removedCharPosOpt.isEmpty) - Nil - else - List( - Generic( - removedCharPosOpt.get.start, - removedCharPosOpt.get.end, - "Parsing failed. Some chars was removed as result of recovery process." - ) - )) - (compRes.expr, parseResult.copy(expr = compRes.parseNodeExpr), errorList) - } - .leftMap(e => (s"Compilation failed: ${Show[CompilationError].show(e)}", e.start, e.end)) - } - - def compileDecls(input: String, offset: LibrariesOffset, ctx: CompilerContext, version: StdLibVersion): Either[String, EXPR] = { - val adjustedDecls = s"$input\n${GlobalValNames.Unit}" - compileUntyped(adjustedDecls, offset, ctx, version) - } - - private def compileExpr(expr: Expressions.EXPR): CompileM[(Terms.EXPR, FINAL, Expressions.EXPR, Iterable[CompilationError])] = - compileExprWithCtx(expr, allowIllFormedStrings = false).map(r => (r.expr, r.t, r.parseNodeExpr, r.errors)) - +class ExpressionCompiler(val version: StdLibVersion) { private def compileExprWithCtx( expr: Expressions.EXPR, saveExprContext: Boolean = false, @@ -172,6 +78,10 @@ object ExpressionCompiler { } } + private def compileExpr(expr: Expressions.EXPR): CompileM[(Terms.EXPR, FINAL, Expressions.EXPR, Iterable[CompilationError])] = + compileExprWithCtx(expr, allowIllFormedStrings = false) + .map(r => (r.expr, r.t, r.parseNodeExpr, r.errors)) + private def compileIf( p: Pos, condExpr: Expressions.EXPR, @@ -353,63 +263,7 @@ object ExpressionCompiler { } } yield result - private def exprContainsRef(expr: Expressions.EXPR, ref: String): Boolean = - expr match { - case Expressions.GETTER(_, expr, _, _, _, _) => - exprContainsRef(expr, ref) - - case Expressions.BLOCK(_, decl, body, _, _) => - val refIsOverlappedByDecl = - decl.name match { - case PART.VALID(_, name) if name == ref => true - case _ => false - } - if (refIsOverlappedByDecl) false - else { - val declContainsRef = - decl match { - case Expressions.LET(_, _, value, _, _) => - exprContainsRef(value, ref) - case Expressions.FUNC(_, expr, _, args) => - val refIsOverlappedByArg = - args.exists { - case (PART.VALID(_, name), _) if name == ref => true - case _ => false - } - if (!refIsOverlappedByArg) exprContainsRef(expr, ref) - else false - } - declContainsRef || exprContainsRef(body, ref) - } - - case Expressions.IF(_, cond, ifTrue, ifFalse, _, _) => - exprContainsRef(cond, ref) || - exprContainsRef(ifTrue, ref) || - exprContainsRef(ifFalse, ref) - - case Expressions.FUNCTION_CALL(_, _, args, _, _) => - args.exists(exprContainsRef(_, ref)) - - case Expressions.REF(_, PART.VALID(_, name), _, _) if name == ref => - true - - case BINARY_OP(_, a, _, b, _, _) => - exprContainsRef(a, ref) || exprContainsRef(b, ref) - - case Expressions.MATCH(_, matchingExpr, cases, _, _) => - exprContainsRef(matchingExpr, ref) || - cases.exists { - case MATCH_CASE(_, TypedVar(Some(PART.VALID(_, varName)), _), caseExpr, _, _) if varName != ref => - exprContainsRef(caseExpr, ref) - case MATCH_CASE(_, TypedVar(None, _), caseExpr, _, _) => - exprContainsRef(caseExpr, ref) - case _ => false - } - - case _ => false - } - - def compileBlock( + private def compileBlock( pos: Expressions.Pos, declaration: Expressions.Declaration, expr: Expressions.EXPR, @@ -439,7 +293,12 @@ object ExpressionCompiler { _.getBytes(StandardCharsets.UTF_8).length <= ContractLimits.MaxDeclarationNameInBytes ) - def compileLet(p: Pos, let: Expressions.LET, saveExprContext: Boolean, allowIllFormedStrings: Boolean): CompileM[CompilationStepResultDec] = + protected def compileLet( + p: Pos, + let: Expressions.LET, + saveExprContext: Boolean, + allowIllFormedStrings: Boolean + ): CompileM[CompilationStepResultDec] = for { _ <- checkDeclarationNameSize(p, let) letNameWithErr <- validateShadowing(p, let).handleError() @@ -458,7 +317,7 @@ object ExpressionCompiler { } } yield result - def compileFunc( + protected def compileFunc( p: Pos, func: Expressions.FUNC, saveExprContext: Boolean, @@ -508,10 +367,10 @@ object ExpressionCompiler { } yield (result, argTypesWithErr._1.map(_.map(nameAnfInfo => (nameAnfInfo._1, nameAnfInfo._2.vType))).getOrElse(List.empty)) } - def updateCtx(letName: String, letType: Types.FINAL, p: Pos): CompileM[Unit] = + protected def updateCtx(letName: String, letType: Types.FINAL, p: Pos): CompileM[Unit] = modify[Id, CompilerContext, CompilationError](vars.modify(_)(_ + (letName -> VariableInfo(p, letType)))) - def updateCtx(funcName: String, typeSig: FunctionTypeSignature, p: Pos): CompileM[Unit] = + protected def updateCtx(funcName: String, typeSig: FunctionTypeSignature, p: Pos): CompileM[Unit] = modify[Id, CompilerContext, CompilationError](functions.modify(_)(_ + (funcName -> FunctionInfo(p, List(typeSig))))) private def compileLetBlock( @@ -753,12 +612,7 @@ object ExpressionCompiler { } } - def mkIf(p: Pos, cond: EXPR, ifTrue: (EXPR, FINAL), ifFalse: (EXPR, FINAL)): Either[CompilationError, (EXPR, FINAL)] = { - val t = TypeInferrer.findCommonType(ifTrue._2, ifFalse._2) - (IF(cond, ifTrue._1, ifFalse._1), t).asRight - } - - def mkIfCases( + private def mkIfCases( cases: List[MATCH_CASE], caseTypes: List[FINAL], refTmp: Expressions.REF, @@ -840,7 +694,8 @@ object ExpressionCompiler { val typeIf = tTypes.foldLeft(isInst(hType))((other, matchType) => BINARY_OP(mc.position, isInst(matchType), BinaryOperation.OR_OP, other)) Right(makeIfCase(typeIf, blockWithNewVar, further)) - case Nil => ??? + case Nil => + ??? } } yield cases case (_: TypedVar, t) => @@ -849,51 +704,49 @@ object ExpressionCompiler { case (ConstsPat(consts, _), _) => val cond = consts .map(c => BINARY_OP(mc.position, c, BinaryOperation.EQ_OP, refTmp)) - .reduceRight((c, r) => BINARY_OP(mc.position, c, BinaryOperation.OR_OP, r)) + .reduceRight(BINARY_OP(mc.position, _, BinaryOperation.OR_OP, _)) Right(makeIfCase(cond, blockWithNewVar, further)) case (p: CompositePattern, _) => - val pos = p.position - val newRef = p.caseType.fold(refTmp)(t => refTmp.copy(resultType = Some(caseType))) - val conditions = makeConditionsFromCompositePattern(p, newRef) - val cond = if (conditions.isEmpty) { - Expressions.TRUE(pos): Expressions.EXPR - } else { - conditions.reduceRight { (c, r) => - BINARY_OP(pos, c, BinaryOperation.AND_OP, r): Expressions.EXPR + val pos = p.position + val newRef = p.caseType.fold(refTmp)(_ => refTmp.copy(resultType = Some(caseType))) + makeConditionsFromCompositePattern(ctx, p, newRef) + .map { conditions => + val cond = + if (conditions.isEmpty) + Expressions.TRUE(pos) + else + conditions.reduceRight(BINARY_OP(pos, _, BinaryOperation.AND_OP, _)) + val checkingCond = + if (p.isInstanceOf[TuplePat]) { + val (resolvedTypes, size) = resolveTypesFromCompositePattern(p) + if (p.patternsWithFields.size == size) { + val typeChecks = + resolvedTypes + .map(t => Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(refTmp, Expressions.CONST_STRING(pos, t)))) + .reduceLeft[Expressions.EXPR] { case (c, r) => BINARY_OP(pos, c, BinaryOperation.OR_OP, r) } + BINARY_OP(pos, cond, BinaryOperation.AND_OP, typeChecks) + } else { + val size = Expressions.CONST_LONG(pos, p.patternsWithFields.size) + val getSize = Expressions.FUNCTION_CALL(pos, PART.VALID(pos, "size"), List(refTmp)) + val compareSize = BINARY_OP(pos, getSize, BinaryOperation.EQ_OP, size) + BINARY_OP(pos, cond, BinaryOperation.AND_OP, compareSize) + } + } else + cond + makeIfCase( + p.caseType.fold(checkingCond)(t => + BINARY_OP( + pos, + Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(refTmp, Expressions.CONST_STRING(pos, t.name))), + BinaryOperation.AND_OP, + Expressions.BLOCK(pos, Expressions.LET(pos, newRef.key, newRef, Some(caseType), true), checkingCond) + ) + ), + blockWithNewVar, + further + ) } - } - val checkingCond = - if (p.isInstanceOf[TuplePat]) { - val (resolvedTypes, size) = resolveTypesFromCompositePattern(p) - if (p.patternsWithFields.size == size) { - val typeChecks = - resolvedTypes - .map(t => Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(refTmp, Expressions.CONST_STRING(pos, t)))) - .reduceLeft[Expressions.EXPR] { case (c, r) => BINARY_OP(pos, c, BinaryOperation.OR_OP, r) } - BINARY_OP(pos, cond, BinaryOperation.AND_OP, typeChecks) - } else { - val size = Expressions.CONST_LONG(pos, p.patternsWithFields.size) - val getSize = Expressions.FUNCTION_CALL(pos, PART.VALID(pos, "size"), List(refTmp)) - val compareSize = BINARY_OP(pos, getSize, BinaryOperation.EQ_OP, size) - BINARY_OP(pos, cond, BinaryOperation.AND_OP, compareSize) - } - } else - cond - Right( - makeIfCase( - p.caseType.fold(checkingCond)(t => - BINARY_OP( - pos, - Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(refTmp, Expressions.CONST_STRING(pos, t.name))), - BinaryOperation.AND_OP, - Expressions.BLOCK(pos, Expressions.LET(pos, newRef.key, newRef, Some(caseType), true), checkingCond) - ) - ), - blockWithNewVar, - further - ) - ) } } } @@ -923,47 +776,56 @@ object ExpressionCompiler { Expressions.GETTER(pos, exp, field, checkObjectType = false) } - private def makeConditionsFromCompositePattern(p: CompositePattern, newRef: Expressions.REF): Seq[Expressions.EXPR] = - p.subpatterns collect { + private def makeConditionsFromCompositePattern( + ctx: CompilerContext, + p: CompositePattern, + newRef: Expressions.REF + ): Either[Generic, Seq[Expressions.EXPR]] = + p.subpatterns.traverse { case (pat @ TypedVar(_, Expressions.Union(types)), path) if types.nonEmpty => val pos = pat.position val v = mkGet(path, newRef, pos) - types - .map { + val r = types + .collect { case Expressions.Single(t, None) => - Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))): Expressions.EXPR + Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))) case Expressions.Single(PART.VALID(pos, Type.ListTypeName), Some(PART.VALID(_, Expressions.AnyType(_)))) => val t = PART.VALID(pos, "List[Any]") - Expressions.FUNCTION_CALL( - pos, - PART.VALID(pos, IsInstanceOf), - List(v, Expressions.CONST_STRING(pos, t)) - ): Expressions.EXPR - case _ => ??? - } - .reduceRight[Expressions.EXPR] { (c, r) => - BINARY_OP(pos, c, BinaryOperation.OR_OP, r) + Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))) } + .reduceRight[Expressions.EXPR](BINARY_OP(pos, _, BinaryOperation.OR_OP, _)) + Right(r) case (pat @ TypedVar(_, Expressions.Single(PART.VALID(_, Type.ListTypeName), Some(PART.VALID(_, Expressions.AnyType(_))))), path) => val pos = pat.position val v = mkGet(path, newRef, pos) val t = PART.VALID(pos, "List[Any]") - Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))): Expressions.EXPR + val r = Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))) + Right(r) case (pat @ TypedVar(_, Expressions.Single(t, None)), path) => val pos = pat.position val v = mkGet(path, newRef, pos) - Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))): Expressions.EXPR - case (TypedVar(_, Expressions.Single(_, _)), _) => ??? + val r = Expressions.FUNCTION_CALL(pos, PART.VALID(pos, IsInstanceOf), List(v, Expressions.CONST_STRING(pos, t))) + Right(r) case (pat @ ConstsPat(consts, _), path) => val pos = pat.position val v = mkGet(path, newRef, pos) consts - .map { c => - BINARY_OP(pos, c, BinaryOperation.EQ_OP, v) - } - .reduceRight[BINARY_OP] { (c, r) => - BINARY_OP(pos, c, BinaryOperation.OR_OP, r) + .traverse { + case const @ ( + _: Expressions.CONST_LONG | _: Expressions.CONST_STRING | _: Expressions.CONST_BYTESTR | _: Expressions.TRUE | + _: Expressions.FALSE | _: Expressions.REF + ) => + Right(BINARY_OP(pos, const, BinaryOperation.EQ_OP, v)) + case func @ Expressions.FUNCTION_CALL(pos, PART.VALID(_, name), _, _, _) if ctx.predefTypes.contains(name) => + Right(BINARY_OP(pos, func, BinaryOperation.EQ_OP, v)) + case expr if version < V8 => + Right(BINARY_OP(pos, expr, BinaryOperation.EQ_OP, v)) + case expr => + Left(Generic(expr.position.start, expr.position.end, "Only constant value could be matched with object field")) } + .map(_.reduceRight(BINARY_OP(pos, _, BinaryOperation.OR_OP, _))) + case _ => + Right(Expressions.TRUE(newRef.position)) } private def resolveTypesFromCompositePattern(p: CompositePattern): (Seq[PART[String]], Int) = { @@ -1070,27 +932,102 @@ object ExpressionCompiler { types.toList .traverse(handleCompositeType(pos, _, expectedType, varName)) .map(types => TUPLE(types)) - case Expressions.AnyType(pos) => (ANY: FINAL).pure[CompileM] + case _: Expressions.AnyType => + (ANY: FINAL).pure[CompileM] } - def handlePart[T](part: PART[T]): CompileM[T] = part match { + protected def handlePart[T](part: PART[T]): CompileM[T] = part match { case PART.VALID(_, x) => x.pure[CompileM] case PART.INVALID(p, message) => raiseError(Generic(p.start, p.end, message)) } +} +object ExpressionCompiler { implicit class RichBoolean(val b: Boolean) extends AnyVal { final def toOption[A](a: => A): Option[A] = if (b) Some(a) else None } - def apply(c: CompilerContext, expr: Expressions.EXPR, allowIllFormedStrings: Boolean = false): Either[String, (EXPR, FINAL)] = - applyWithCtx(c, expr, allowIllFormedStrings).map(r => (r._2, r._3)) + def compileWithParseResult( + input: String, + offset: LibrariesOffset, + ctx: CompilerContext, + version: StdLibVersion, + saveExprContext: Boolean = true + ): Either[(String, Int, Int), (EXPR, Expressions.SCRIPT, Iterable[CompilationError])] = + new Parser(version)(offset) + .parseExpressionWithErrorRecovery(input) + .flatMap { case (parseResult, removedCharPosOpt) => + new ExpressionCompiler(version) + .compileExprWithCtx(parseResult.expr, saveExprContext, allowIllFormedStrings = false) + .run(ctx) + .value + ._2 + .map { compRes => + val errorList = + compRes.errors ++ + (if (compRes.t equivalent BOOLEAN) Nil else List(Generic(0, 0, "Script should return boolean"))) ++ + (if (removedCharPosOpt.isEmpty) + Nil + else + List( + Generic( + removedCharPosOpt.get.start, + removedCharPosOpt.get.end, + "Parsing failed. Some chars was removed as result of recovery process." + ) + )) + (compRes.expr, parseResult.copy(expr = compRes.parseNodeExpr), errorList) + } + .leftMap(e => (s"Compilation failed: ${Show[CompilationError].show(e)}", e.start, e.end)) + } + + def compile( + input: String, + offset: LibrariesOffset, + ctx: CompilerContext, + version: StdLibVersion, + allowIllFormedStrings: Boolean = false + ): Either[String, (EXPR, FINAL)] = { + val parser = new Parser(version)(offset) + parser.parseExpr(input) match { + case fastparse.Parsed.Success(expr, _) => apply(ctx, version, expr, allowIllFormedStrings) + case f: fastparse.Parsed.Failure => Left(parser.toString(input, f)) + } + } + + def compileBoolean(input: String, offset: LibrariesOffset, ctx: CompilerContext, version: StdLibVersion): Either[String, EXPR] = { + compile(input, offset, ctx, version).flatMap { + case (expr, BOOLEAN) => Right(expr) + case _ => Left("Script should return boolean") + } + } + + def compileUntyped(input: String, offset: LibrariesOffset, ctx: CompilerContext, version: StdLibVersion): Either[String, EXPR] = { + compile(input, offset, ctx, version) + .map { case (expr, _) => expr } + } + + def compileDecls(input: String, offset: LibrariesOffset, ctx: CompilerContext, version: StdLibVersion): Either[String, EXPR] = { + val adjustedDecls = s"$input\n${GlobalValNames.Unit}" + compileUntyped(adjustedDecls, offset, ctx, version) + } + + def apply( + c: CompilerContext, + version: StdLibVersion, + expr: Expressions.EXPR, + allowIllFormedStrings: Boolean = false + ): Either[String, (EXPR, FINAL)] = + applyWithCtx(c, version, expr, allowIllFormedStrings).map(r => (r._2, r._3)) def applyWithCtx( c: CompilerContext, + version: StdLibVersion, expr: Expressions.EXPR, allowIllFormedStrings: Boolean = false ): Either[String, (CompilerContext, EXPR, FINAL)] = - compileExprWithCtx(expr, allowIllFormedStrings = allowIllFormedStrings) + new ExpressionCompiler(version) + .compileExprWithCtx(expr, allowIllFormedStrings = allowIllFormedStrings) .run(c) .value ._2 diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/TypeCast.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/TypeCast.scala index 38b6207c034..eea379da395 100644 --- a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/TypeCast.scala +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/compiler/TypeCast.scala @@ -1,7 +1,6 @@ package com.wavesplatform.lang.v1.compiler import com.wavesplatform.common.utils.* import com.wavesplatform.lang.v1.compiler.CompilationError.{GenericFunctionNotFound, TypeCastAllowedOnlyForGenericList} -import com.wavesplatform.lang.v1.compiler.ExpressionCompiler.CompilationStepResultExpr import com.wavesplatform.lang.v1.compiler.Terms.* import com.wavesplatform.lang.v1.compiler.Types.* import com.wavesplatform.lang.v1.evaluator.ctx.impl.GlobalValNames diff --git a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/parser/Parser.scala b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/parser/Parser.scala index 40cbc52a390..0a61ef2788d 100644 --- a/lang/shared/src/main/scala/com/wavesplatform/lang/v1/parser/Parser.scala +++ b/lang/shared/src/main/scala/com/wavesplatform/lang/v1/parser/Parser.scala @@ -309,10 +309,11 @@ class Parser(stdLibVersion: StdLibVersion)(implicit offset: LibrariesOffset) { }) ).?.map(_.getOrElse(Union(Seq()))) + val objPatMin = if (stdLibVersion >= V8) 1 else 0 def pattern(implicit c: fastparse.P[Any]): P[Pattern] = (varDefP ~ comment ~ typesDefP).map { case (v, t) => TypedVar(v, t) } | (Index ~ "(" ~ pattern.rep(min = 2, sep = ",") ~/ ")" ~ Index).map(p => TuplePat(p._2, Pos(p._1, p._3))) | - (Index ~ anyVarName() ~ "(" ~ (anyVarName() ~ "=" ~ pattern).rep(sep = ",") ~ ")" ~ Index) + (Index ~ anyVarName() ~ "(" ~ (anyVarName() ~ "=" ~ pattern).rep(min = objPatMin, sep = ",") ~ ")" ~ Index) .map(p => ObjPat(p._3.map(kp => (PART.toOption(kp._1).get, kp._2)).toMap, Single(p._2, None), Pos(p._1, p._4))) | (Index ~ baseExpr.rep(min = 1, sep = "|") ~ Index).map(p => ConstsPat(p._2, Pos(p._1, p._3))) diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/DecompilerTest.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/DecompilerTest.scala index 5874a51e302..d78e73802aa 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/DecompilerTest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/DecompilerTest.scala @@ -561,7 +561,7 @@ class DecompilerTest extends PropSpec { def compileExpr(code: String, v: StdLibVersion = V3): Either[String, (EXPR, TYPE)] = { val untyped = Parser.parseExpr(code).get.value - val typed = ExpressionCompiler(getTestContext(v).compilerContext, untyped) + val typed = ExpressionCompiler(getTestContext(v).compilerContext, v, untyped) typed } diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ErrorTest.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ErrorTest.scala index 34180e1189f..e3c17978cc7 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ErrorTest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ErrorTest.scala @@ -3,7 +3,7 @@ package com.wavesplatform.lang.compiler import com.wavesplatform.common.state.ByteStr import com.wavesplatform.lang.Common.multiplierFunction import com.wavesplatform.lang.contract.DApp -import com.wavesplatform.lang.directives.values.V5 +import com.wavesplatform.lang.directives.values.{V3, V5} import com.wavesplatform.lang.v1.compiler.{ExpressionCompiler, TestCompiler} import com.wavesplatform.lang.v1.evaluator.ctx.impl.GlobalValNames import com.wavesplatform.lang.v1.parser.BinaryOperation.SUM_OP @@ -158,7 +158,7 @@ class ErrorTest extends PropSpec { private def errorTests(exprs: ((String, String), Expressions.EXPR)*): Unit = exprs.foreach { case ((label, error), input) => property(s"Error: $label") { - ExpressionCompiler(compilerContext, input) should produce(error) + ExpressionCompiler(compilerContext, V3, input) should produce(error) } } diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ExpressionCompilerV1Test.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ExpressionCompilerV1Test.scala index 871e4ba353f..db285b5f704 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ExpressionCompilerV1Test.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ExpressionCompilerV1Test.scala @@ -35,7 +35,7 @@ class ExpressionCompilerV1Test extends PropSpec { property("should infer generic function return type") { import com.wavesplatform.lang.v1.parser.Expressions.* - val v = ExpressionCompiler(compilerContext, FUNCTION_CALL(AnyPos, PART.VALID(AnyPos, idT.name), List(CONST_LONG(AnyPos, 1)))).explicitGet() + val v = ExpressionCompiler(compilerContext, V3, FUNCTION_CALL(AnyPos, PART.VALID(AnyPos, idT.name), List(CONST_LONG(AnyPos, 1)))).explicitGet() v._2 shouldBe LONG } @@ -44,6 +44,7 @@ class ExpressionCompilerV1Test extends PropSpec { val v = ExpressionCompiler( compilerContext, + V3, FUNCTION_CALL( AnyPos, PART.VALID(AnyPos, "getElement"), @@ -59,7 +60,7 @@ class ExpressionCompilerV1Test extends PropSpec { } val expectedResult = Right(LONG) - ExpressionCompiler(compilerContext, expr).map(_._2) match { + ExpressionCompiler(compilerContext, V3, expr).map(_._2) match { case Right(x) => Right(x) shouldBe expectedResult case e @ Left(_) => e shouldBe expectedResult } @@ -68,11 +69,11 @@ class ExpressionCompilerV1Test extends PropSpec { property("string limit") { val maxString = "a" * Terms.DataEntryValueMax val expr = Parser.parseExpr(s""" "$maxString" """).get.value - ExpressionCompiler(compilerContext, expr).map(_._1) shouldBe CONST_STRING(maxString) + ExpressionCompiler(compilerContext, V3, expr).map(_._1) shouldBe CONST_STRING(maxString) val tooBigString = maxString + "a" val expr2 = Parser.parseExpr(s""" "$tooBigString" """).get.value - ExpressionCompiler(compilerContext, expr2) should produce("String size=32768 exceeds 32767 bytes") + ExpressionCompiler(compilerContext, V3, expr2) should produce("String size=32768 exceeds 32767 bytes") } @@ -94,8 +95,8 @@ class ExpressionCompilerV1Test extends PropSpec { """.stripMargin Parser.parseExpr(script).get.value } - ExpressionCompiler(compilerContext, funcExpr) should produce(s"Function '$tooLongName' size = 256 bytes exceeds 255") - ExpressionCompiler(compilerContext, letExpr) should produce(s"Let '$tooLongName' size = 256 bytes exceeds 255") + ExpressionCompiler(compilerContext, V3, funcExpr) should produce(s"Function '$tooLongName' size = 256 bytes exceeds 255") + ExpressionCompiler(compilerContext, V3, letExpr) should produce(s"Let '$tooLongName' size = 256 bytes exceeds 255") } @@ -117,18 +118,18 @@ class ExpressionCompilerV1Test extends PropSpec { """.stripMargin Parser.parseExpr(script).get.value } - ExpressionCompiler(compilerContext, funcExpr) shouldBe Symbol("right") - ExpressionCompiler(compilerContext, letExpr) shouldBe Symbol("right") + ExpressionCompiler(compilerContext, V3, funcExpr) shouldBe Symbol("right") + ExpressionCompiler(compilerContext, V3, letExpr) shouldBe Symbol("right") } property("tuple type checks") { val script = """ ("a", true, 123, base58'aaaa')._3 == true """ val expr = Parser.parseExpr(script).get.value - ExpressionCompiler(compilerContextV4, expr) should produce("Can't match inferred types of T over Int, Boolean") + ExpressionCompiler(compilerContextV4, V4, expr) should produce("Can't match inferred types of T over Int, Boolean") val script2 = """ ("a", true, 123, base58'aaaa') == ("a", true, "b", base58'aaaa') """ val expr2 = Parser.parseExpr(script2).get.value - ExpressionCompiler(compilerContextV4, expr2) should produce( + ExpressionCompiler(compilerContextV4, V4, expr2) should produce( "Can't match inferred types of T over (String, Boolean, Int, ByteVector), (String, Boolean, String, ByteVector)" ) @@ -146,7 +147,7 @@ class ExpressionCompilerV1Test extends PropSpec { | (((v, q), (true, v)), q) == (((1, true), (q, q)), v) """.stripMargin val expr3 = Parser.parseExpr(script3).get.value - ExpressionCompiler(compilerContextV4, expr3) shouldBe Symbol("right") + ExpressionCompiler(compilerContextV4, V4, expr3) shouldBe Symbol("right") val script4 = """ @@ -156,7 +157,7 @@ class ExpressionCompilerV1Test extends PropSpec { | (((v, q), (true, v)), q) == (((1, true), (v, q)), v) """.stripMargin val expr4 = Parser.parseExpr(script4).get.value - ExpressionCompiler(compilerContextV4, expr4) should produce( + ExpressionCompiler(compilerContextV4, V4, expr4) should produce( "Can't match inferred types of T over " + "(((Int|String, Boolean|Int|String), (Boolean, Int|String)), Boolean|Int|String), " + "(((Int, Boolean), (Int|String, Boolean|Int|String)), Int|String) in 102-154" @@ -171,7 +172,7 @@ class ExpressionCompilerV1Test extends PropSpec { | a1 == b1 """.stripMargin val expr = Parser.parseExpr(script).get.value - ExpressionCompiler(compilerContextV4, expr) shouldBe Symbol("right") + ExpressionCompiler(compilerContextV4, V4, expr) shouldBe Symbol("right") } property("function with tuple args") { @@ -189,7 +190,7 @@ class ExpressionCompilerV1Test extends PropSpec { | """.stripMargin val expr = Parser.parseExpr(script).get.value - ExpressionCompiler(compilerContextV4, expr) shouldBe Symbol("right") + ExpressionCompiler(compilerContextV4, V4, expr) shouldBe Symbol("right") val script2 = """ @@ -200,7 +201,7 @@ class ExpressionCompilerV1Test extends PropSpec { | """.stripMargin val expr2 = Parser.parseExpr(script2).get.value - ExpressionCompiler(compilerContextV4, expr2) should produce( + ExpressionCompiler(compilerContextV4, V4, expr2) should produce( "Non-matching types: expected: (String, Boolean), actual: Boolean in 69-86" ) @@ -213,7 +214,7 @@ class ExpressionCompilerV1Test extends PropSpec { | """.stripMargin val expr3 = Parser.parseExpr(script3).get.value - ExpressionCompiler(compilerContextV4, expr3) should produce( + ExpressionCompiler(compilerContextV4, V4, expr3) should produce( "Non-matching types: expected: ((Int, String), Boolean)|(Int, String, Boolean), actual: (Int, String) in 73-84" ) } @@ -237,7 +238,7 @@ class ExpressionCompilerV1Test extends PropSpec { | """.stripMargin val expr = Parser.parseExpr(script).get.value - ExpressionCompiler(compilerContextV4, expr) shouldBe Symbol("right") + ExpressionCompiler(compilerContextV4, V4, expr) shouldBe Symbol("right") val script2 = """ @@ -249,7 +250,7 @@ class ExpressionCompilerV1Test extends PropSpec { | true """.stripMargin val expr2 = Parser.parseExpr(script2).get.value - ExpressionCompiler(compilerContextV4, expr2) should produce( + ExpressionCompiler(compilerContextV4, V4, expr2) should produce( "Matching not exhaustive: " + "possibleTypes are (Int, String), ((Boolean, Int), ByteVector), " + "while matched are (Int, String), (Boolean, Int, ByteVector)" @@ -274,7 +275,7 @@ class ExpressionCompilerV1Test extends PropSpec { | """.stripMargin val expr3 = Parser.parseExpr(script3).get.value - ExpressionCompiler(compilerContextV4, expr3) shouldBe Symbol("right") + ExpressionCompiler(compilerContextV4, V4, expr3) shouldBe Symbol("right") val script4 = """ @@ -288,7 +289,7 @@ class ExpressionCompilerV1Test extends PropSpec { | """.stripMargin val expr4 = Parser.parseExpr(script4).get.value - ExpressionCompiler(compilerContextV4, expr4) should produce( + ExpressionCompiler(compilerContextV4, V4, expr4) should produce( "Matching not exhaustive: " + "possibleTypes are (Int, String), (Boolean, String), (ByteVector, Boolean, (String, (Int, Boolean))|Int), " + "while matched are (Boolean|Int, String), (ByteVector, Boolean, Int), (ByteVector, Boolean, (String, Int, Boolean)) " + @@ -303,7 +304,7 @@ class ExpressionCompilerV1Test extends PropSpec { | } """.stripMargin val expr5 = Parser.parseExpr(script5).get.value - ExpressionCompiler(compilerContextV4, expr5) shouldBe Symbol("right") + ExpressionCompiler(compilerContextV4, V4, expr5) shouldBe Symbol("right") } property("JS API compile limit exceeding error") { @@ -330,6 +331,7 @@ class ExpressionCompilerV1Test extends PropSpec { def checkExtract(version: StdLibVersion) = ExpressionCompiler( getTestContext(version).compilerContext, + version, Parser.parseExpr(" extract(1) ").get.value ) @@ -358,7 +360,7 @@ class ExpressionCompilerV1Test extends PropSpec { DirectiveDictionary[StdLibVersion].all .foreach { version => - val result = ExpressionCompiler(getTestContext(version).compilerContext, Parser.parseExpr(expr).get.value) + val result = ExpressionCompiler(getTestContext(version).compilerContext, version, Parser.parseExpr(expr).get.value) if (version >= V4) result shouldBe Symbol("right") else @@ -390,7 +392,7 @@ class ExpressionCompilerV1Test extends PropSpec { version <- DirectiveDictionary[StdLibVersion].all scriptType <- DirectiveDictionary[ScriptType].all } { - val result = ExpressionCompiler(getTestContext(version, scriptType).compilerContext, expr(version, scriptType)) + val result = ExpressionCompiler(getTestContext(version, scriptType).compilerContext, version, expr(version, scriptType)) if (version < V5 || scriptType != Account) result.swap.getOrElse(???).split("Can't find a function").length shouldBe 9 else @@ -415,7 +417,7 @@ class ExpressionCompilerV1Test extends PropSpec { DirectiveDictionary[StdLibVersion].all .foreach { version => - val result = ExpressionCompiler(getTestContext(version).compilerContext, expr(version)) + val result = ExpressionCompiler(getTestContext(version).compilerContext, version, expr(version)) if (version < V5) result should produce( "Compilation failed: [" + @@ -446,7 +448,7 @@ class ExpressionCompilerV1Test extends PropSpec { DirectiveDictionary[StdLibVersion].all .foreach { version => - val result = ExpressionCompiler(getTestContext(version).compilerContext, expr(version)) + val result = ExpressionCompiler(getTestContext(version).compilerContext, version, expr(version)) if (version < V5) result shouldBe Symbol("right") else @@ -484,7 +486,7 @@ class ExpressionCompilerV1Test extends PropSpec { DirectiveDictionary[StdLibVersion].all .foreach { version => - ExpressionCompiler(getTestContext(version).compilerContext, expr(version)) shouldBe Symbol("right") + ExpressionCompiler(getTestContext(version).compilerContext, version, expr(version)) shouldBe Symbol("right") } } @@ -512,7 +514,7 @@ class ExpressionCompilerV1Test extends PropSpec { DirectiveDictionary[StdLibVersion].all .foreach { version => - val result = ExpressionCompiler(getTestContext(version).compilerContext, expr(version)) + val result = ExpressionCompiler(getTestContext(version).compilerContext, version, expr(version)) if (version < V5) result shouldBe Symbol("right") else { @@ -542,7 +544,7 @@ class ExpressionCompilerV1Test extends PropSpec { DirectiveDictionary[StdLibVersion].all .filter(_ >= V3) .foreach { version => - val result = ExpressionCompiler(getTestContext(version).compilerContext, expr(version)) + val result = ExpressionCompiler(getTestContext(version).compilerContext, version, expr(version)) val error = result.swap.getOrElse(???) error should include("Can't find a function 'invoke'") error should include("Can't find a function 'reentrantInvoke'") @@ -1149,7 +1151,7 @@ class ExpressionCompilerV1Test extends PropSpec { propertyName: String )(expr: Expressions.EXPR, expectedResult: Either[String, (EXPR, TYPE)] => org.scalatest.compatible.Assertion, ctx: CompilerContext): Unit = property(propertyName) { - val res = compiler.ExpressionCompiler(ctx, expr) + val res = compiler.ExpressionCompiler(ctx, V3, expr) expectedResult(res) } diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ScriptPreprocessorTest.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ScriptPreprocessorTest.scala index 8ce9d375f90..b953caabf5f 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ScriptPreprocessorTest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/compiler/ScriptPreprocessorTest.scala @@ -6,7 +6,6 @@ import cats.kernel.Monoid import com.wavesplatform.lang.directives.values.V3 import com.wavesplatform.lang.directives.{Directive, DirectiveParser} import com.wavesplatform.lang.script.ScriptPreprocessor -import com.wavesplatform.lang.v1.CTX import com.wavesplatform.lang.v1.compiler.ExpressionCompiler import com.wavesplatform.lang.v1.compiler.Terms.{CONST_BOOLEAN, EVALUATED} import com.wavesplatform.lang.v1.evaluator.Contextful.NoContext @@ -29,9 +28,9 @@ class ScriptPreprocessorTest extends PropSpec with ScriptGenParser { } yield r private def eval(code: String): Either[String, EVALUATED] = { - val untyped = Parser.parseExpr(code).get.value - val ctx: CTX[NoContext] = Monoid.combineAll(Seq(PureContext.build(V3, useNewPowPrecision = true))) - val typed = ExpressionCompiler(ctx.compilerContext, untyped) + val untyped = Parser.parseExpr(code).get.value + val ctx = Monoid.combineAll(Seq(PureContext.build(V3, useNewPowPrecision = true))) + val typed = ExpressionCompiler(ctx.compilerContext, V3, untyped) typed.flatMap(v => evaluator[EVALUATED](ctx.evaluationContext, v._1).leftMap(_.toString)) } diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/estimator/ScriptEstimatorTestBase.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/estimator/ScriptEstimatorTestBase.scala index 5eae03001d6..ed72f994c1f 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/estimator/ScriptEstimatorTestBase.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/estimator/ScriptEstimatorTestBase.scala @@ -57,7 +57,7 @@ class ScriptEstimatorTestBase(estimators: ScriptEstimator*) extends PropSpec { protected def compile(code: String)(implicit version: StdLibVersion): EXPR = { val untyped = Parser.parseExpr(code).get.value - ExpressionCompiler(ctx.compilerContext, untyped).map(_._1).explicitGet() + ExpressionCompiler(ctx.compilerContext, version, untyped).map(_._1).explicitGet() } protected def estimate( diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorSpec.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorSpec.scala index 8a56d49c8e4..059eb1f2b52 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorSpec.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorSpec.scala @@ -1,18 +1,18 @@ package com.wavesplatform.lang.evaluator -import cats.implicits.* import cats.Id +import cats.implicits.* import com.wavesplatform.common.utils.EitherExt2 -import com.wavesplatform.lang.{Common, ExecutionError} -import com.wavesplatform.lang.directives.{DirectiveDictionary, DirectiveSet} import com.wavesplatform.lang.directives.values.* +import com.wavesplatform.lang.directives.{DirectiveDictionary, DirectiveSet} import com.wavesplatform.lang.utils.lazyContexts import com.wavesplatform.lang.v1.compiler.ExpressionCompiler import com.wavesplatform.lang.v1.compiler.Terms.{EVALUATED, EXPR} import com.wavesplatform.lang.v1.evaluator.ContractEvaluator.LogExtraInfo import com.wavesplatform.lang.v1.evaluator.{EvaluatorV2, Log} -import com.wavesplatform.lang.v1.parser.Parser +import com.wavesplatform.lang.v1.parser.Parser.LibrariesOffset.NoLibraries import com.wavesplatform.lang.v1.testing.ScriptGen +import com.wavesplatform.lang.{Common, ExecutionError} import com.wavesplatform.test.PropSpec import org.scalatest.Inside import org.scalatest.exceptions.TestFailedException @@ -78,9 +78,8 @@ abstract class EvaluatorSpec extends PropSpec with ScriptGen with Inside { EvaluatorV2.applyCompleted(evalCtx, expr, LogExtraInfo(), version, correctFunctionCallScope = true, newMode = true, enableExecutionLog = false) } - def compile(code: String, version: StdLibVersion): Either[String, EXPR] = { - val ctx = lazyContexts((DirectiveSet(version, Account, Expression).explicitGet(), true, true)).value() - val parsed = Parser.parseExpr(code).get.value - ExpressionCompiler(ctx.compilerContext, parsed, allowIllFormedStrings = true).map(_._1) + private def compile(code: String, version: StdLibVersion): Either[String, EXPR] = { + val ctx = lazyContexts((DirectiveSet(version, Account, Expression).explicitGet(), true, true)).value() + ExpressionCompiler.compile(code, NoLibraries, ctx.compilerContext, version, allowIllFormedStrings = true).map(_._1) } } diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV1V2Test.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV1V2Test.scala index 3c0dbc8745d..f040ca89b32 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV1V2Test.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV1V2Test.scala @@ -880,7 +880,7 @@ class EvaluatorV1V2Test extends PropSpec with EitherValues { evalPure[EVALUATED]( context.evaluationContext[Id], ExpressionCompiler - .apply(context.compilerContext, xs) + .apply(context.compilerContext, version, xs) .explicitGet() ._1 ) diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV2Test.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV2Test.scala index cd7b4ff3240..bc20e70fafc 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV2Test.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/EvaluatorV2Test.scala @@ -67,7 +67,7 @@ class EvaluatorV2Test extends PropSpec with Inside { private def compile(script: String): EXPR = { val parsed = Parser.parseExpr(script).get.value - ExpressionCompiler(ctx.compilerContext, parsed).explicitGet()._1 + ExpressionCompiler(ctx.compilerContext, version, parsed).explicitGet()._1 } property("multiple lets by step") { diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/NestedPatternsTest.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/NestedPatternsTest.scala index 080aded4dfe..b33b7575958 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/NestedPatternsTest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/evaluator/NestedPatternsTest.scala @@ -1,6 +1,7 @@ package com.wavesplatform.lang.evaluator -import com.wavesplatform.lang.directives.values.{StdLibVersion, V6} +import com.wavesplatform.lang.directives.values.{StdLibVersion, V6, V7, V8} import com.wavesplatform.lang.v1.compiler.Terms.CONST_BOOLEAN +import com.wavesplatform.test.produce class NestedPatternsTest extends EvaluatorSpec { implicit val v: StdLibVersion = V6 @@ -126,4 +127,58 @@ class NestedPatternsTest extends EvaluatorSpec { """.stripMargin ) shouldBe Right(CONST_BOOLEAN(true)) } + + property("restrictions from V8") { + val script1 = + s""" + | match Lease(Address(base58''), 1, 1) { + | case Lease(nonce = f()) => true + | case _ => false + | } + """.stripMargin + eval(script1)(V7, checkNext = false) shouldBe Right(CONST_BOOLEAN(true)) + eval(script1)(V8) should produce("Only constant value could be matched with object field in 63-66") + + val script2 = + s""" + | func f() = 777 + | match Lease(Address(base58''), 1, 1) { + | case Lease(nonce = f()) => true + | case _ => false + | } + """.stripMargin + eval(script2)(V7, checkNext = false) shouldBe Right(CONST_BOOLEAN(true)) + eval(script2)(V8) should produce("Only constant value could be matched with object field in 79-82") + + val script3 = + s""" + | func f() = 777 + | match Lease(Address(base58''), 1, 1) { + | case Lease(nonce = {f()}) => true + | case _ => false + | } + """.stripMargin + eval(script3)(V7, checkNext = false) shouldBe Right(CONST_BOOLEAN(false)) + eval(script3)(V8) should produce("Only constant value could be matched with object field in 80-83") + + val script4 = + s""" + | match Lease(Address(base58''), 1, 1) { + | case Lease(nonce = {1 + 2}) => true + | case _ => false + | } + """.stripMargin + eval(script4)(V7, checkNext = false) shouldBe Right(CONST_BOOLEAN(false)) + eval(script4)(V8) should produce("Only constant value could be matched with object field in 64-69") + + val script5 = + s""" + | match Lease(Address(base58''), 1, 1) { + | case Lease(nonce = {if (true) then 2 else 1}) => true + | case _ => false + | } + """.stripMargin + eval(script5)(V7, checkNext = false) shouldBe Right(CONST_BOOLEAN(false)) + eval(script5)(V8) should produce("Only constant value could be matched with object field in 64-87") + } } diff --git a/lang/tests/src/test/scala/com/wavesplatform/lang/serde/SerdeTest.scala b/lang/tests/src/test/scala/com/wavesplatform/lang/serde/SerdeTest.scala index 0d9c56bd73e..8de49007e18 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/lang/serde/SerdeTest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/lang/serde/SerdeTest.scala @@ -177,7 +177,7 @@ class SerdeTest extends FreeSpec { } private def roundTripTest(untypedExpr: Expressions.EXPR): Unit = { - val typedExpr = ExpressionCompiler(PureContext.build(V1, useNewPowPrecision = true).compilerContext, untypedExpr).map(_._1).explicitGet() + val typedExpr = ExpressionCompiler(PureContext.build(V1, useNewPowPrecision = true).compilerContext, V1, untypedExpr).map(_._1).explicitGet() roundTripTest(typedExpr) } diff --git a/lang/tests/src/test/scala/com/wavesplatform/utils/MerkleTest.scala b/lang/tests/src/test/scala/com/wavesplatform/utils/MerkleTest.scala index 23c3ae99426..95039717194 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/utils/MerkleTest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/utils/MerkleTest.scala @@ -109,7 +109,7 @@ class MerkleTest extends PropSpec { val untyped = Parser.parseExpr(code).get.value val ctx = lazyContexts((DirectiveSet(version, Account, Expression).explicitGet(), true, true))() val evalCtx = ctx.evaluationContext[Id](Common.emptyBlockchainEnvironment()) - val typed = ExpressionCompiler(ctx.compilerContext, untyped) + val typed = ExpressionCompiler(ctx.compilerContext, V3, untyped) typed.flatMap(v => EvaluatorV2.applyCompleted(evalCtx, v._1, LogExtraInfo(), version, true, true, false)._3.leftMap(_.toString)) } diff --git a/lang/tests/src/test/scala/com/wavesplatform/utils/RSATest.scala b/lang/tests/src/test/scala/com/wavesplatform/utils/RSATest.scala index 0f15a68e63e..570ff7ad77b 100644 --- a/lang/tests/src/test/scala/com/wavesplatform/utils/RSATest.scala +++ b/lang/tests/src/test/scala/com/wavesplatform/utils/RSATest.scala @@ -337,7 +337,7 @@ class RSATest extends PropSpec with BeforeAndAfterAll { private def eval[T <: EVALUATED](code: String, ctx: CTX[NoContext] = PureContext.build(V4, useNewPowPrecision = true) |+| CryptoContext.build(Global, V4)): Either[String, T] = { val untyped = Parser.parseExpr(code).get.value - val typed = ExpressionCompiler(ctx.compilerContext, untyped) + val typed = ExpressionCompiler(ctx.compilerContext, V4, untyped) typed.flatMap(v => evaluator[T](ctx.evaluationContext, v._1).leftMap(_.message)) } diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/AssetTransactionsDiffTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/AssetTransactionsDiffTest.scala index 1994f86aee0..ef12359b942 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/AssetTransactionsDiffTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/AssetTransactionsDiffTest.scala @@ -218,7 +218,7 @@ class AssetTransactionsDiffTest extends PropSpec with BlocksTransactionsHelpers private def createScript(code: String, version: StdLibVersion) = { val Parsed.Success(expr, _) = Parser.parseExpr(code).get - ExprScript(version, ExpressionCompiler(compilerContext(version, Expression, isAssetScript = false), expr).explicitGet()._1).explicitGet() + ExprScript(version, ExpressionCompiler(compilerContext(version, Expression, isAssetScript = false), version, expr).explicitGet()._1).explicitGet() } def genesisIssueTransferReissue( diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/TransactionValidationErrorPrintTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/TransactionValidationErrorPrintTest.scala index ef67f40a402..eb9982bd253 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/TransactionValidationErrorPrintTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/TransactionValidationErrorPrintTest.scala @@ -53,11 +53,11 @@ class TransactionValidationErrorPrintTest extends PropSpec with Inside with With val untypedScript = Parser.parseExpr(assetScript).get.value - val typedScript = ExprScript(V6, ExpressionCompiler(compilerContext(V6, Expression, isAssetScript = false), untypedScript).explicitGet()._1) + val typedScript = ExprScript(V6, ExpressionCompiler(compilerContext(V6, Expression, isAssetScript = false), V6, untypedScript).explicitGet()._1) .explicitGet() val preTypedScript = - ExprScript(V6, ExpressionCompiler(compilerContext(V6, Expression, isAssetScript = false), Parser.parseExpr("true").get.value).explicitGet()._1) + ExprScript(V6, ExpressionCompiler(compilerContext(V6, Expression, isAssetScript = false), V6, Parser.parseExpr("true").get.value).explicitGet()._1) .explicitGet() val seed = Address.fromString("3MydsP4UeQdGwBq7yDbMvf9MzfB2pxFoUKU").explicitGet() diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/SmartAssetEvalTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/SmartAssetEvalTest.scala index 4fa3b295680..2c3cea3ec0e 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/SmartAssetEvalTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/SmartAssetEvalTest.scala @@ -31,7 +31,7 @@ class SmartAssetEvalTest extends PropSpec with WithState { """.stripMargin val parsedEmptyScript = Parser.parseExpr(emptyScript).get.value val emptyExprScript = - ExprScript(V3, ExpressionCompiler(compilerContext(V3, Expression, isAssetScript = true), parsedEmptyScript).explicitGet()._1) + ExprScript(V3, ExpressionCompiler(compilerContext(V3, Expression, isAssetScript = true), V3, parsedEmptyScript).explicitGet()._1) .explicitGet() val issue = TxHelpers.issue(firstAcc, 100, script = Some(emptyExprScript), reissuable = false) val asset = IssuedAsset(issue.id()) @@ -50,7 +50,7 @@ class SmartAssetEvalTest extends PropSpec with WithState { | """.stripMargin val untypedScript = Parser.parseExpr(assetScript).get.value - val typedScript = ExprScript(V3, ExpressionCompiler(compilerContext(V3, Expression, isAssetScript = true), untypedScript).explicitGet()._1) + val typedScript = ExprScript(V3, ExpressionCompiler(compilerContext(V3, Expression, isAssetScript = true), V3, untypedScript).explicitGet()._1) .explicitGet() val setAssetScript = TxHelpers.setAssetScript(firstAcc, asset, typedScript) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/performance/SigVerifyPerformanceTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/performance/SigVerifyPerformanceTest.scala index 1d72d4f469b..72a48d31fcd 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/performance/SigVerifyPerformanceTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/performance/SigVerifyPerformanceTest.scala @@ -34,7 +34,7 @@ class SigVerifyPerformanceTest extends PropSpec with WithState { ignore("parallel native signature verification vs sequential scripted signature verification") { val textScript = "sigVerify(tx.bodyBytes,tx.proofs[0],tx.senderPk)" val untypedScript = Parser.parseExpr(textScript).get.value - val typedScript = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untypedScript).explicitGet()._1 + val typedScript = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untypedScript).explicitGet()._1 val (gen, setScript, transfers, scriptTransfers) = differentTransfers(typedScript) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ObsoleteTransactionBindingsTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ObsoleteTransactionBindingsTest.scala index f34cf5f73d9..dd3690471ac 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ObsoleteTransactionBindingsTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ObsoleteTransactionBindingsTest.scala @@ -74,7 +74,7 @@ class ObsoleteTransactionBindingsTest extends PropSpec with WithState { recipients.map { recipient => val payment = TxHelpers.payment(master, recipient.toAddress, ENOUGH_AMT * 2) val untypedScript = Parser.parseExpr(script(genesis, payment)).get.value - val typedScript = ExprScript(ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untypedScript).explicitGet()._1) + val typedScript = ExprScript(ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untypedScript).explicitGet()._1) .explicitGet() val setScriptTransaction = TxHelpers.setScript(recipient, typedScript) val nextTransfer = TxHelpers.transfer(recipient, master.toAddress) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ScriptVersionsTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ScriptVersionsTest.scala index 0b52b3541c8..445763153b3 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ScriptVersionsTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/ScriptVersionsTest.scala @@ -30,7 +30,7 @@ class ScriptVersionsTest extends FreeSpec { ): Either[String, EVALUATED] = { val expr = Parser.parseExpr(script).get.value for { - compileResult <- ExpressionCompiler(compilerContext(version, Expression, isAssetScript = false), expr) + compileResult <- ExpressionCompiler(compilerContext(version, Expression, isAssetScript = false), version, expr) (typedExpr, _) = compileResult s <- ExprScript(version, typedExpr, checkSize = false) r <- eval(s, tx, blockchain) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/SerContextFunctionsTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/SerContextFunctionsTest.scala index e905d11339b..2799e788f6f 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/SerContextFunctionsTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/SerContextFunctionsTest.scala @@ -49,7 +49,7 @@ class SerContextFunctionsTest extends PropSpec { ) val untypedScript = Parser.parseExpr(scriptWithAllV1Functions(dtx, ttx)).get.value - val compiledScript = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untypedScript).explicitGet()._1 + val compiledScript = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untypedScript).explicitGet()._1 val bytes = Base64.decode("BAAAAANybmQJAAAAAAAAAgkAAGoAAAACCAUAAAACdHgAAAAJdGltZXN0YW1wAAAAAAAAAAACAAAAAAAAAAAABAAAAAdsb25nQWxsAwMDAwkAAAAAAAACCQAAaAAAAAIAAAAAAAAAA+gAAAAAAAAAAAIAAAAAAAAAB9AJAAAAAAAAAgkAAGkAAAACAAAAAAAAAAPoAAAAAAAAAAACAAAAAAAAAAH0BwkAAAAAAAACCQAAagAAAAIAAAAAAAAAA+gAAAAAAAAAAAIAAAAAAAAAAAAHCQAAAAAAAAIJAABkAAAAAgAAAAAAAAAD6AAAAAAAAAAAAgAAAAAAAAAD6gcJAAAAAAAAAgkAAGUAAAACAAAAAAAAAAPoAAAAAAAAAAACAAAAAAAAAAPmBwQAAAAJc3VtU3RyaW5nCQAAAAAAAAIJAAEsAAAAAgkAASwAAAACAgAAAAJoYQIAAAABLQIAAAACaGECAAAABWhhLWhhBAAAAA1zdW1CeXRlVmVjdG9yBAAAAAckbWF0Y2gwBQAAAAJ0eAMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAPRGF0YVRyYW5zYWN0aW9uBAAAAAJkMAUAAAAHJG1hdGNoMAQAAAAEYm9keQgFAAAAAmQwAAAACWJvZHlCeXRlcwkAAAAAAAACCQAAywAAAAIFAAAABGJvZHkBAAAAZAwB1SiqvsNcoQDYfHt6EoYy+vGc1EUxgZRXRFEToyoh7yIABAADaW50AAAAAAAAAAAYAARib29sAQEABGJsb2ICAAVhbGljZQADc3RyAwAEdGVzdAAAAWODBPoKAAAAAAABhqAJAADLAAAAAgEAAABkDAHVKKq+w1yhANh8e3oShjL68ZzURTGBlFdEUROjKiHvIgAEAANpbnQAAAAAAAAAABgABGJvb2wBAQAEYmxvYgIABWFsaWNlAANzdHIDAAR0ZXN0AAABY4ME+goAAAAAAAGGoAEAAABkDAHVKKq+w1yhANh8e3oShjL68ZzURTGBlFdEUROjKiHvIgAEAANpbnQAAAAAAAAAABgABGJvb2wBAQAEYmxvYgIABWFsaWNlAANzdHIDAAR0ZXN0AAABY4ME+goAAAAAAAGGoAMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAATVHJhbnNmZXJUcmFuc2FjdGlvbgYHBAAAAAdlcVVuaW9uBAAAAAckbWF0Y2gwBQAAAAJ0eAMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAPRGF0YVRyYW5zYWN0aW9uBgMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAATVHJhbnNmZXJUcmFuc2FjdGlvbgQAAAACdDAFAAAAByRtYXRjaDAJAAAAAAAAAggFAAAAAnQwAAAACXJlY2lwaWVudAkBAAAAB0FkZHJlc3MAAAABAQAAABoBVGQXjd+A7J8+39ZW6Opokqso6ed67rqcnQcEAAAABWJhc2ljAwMDBQAAAAdsb25nQWxsBQAAAAlzdW1TdHJpbmcHBQAAAA1zdW1CeXRlVmVjdG9yBwUAAAAHZXFVbmlvbgcEAAAABm5lUHJpbQMDCQEAAAACIT0AAAACAAAAAAAAAAPoAAAAAAAAAAPnCQEAAAACIT0AAAACCQABLAAAAAICAAAAAmhhAgAAAAJoYQIAAAAFaGEtaGEHCQEAAAACIT0AAAACCAUAAAACdHgAAAAJYm9keUJ5dGVzAQAAAASFqFqFBwQAAAAYbmVEYXRhRW50cnlBbmRHZXRFbGVtZW50BAAAAAckbWF0Y2gwBQAAAAJ0eAMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAPRGF0YVRyYW5zYWN0aW9uBAAAAAJkMQUAAAAHJG1hdGNoMAkBAAAAAiE9AAAAAgkAAZEAAAACCAUAAAACZDEAAAAEZGF0YQAAAAAAAAAAAAkBAAAACURhdGFFbnRyeQAAAAICAAAAAmhhBgMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAATVHJhbnNmZXJUcmFuc2FjdGlvbgYHBAAAABhuZU9wdGlvbkFuZEV4dHJhY3RIZWlnaHQEAAAAByRtYXRjaDAFAAAAAnR4AwkAAAEAAAACBQAAAAckbWF0Y2gwAgAAAA9EYXRhVHJhbnNhY3Rpb24GAwkAAAEAAAACBQAAAAckbWF0Y2gwAgAAABNUcmFuc2ZlclRyYW5zYWN0aW9uCQEAAAACIT0AAAACCQEAAAAHZXh0cmFjdAAAAAEJAAPpAAAAAQgFAAAAAnR4AAAAAmlkAAAAAAAAAAAABwQAAAACbmUDAwUAAAAGbmVQcmltBQAAABhuZURhdGFFbnRyeUFuZEdldEVsZW1lbnQHBQAAABhuZU9wdGlvbkFuZEV4dHJhY3RIZWlnaHQHBAAAAAdndGVMb25nAwkAAGYAAAACAAAAAAAAAAPoAAAAAAAAAAPnCQAAZwAAAAIAAAAAAAAAA+gAAAAAAAAAA+cHBAAAAAtnZXRMaXN0U2l6ZQQAAAAHJG1hdGNoMAUAAAACdHgDCQAAAQAAAAIFAAAAByRtYXRjaDACAAAAD0RhdGFUcmFuc2FjdGlvbgQAAAACZDIFAAAAByRtYXRjaDAJAQAAAAIhPQAAAAIJAAGQAAAAAQgFAAAAAmQyAAAABGRhdGEAAAAAAAAAAAADCQAAAQAAAAIFAAAAByRtYXRjaDACAAAAE1RyYW5zZmVyVHJhbnNhY3Rpb24GBwQAAAAFdW5hcnkDCQAAAAAAAAIA//////////8A//////////8JAAAAAAAAAgcJAQAAAAEhAAAAAQYHBAAAAAhmckFjdGlvbgkAAAAAAAACCQAAawAAAAMAAAAAAAAAAAwAAAAAAAAAAAMAAAAAAAAAAAQAAAAAAAAAAAkEAAAACGJ5dGVzT3BzBAAAAAckbWF0Y2gwBQAAAAJ0eAMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAPRGF0YVRyYW5zYWN0aW9uBAAAAAJkMwUAAAAHJG1hdGNoMAMDAwMJAQAAAAIhPQAAAAIJAADIAAAAAQgFAAAAAmQzAAAACWJvZHlCeXRlcwAAAAAAAAAAAAkBAAAAAiE9AAAAAgkAAMkAAAACCAUAAAACZDMAAAAJYm9keUJ5dGVzAAAAAAAAAAABAQAAAAIJMQcJAQAAAAIhPQAAAAIJAADKAAAAAggFAAAAAmQzAAAACWJvZHlCeXRlcwAAAAAAAAAAAQEAAAACCTEHCQEAAAACIT0AAAACCQEAAAAOdGFrZVJpZ2h0Qnl0ZXMAAAACCAUAAAACZDMAAAAJYm9keUJ5dGVzAAAAAAAAAAABAQAAAAIJMQcJAQAAAAIhPQAAAAIJAQAAAA5kcm9wUmlnaHRCeXRlcwAAAAIIBQAAAAJkMwAAAAlib2R5Qnl0ZXMAAAAAAAAAAAEBAAAAAgkxBwMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAATVHJhbnNmZXJUcmFuc2FjdGlvbgQAAAACdDEFAAAAByRtYXRjaDAJAAAAAAAAAgkBAAAACWlzRGVmaW5lZAAAAAEIBQAAAAJ0MQAAAApmZWVBc3NldElkBwcEAAAABnN0ck9wcwMDAwMJAQAAAAIhPQAAAAIJAAExAAAAAQIAAAAEaGFoYQAAAAAAAAAAAAkBAAAAAiE9AAAAAgkAAS8AAAACAgAAAARoYWhhAAAAAAAAAAABAgAAAAAHCQEAAAACIT0AAAACCQABMAAAAAICAAAABGhhaGEAAAAAAAAAAAACAAAAAAcJAQAAAAIhPQAAAAIJAQAAAAl0YWtlUmlnaHQAAAACAgAAAARoYWhhAAAAAAAAAAABAgAAAAAHCQEAAAACIT0AAAACCQEAAAAJZHJvcFJpZ2h0AAAAAgIAAAAEaGFoYQAAAAAAAAAAAAIAAAAABwQAAAAEcHVyZQMDAwMDAwMFAAAABWJhc2ljBQAAAAJuZQcFAAAAB2d0ZUxvbmcHBQAAAAtnZXRMaXN0U2l6ZQcFAAAABXVuYXJ5BwUAAAAIZnJBY3Rpb24HBQAAAAhieXRlc09wcwcFAAAABnN0ck9wcwcEAAAABnR4QnlJZAQAAAAHJG1hdGNoMAUAAAACdHgDCQAAAQAAAAIFAAAAByRtYXRjaDACAAAAD0RhdGFUcmFuc2FjdGlvbgYDCQAAAQAAAAIFAAAAByRtYXRjaDACAAAAE1RyYW5zZmVyVHJhbnNhY3Rpb24EAAAAAWcJAQAAAAdleHRyYWN0AAAAAQkAA+gAAAABAQAAACCB+LHr7irQ7N6spxF91VJYssai++EyJEw1WFYwXb0UCwkAAAAAAAACCAUAAAABZwAAAAJpZAEAAAAggfix6+4q0OzerKcRfdVSWLLGovvhMiRMNVhWMF29FAsHBAAAAAdlbnRyaWVzBAAAAAckbWF0Y2gwBQAAAAJ0eAMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAPRGF0YVRyYW5zYWN0aW9uBAAAAAFkBQAAAAckbWF0Y2gwBAAAAANpbnQJAQAAAAdleHRyYWN0AAAAAQkABBAAAAACCAUAAAABZAAAAARkYXRhAgAAAANpbnQEAAAABGJvb2wJAQAAAAdleHRyYWN0AAAAAQkABBEAAAACCAUAAAABZAAAAARkYXRhAgAAAARib29sBAAAAARibG9iCQEAAAAHZXh0cmFjdAAAAAEJAAQSAAAAAggFAAAAAWQAAAAEZGF0YQIAAAAEYmxvYgQAAAADc3RyCQEAAAAHZXh0cmFjdAAAAAEJAAQTAAAAAggFAAAAAWQAAAAEZGF0YQIAAAADc3RyBAAAAAlkYXRhQnlLZXkDAwMJAAAAAAAAAgkAAaQAAAABBQAAAANpbnQCAAAAAjI0BgkAAAAAAAACCQABpQAAAAEFAAAABGJvb2wCAAAABHRydWUGCQAAZgAAAAIJAADIAAAAAQUAAAAEYmxvYgAAAAAAAAAAAAYJAAAAAAAAAgUAAAADc3RyAgAAAAR0ZXN0BAAAAAJkMAkBAAAAB2V4dHJhY3QAAAABCQEAAAAKZ2V0SW50ZWdlcgAAAAIIBQAAAAFkAAAABGRhdGEAAAAAAAAAAAAEAAAAAmQxCQEAAAAHZXh0cmFjdAAAAAEJAQAAAApnZXRCb29sZWFuAAAAAggFAAAAAWQAAAAEZGF0YQAAAAAAAAAAAQQAAAACZDIJAQAAAAlnZXRCaW5hcnkAAAACCAUAAAABZAAAAARkYXRhAAAAAAAAAAACBAAAAAJkMwkBAAAACWdldFN0cmluZwAAAAIIBQAAAAFkAAAABGRhdGEAAAAAAAAAAAMEAAAAC2RhdGFCeUluZGV4AwMDCQAAAAAAAAIJAAGaAAAAAQUAAAACZDABAAAABGm3HXkGCQAAAAAAAAIJAAGcAAAAAQUAAAACZDEBAAAABIIYo5IGCQEAAAAJaXNEZWZpbmVkAAAAAQUAAAACZDIGCQAAAAAAAAIJAAGbAAAAAQkBAAAAB2V4dHJhY3QAAAABBQAAAAJkMwEAAAAEmnopqgMFAAAACWRhdGFCeUtleQUAAAALZGF0YUJ5SW5kZXgHAwkAAAEAAAACBQAAAAckbWF0Y2gwAgAAABNUcmFuc2ZlclRyYW5zYWN0aW9uBAAAAANhZGQJAQAAAAdBZGRyZXNzAAAAAQEAAAAaAVRkF43fgOyfPt/WVujqaJKrKOnneu66nJ0EAAAABGxvbmcJAAAAAAAAAgkBAAAAB2V4dHJhY3QAAAABCQAEGgAAAAIFAAAAA2FkZAIAAAADaW50AAAAAAAAAAAYBAAAAAVib29sMQkAAAAAAAACCQEAAAAHZXh0cmFjdAAAAAEJAAQbAAAAAgUAAAADYWRkAgAAAARib29sBgQAAAADYmluCQAAAAAAAAIJAQAAAAdleHRyYWN0AAAAAQkABBwAAAACBQAAAANhZGQCAAAABGJsb2IBAAAABWFsaWNlBAAAAARzdHIxCQAAAAAAAAIJAQAAAAdleHRyYWN0AAAAAQkABB0AAAACBQAAAANhZGQCAAAAA3N0cgIAAAAEdGVzdAMDAwUAAAAEbG9uZwUAAAAFYm9vbDEHBQAAAANiaW4HBQAAAARzdHIxBwMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAWQ3JlYXRlQWxpYXNUcmFuc2FjdGlvbgkAAAIAAAABAgAAAAVvaCBubwMJAAABAAAAAgUAAAAHJG1hdGNoMAIAAAAPQnVyblRyYW5zYWN0aW9uCQEAAAAFdGhyb3cAAAAABwQAAAAHYUZyb21QSwkAAAAAAAACCQEAAAAUYWRkcmVzc0Zyb21QdWJsaWNLZXkAAAABCAUAAAACdHgAAAAPc2VuZGVyUHVibGljS2V5CAUAAAACdHgAAAAGc2VuZGVyBAAAAA9hRnJvbVN0ck9yUmVjaXAEAAAAByRtYXRjaDAFAAAAAnR4AwkAAAEAAAACBQAAAAckbWF0Y2gwAgAAAA9EYXRhVHJhbnNhY3Rpb24JAAAAAAAAAgkBAAAAEWFkZHJlc3NGcm9tU3RyaW5nAAAAAQIAAAAjM041R1JxekRCaGpWWG5DbjQ0YmFIY3oyR29aeTVxTHh0VGgJAQAAAAdBZGRyZXNzAAAAAQEAAAAaAVSoYvWtn5xSOgZyscKLnKFwwV9ogl+Q7gADCQAAAQAAAAIFAAAAByRtYXRjaDACAAAAE1RyYW5zZmVyVHJhbnNhY3Rpb24EAAAAAnQxBQAAAAckbWF0Y2gwCQAAAAAAAAIJAAQkAAAAAQgFAAAAAnQxAAAACXJlY2lwaWVudAkBAAAAB0FkZHJlc3MAAAABAQAAABoBVGQXjd+A7J8+39ZW6Opokqso6ed67rqcnQcEAAAACGJhbGFuY2VzAwkAAGYAAAACCQAD6wAAAAIIBQAAAAJ0eAAAAAZzZW5kZXIFAAAABHVuaXQAAAAAAAAAAAAJAQAAAAIhPQAAAAIJAQAAAAx3YXZlc0JhbGFuY2UAAAABCAUAAAACdHgAAAAGc2VuZGVyAAAAAAAAAAAABwQAAAAFd2F2ZXMDAwMDAwUAAAAGdHhCeUlkBQAAAAdlbnRyaWVzBwUAAAAIYmFsYW5jZXMHBQAAAAdhRnJvbVBLBwUAAAAPYUZyb21TdHJPclJlY2lwBwkAAGYAAAACBQAAAAZoZWlnaHQAAAAAAAAAAAAHBAAAAANia3MDAwkBAAAAAiE9AAAAAgkAAfYAAAABAQAAAAABAAAAAAkBAAAAAiE9AAAAAgkAAfUAAAABAQAAAAABAAAAAAcJAQAAAAIhPQAAAAIJAAH3AAAAAQEAAAAAAQAAAAAHBAAAAANzaWcJAQAAAAIhPQAAAAIJAAH0AAAAAwEAAAACGr4BAAAAAgA8AQAAAAI1uAYEAAAABXN0cjU4CQAAAAAAAAIJAAJZAAAAAQkAAlgAAAABCAUAAAACdHgAAAACaWQIBQAAAAJ0eAAAAAJpZAQAAAAFc3RyNjQJAAAAAAAAAgkAAlsAAAABCQACWgAAAAEIBQAAAAJ0eAAAAAJpZAgFAAAAAnR4AAAAAmlkBAAAAAZjcnlwdG8DAwMFAAAAA2JrcwUAAAADc2lnBwUAAAAFc3RyNTgHBQAAAAVzdHI2NAcDBQAAAANybmQDBQAAAARwdXJlBQAAAAV3YXZlcwcFAAAABmNyeXB0bw==") SerdeV1.serialize(compiledScript) shouldBe bytes } diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/TransactionBindingsTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/TransactionBindingsTest.scala index f307953d23e..c9eeaf2bc27 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/TransactionBindingsTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/TransactionBindingsTest.scala @@ -881,7 +881,7 @@ class TransactionBindingsTest extends PropSpec with PathMockFactory with EitherV ByteStr.empty ) for { - compileResult <- compiler.ExpressionCompiler(ctx.compilerContext, expr) + compileResult <- compiler.ExpressionCompiler(ctx.compilerContext, V3, expr) (typedExpr, _) = compileResult r <- EvaluatorV1().apply[EVALUATED](ctx.evaluationContext(environment), typedExpr).leftMap(_.message) } yield r @@ -913,7 +913,7 @@ class TransactionBindingsTest extends PropSpec with PathMockFactory with EitherV ) for { - compileResult <- ExpressionCompiler(ctx.compilerContext, expr) + compileResult <- ExpressionCompiler(ctx.compilerContext, V2, expr) (typedExpr, _) = compileResult r <- EvaluatorV1().apply[EVALUATED](ctx.evaluationContext(env), typedExpr).leftMap(_.message) } yield r diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/package.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/package.scala index 43b9d510d85..89c5bae26d0 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/package.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/predef/package.scala @@ -27,7 +27,7 @@ package object predef { def runScript[T <: EVALUATED](script: String, version: StdLibVersion, t: In, blockchain: Blockchain, chainId: Byte): Either[String, T] = { val expr = Parser.parseExpr(script).get.value for { - compileResult <- ExpressionCompiler(compilerContext(version, Expression, isAssetScript = false), expr) + compileResult <- ExpressionCompiler(compilerContext(version, Expression, isAssetScript = false), version, expr) (typedExpr, _) = compileResult directives = DirectiveSet(version, Account, Expression).explicitGet() evalContext <- BlockchainContext.build( diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/BalancesV4Test.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/BalancesV4Test.scala index 87ea7cef5c4..2523329d952 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/BalancesV4Test.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/BalancesV4Test.scala @@ -132,7 +132,7 @@ class BalancesV4Test extends PropSpec with WithState { | assetBalance(Address(base58'$acc'), this.id) == $a && assetBalance(Alias("alias"), this.id) == $a """.stripMargin val parsedScript = Parser.parseExpr(script).get.value - ExprScript(V4, ExpressionCompiler(ctx.compilerContext, parsedScript).explicitGet()._1) + ExprScript(V4, ExpressionCompiler(ctx.compilerContext, V4, parsedScript).explicitGet()._1) .explicitGet() } @@ -194,7 +194,7 @@ class BalancesV4Test extends PropSpec with WithState { | wavesBalance(Address(base58'$acc')).regular == $w """.stripMargin val parsedScript = Parser.parseExpr(script).get.value - ExprScript(V4, ExpressionCompiler(ctx.compilerContext, parsedScript).explicitGet()._1) + ExprScript(V4, ExpressionCompiler(ctx.compilerContext, V4, parsedScript).explicitGet()._1) .explicitGet() } diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/MultiSig2of3Test.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/MultiSig2of3Test.scala index 94e729bea6a..dd3ccb8635a 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/MultiSig2of3Test.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/MultiSig2of3Test.scala @@ -37,7 +37,7 @@ class MultiSig2of3Test extends PropSpec with WithState { | """.stripMargin val untyped = Parser.parseExpr(script).get.value - ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untyped).explicitGet()._1 + ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untyped).explicitGet()._1 } val preconditionsAndTransfer: (GenesisTransaction, SetScriptTransaction, TransferTransaction, Seq[ByteStr]) = { diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/NotaryControlledTransferScenarioTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/NotaryControlledTransferScenarioTest.scala index aa4eae87dbb..05e1d718a45 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/NotaryControlledTransferScenarioTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/NotaryControlledTransferScenarioTest.scala @@ -58,7 +58,7 @@ class NotaryControlledTransferScenarioTest extends PropSpec with WithState { | } """.stripMargin val untypedScript = Parser.parseExpr(assetScript).get.value - val typedScript = ExprScript(ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untypedScript).explicitGet()._1) + val typedScript = ExprScript(ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untypedScript).explicitGet()._1) .explicitGet() val issue = TxHelpers.issue(company, 100, script = Some(typedScript)) @@ -88,7 +88,7 @@ class NotaryControlledTransferScenarioTest extends PropSpec with WithState { private def eval(code: String) = { val untyped = Parser.parseExpr(code).get.value - val typed = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untyped).map(_._1) + val typed = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untyped).map(_._1) typed.flatMap(r => EvaluatorV1().apply[EVALUATED](dummyEvalContext, r).leftMap(_.message)) } diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OnlyTransferIsAllowedTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OnlyTransferIsAllowedTest.scala index d2dbe0db4c3..af9942b3193 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OnlyTransferIsAllowedTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OnlyTransferIsAllowedTest.scala @@ -32,7 +32,7 @@ class OnlyTransferIsAllowedTest extends PropSpec with WithState { | } """.stripMargin val untyped = Parser.parseExpr(scriptText).get.value - val transferAllowed = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untyped).explicitGet()._1 + val transferAllowed = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untyped).explicitGet()._1 val (genesis, script, lease, transfer) = preconditions(transferAllowed) assertDiffAndState(Seq(TestBlock.create(Seq(genesis, script))), TestBlock.create(Seq(transfer)), smartEnabledFS) { case _ => () } diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OracleDataTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OracleDataTest.scala index c23560e0447..b6f6a8d77c5 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OracleDataTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/OracleDataTest.scala @@ -55,7 +55,7 @@ class OracleDataTest extends PropSpec with WithState { |}""".stripMargin val untypedAllFieldsRequiredScript = Parser.parseExpr(allFieldsRequiredScript).get.value val typedAllFieldsRequiredScript = - ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untypedAllFieldsRequiredScript).explicitGet()._1 + ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untypedAllFieldsRequiredScript).explicitGet()._1 val setScript = TxHelpers.setScript(master, ExprScript(typedAllFieldsRequiredScript).explicitGet()) val transferFromScripted = TxHelpers.transfer(master, alice.toAddress) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransactionFieldAccessTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransactionFieldAccessTest.scala index 8b391e58313..722fd8fca31 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransactionFieldAccessTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransactionFieldAccessTest.scala @@ -24,7 +24,7 @@ class TransactionFieldAccessTest extends PropSpec with WithState { val genesis = TxHelpers.genesis(master.toAddress) val untyped = Parser.parseExpr(code).get.value - val typed = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), untyped).explicitGet()._1 + val typed = ExpressionCompiler(compilerContext(V1, Expression, isAssetScript = false), V1, untyped).explicitGet()._1 val setScript = TxHelpers.setScript(master, ExprScript(typed).explicitGet()) val transfer = TxHelpers.transfer(master, recipient.toAddress, ENOUGH_AMT / 2) val lease = TxHelpers.lease(master, recipient.toAddress, ENOUGH_AMT / 2) diff --git a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransferByIdTest.scala b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransferByIdTest.scala index eb3ba6de556..925ea4e4b6d 100644 --- a/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransferByIdTest.scala +++ b/node/src/test/scala/com/wavesplatform/state/diffs/smart/scenarios/TransferByIdTest.scala @@ -34,7 +34,7 @@ class TransferByIdTest extends PropSpec with WithState { val expr: EXPR = { val parsed = Parser.parseExpr(scriptSrc).get.value - ExpressionCompiler(compilerContext(V3, Expression, isAssetScript = false), parsed).explicitGet()._1 + ExpressionCompiler(compilerContext(V3, Expression, isAssetScript = false), V3, parsed).explicitGet()._1 } property("Transfer by id works fine") { diff --git a/node/src/test/scala/com/wavesplatform/transaction/smart/script/estimator/FunctionComplexityTest.scala b/node/src/test/scala/com/wavesplatform/transaction/smart/script/estimator/FunctionComplexityTest.scala index 3a4d9ab730a..e8e1a4ca84e 100644 --- a/node/src/test/scala/com/wavesplatform/transaction/smart/script/estimator/FunctionComplexityTest.scala +++ b/node/src/test/scala/com/wavesplatform/transaction/smart/script/estimator/FunctionComplexityTest.scala @@ -106,7 +106,7 @@ class FunctionComplexityTest(estimator: ScriptEstimator) extends PropSpec { property("estimate script with all functions") { def check(version: StdLibVersion, expectedCost: Int) = { - val expr = ExpressionCompiler(ctx(version).compilerContext, getAllFuncExpression(version)).explicitGet()._1 + val expr = ExpressionCompiler(ctx(version).compilerContext, version, getAllFuncExpression(version)).explicitGet()._1 estimate(expr, ctx(version), utils.functionCosts(version)) shouldBe Right(expectedCost) } diff --git a/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/Repl.scala b/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/Repl.scala index b4872df83f3..f5a3e003909 100644 --- a/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/Repl.scala +++ b/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/Repl.scala @@ -51,13 +51,13 @@ case class Repl( currentState, view, (oldCtx: (CompilerContext, EvaluationContext[Environment, Future])) => - engine.eval(expr, oldCtx._1, oldCtx._2).map { + engine.eval(expr, version, oldCtx._1, oldCtx._2).map { case Left(e) => (Left(e), oldCtx) case Right((r, newCtx)) => (Right(r), newCtx) }: Future[(Either[String, String], (CompilerContext, EvaluationContext[Environment, Future]))] ) - def initLibraries(): Unit = { + private def initLibraries(): Unit = { val libraryState = state( ( initialCtx.compilerContext, @@ -72,7 +72,7 @@ case class Repl( view, oldCtx => new ReplEngine[Id]() - .eval(libraries.mkString("\n"), oldCtx._1, oldCtx._2) + .eval(libraries.mkString("\n"), version, oldCtx._1, oldCtx._2) .fold( e => throw new RuntimeException(e), { case (r, ctx @ (compilerCtx, evaluationCtx)) => diff --git a/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/ReplEngine.scala b/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/ReplEngine.scala index 270f8230578..927df4b029b 100644 --- a/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/ReplEngine.scala +++ b/repl/shared/src/main/scala/com/wavesplatform/lang/v1/repl/ReplEngine.scala @@ -3,6 +3,7 @@ package com.wavesplatform.lang.v1.repl import cats.Monad import cats.data.EitherT import cats.implicits.* +import com.wavesplatform.lang.directives.values.StdLibVersion import com.wavesplatform.lang.v1.compiler.CompilerContext.VariableInfo import com.wavesplatform.lang.v1.compiler.Terms.EVALUATED import com.wavesplatform.lang.v1.compiler.Types.{FINAL, UNIT} @@ -20,13 +21,14 @@ class ReplEngine[F[_]: Monad] { def eval( expr: String, + version: StdLibVersion, compileCtx: CompilerContext, evalCtx: EvaluationContext[Environment, F] ): F[Either[String, (String, (CompilerContext, EvaluationContext[Environment, F]))]] = { val r = for { parsed <- EitherT.fromEither[F](parse(expr)) - (newCompileCtx, compiled, exprType) <- EitherT.fromEither[F](ExpressionCompiler.applyWithCtx(compileCtx, parsed)) + (newCompileCtx, compiled, exprType) <- EitherT.fromEither[F](ExpressionCompiler.applyWithCtx(compileCtx, version, parsed)) evaluated <- EitherT(evaluator.applyWithCtx(evalCtx, compiled)).leftMap(error => if (error.message.isEmpty) "Evaluation error" else error.message )