Skip to content

Commit

Permalink
DEX-361 dApp execution error during order placement (#18)
Browse files Browse the repository at this point in the history
* A blockchain access during an order's validation now denied on higher level. All new functions those doesn't require a blockchain's access will be supported automatically;
* Error messages now contains blockchain's functions instead of the script ones. Waiting for a fix from Smart Contracts;
* Fixed a bug with order's validation on dApp script;

Internals:
* OrderValidator. verifyOrderByAccountScript tracing exceptions;
* MatcherContext now is BlockchainContext (used in a normal scenario), but with mocked Blockchain: all methods throw SecurityException;
* MatcherScriptRunner works correctly with dApps;

Tests:
* com.wavesplatform.dex.error.ProduceError with regular expressions;
* Added MatcherScriptRunnerTest;
  • Loading branch information
vsuharnikov authored and koloale committed Jul 22, 2019
1 parent 6939661 commit a7d7446
Show file tree
Hide file tree
Showing 9 changed files with 249 additions and 96 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ class OrdersFromScriptedAccTestSuite extends MatcherSuiteBase {
setContract(Some("true && (height > 0)"), bob)
assertBadRequestAndResponse(
node.placeOrder(bob, aliceWavesPair, OrderType.BUY, 500, 2.waves * Order.PriceConstant, smartTradeFee, version = 2, 10.minutes),
"height is inaccessible when running script on matcher"
"An access to the blockchain.height is denied on DEX"
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ class ProofAndAssetPairTestSuite extends MatcherSuiteBase {
assertBadRequestAndResponse(
node
.placeOrder(alice, predefAssetPair, OrderType.BUY, 500, 2.waves * Order.PriceConstant, smartMatcherFee, version = 2, 10.minutes),
"The account's script of .* returned the error"
"An access to the blockchain.height is denied on DEX"
)
}

Expand Down
12 changes: 12 additions & 0 deletions dex/src/main/scala/com/wavesplatform/dex/error/package.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
package com.wavesplatform.dex

import java.io.{PrintWriter, StringWriter}

package object error {
def formatStackTrace(e: Throwable): String = {
val r = new StringWriter()
e.printStackTrace(new PrintWriter(r))
r.flush()
r.getBuffer.toString
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,15 +25,15 @@ import com.wavesplatform.transaction.assets.exchange.OrderOps._
import com.wavesplatform.transaction.assets.exchange._
import com.wavesplatform.transaction.smart.Verifier
import com.wavesplatform.transaction.smart.script.ScriptRunner
import com.wavesplatform.utils.Time
import com.wavesplatform.utils.{ScorexLogging, Time}
import kamon.Kamon
import shapeless.Coproduct

import scala.Either.cond
import scala.math.BigDecimal.RoundingMode
import scala.util.control.NonFatal

object OrderValidator {
object OrderValidator extends ScorexLogging {

type Result[T] = Either[MatcherError, T]

Expand All @@ -57,13 +57,15 @@ object OrderValidator {
error.AccountFeatureUnsupported(BlockchainFeatures.SmartAccountTrading).asLeft
else if (order.version <= 1) error.AccountNotSupportOrderVersion(address, 2, order.version).asLeft
else
try MatcherScriptRunner(script, order, isTokenScript = false) match {
try MatcherScriptRunner(script, order) match {
case (_, Left(execError)) => error.AccountScriptReturnedError(address, execError).asLeft
case (_, Right(FALSE)) => error.AccountScriptDeniedOrder(address).asLeft
case (_, Right(TRUE)) => lift(order)
case (_, Right(x)) => error.AccountScriptUnexpectResult(address, x.toString).asLeft
} catch {
case NonFatal(e) => error.AccountScriptException(address, e.getClass.getCanonicalName, e.getMessage).asLeft
case NonFatal(e) =>
log.trace(error.formatStackTrace(e))
error.AccountScriptException(address, e.getClass.getCanonicalName, e.getMessage).asLeft
}
}

Expand Down Expand Up @@ -219,10 +221,10 @@ object OrderValidator {
* @param multiplier coefficient that is used in market aware for specifying deviation bounds
*/
private[dex] def getMinValidFeeForSettings(order: Order,
orderFeeSettings: OrderFeeSettings,
matchPrice: Long,
rateCache: RateCache,
multiplier: Double = 1): Long = {
orderFeeSettings: OrderFeeSettings,
matchPrice: Long,
rateCache: RateCache,
multiplier: Double = 1): Long = {

orderFeeSettings match {
case DynamicSettings(dynamicBaseFee) => multiplyFeeByDouble(dynamicBaseFee, rateCache.getRate(order.matcherFeeAssetId).get)
Expand Down
179 changes: 105 additions & 74 deletions dex/src/main/scala/com/wavesplatform/dex/smart/MatcherContext.scala
Original file line number Diff line number Diff line change
@@ -1,94 +1,125 @@
package com.wavesplatform.dex.smart

import cats.Eval
import cats.data.EitherT
import cats.implicits._
import cats.kernel.Monoid
import com.wavesplatform.lang.directives.DirectiveDictionary
import com.wavesplatform.lang.directives.values.StdLibVersion.VersionDic
import com.wavesplatform.account.{Address, Alias}
import com.wavesplatform.block.Block.BlockId
import com.wavesplatform.block.{Block, BlockHeader}
import com.wavesplatform.common.state.ByteStr
import com.wavesplatform.lang.directives.values._
import com.wavesplatform.lang.v1.compiler.Terms.{CaseObj, EVALUATED}
import com.wavesplatform.lang.v1.compiler.Types.{CASETYPEREF, FINAL, LONG, UNIT}
import com.wavesplatform.lang.v1.evaluator.FunctionIds._
import com.wavesplatform.lang.script.Script
import com.wavesplatform.lang.v1.evaluator.ctx._
import com.wavesplatform.lang.v1.evaluator.ctx.impl.waves.Bindings
import com.wavesplatform.lang.v1.evaluator.ctx.impl.waves.Bindings.{ordType, orderObject}
import com.wavesplatform.lang.v1.evaluator.ctx.impl.waves.Types._
import com.wavesplatform.lang.v1.evaluator.ctx.impl.{CryptoContext, PureContext}
import com.wavesplatform.lang.v1.traits.domain.{OrdType, Recipient}
import com.wavesplatform.lang.v1.{CTX, FunctionHeader}
import com.wavesplatform.lang.{ExecutionError, Global}
import com.wavesplatform.lang.{ExecutionError, ValidationError}
import com.wavesplatform.settings.BlockchainSettings
import com.wavesplatform.state.reader.LeaseDetails
import com.wavesplatform.state.{
AccountDataInfo,
AssetDescription,
AssetDistribution,
AssetDistributionPage,
BalanceSnapshot,
Blockchain,
DataEntry,
Height,
InvokeScriptResult,
LeaseBalance,
Portfolio,
TransactionId,
VolumeAndFee
}
import com.wavesplatform.transaction.assets.IssueTransaction
import com.wavesplatform.transaction.assets.exchange.Order
import com.wavesplatform.transaction.smart.RealTransactionWrapper
import com.wavesplatform.transaction.lease.LeaseTransaction
import com.wavesplatform.transaction.smart.BlockchainContext
import com.wavesplatform.transaction.{Asset, Transaction, TransactionParser}
import com.wavesplatform.utils.CloseableIterator
import monix.eval.Coeval
import shapeless.Coproduct

import scala.util.control.NoStackTrace

// Used only for order validation
object MatcherContext {

def build(version: StdLibVersion, nByte: Byte, in: Eval[Order], proofsEnabled: Boolean): EvaluationContext = {
val baseContext = Monoid.combine(PureContext.build(Global, version), CryptoContext.build(Global, version)).evaluationContext

val inputEntityCoeval: Eval[Either[ExecutionError, CaseObj]] =
Eval.defer(in.map(o => Right(orderObject(RealTransactionWrapper.ord(o), proofsEnabled, version))))

def constVal[T <: EVALUATED](x: T): LazyVal = LazyVal(EitherT(Eval.now[Either[ExecutionError, T]](Right(x))))
def inaccessibleVal(name: String): LazyVal =
LazyVal(EitherT(Eval.now[Either[ExecutionError, Nothing]](Left(s"$name is inaccessible when running script on matcher"))))

val orderType: CASETYPEREF = buildOrderType(proofsEnabled)
val matcherTypes = Seq(addressType, orderType, assetPairType)
val thisVal = in.map(order => Bindings.senderObject(Recipient.Address(order.senderPublicKey.toAddress.bytes)))
def build(version: StdLibVersion, nByte: Byte, inE: Eval[Order], isDApp: Boolean): Either[ExecutionError, EvaluationContext] = {
val in: Coeval[Order] = Coeval.delay(inE.value)
BlockchainContext
.build(
version,
nByte,
in.map(o => Coproduct[BlockchainContext.In](o)),
Coeval.raiseError(new Denied("height")),
deniedBlockchain,
isTokenContext = false,
isContract = isDApp,
in.map(_.senderPublicKey.toAddress.bytes)
)
}

val vars: Map[String, ((FINAL, String), LazyVal)] = Seq[(String, FINAL, String, LazyVal)](
("this", addressType, "Script address", LazyVal(EitherT.right(thisVal))),
("tx", orderType, "Processing order", LazyVal(EitherT(inputEntityCoeval))),
("height", LONG, "undefined height placeholder", inaccessibleVal("height")),
("lastBlock", blockInfo, "undefined lastBlock placeholder", inaccessibleVal("lastBlock")),
("Sell", ordTypeType, "Sell OrderType", constVal(ordType(OrdType.Sell))),
("Buy", ordTypeType, "Buy OrderType", constVal(ordType(OrdType.Buy)))
).map { case (name, retType, desc, value) => (name, ((retType, desc), value)) }(collection.breakOut)
private class Denied(methodName: String) extends SecurityException(s"An access to the blockchain.$methodName is denied on DEX") with NoStackTrace
private def kill(methodName: String) = throw new Denied(methodName)

def inaccessibleFunction(internalName: Short, name: String): BaseFunction = {
val msg = s"Function $name is inaccessible when running script on matcher"
NativeFunction(name, 1, internalName, UNIT, msg, Seq.empty: _*) { case _ => msg.asLeft }
}
private val deniedBlockchain = new Blockchain {
override def settings: BlockchainSettings = kill("settings")
override def height: Int = kill("height")
override def score: BigInt = kill("score")
override def blockHeaderAndSize(height: Int): Option[(BlockHeader, Int)] = kill("blockHeaderAndSize")
override def blockHeaderAndSize(blockId: ByteStr): Option[(BlockHeader, Int)] = kill("blockHeaderAndSize")
override def lastBlock: Option[Block] = kill("lastBlock")
override def carryFee: Long = kill("carryFee")
override def blockBytes(height: Int): Option[Array[Byte]] = kill("blockBytes")
override def blockBytes(blockId: ByteStr): Option[Array[Byte]] = kill("blockBytes")
override def heightOf(blockId: ByteStr): Option[Int] = kill("heightOf")

def inaccessibleUserFunction(name: String): BaseFunction = {
val msg = s"Function $name is inaccessible when running script on matcher"
NativeFunction(
name,
DirectiveDictionary[StdLibVersion].all.map(_ -> 1L).toMap,
FunctionTypeSignature(UNIT, Seq.empty, FunctionHeader.User(name)),
_ => msg.asLeft,
msg,
Array.empty
)
}
/** Returns the most recent block IDs, starting from the most recent one */
override def lastBlockIds(howMany: Int): Seq[ByteStr] = kill("lastBlockIds")

val nativeFunctions = Array(
DATA_LONG_FROM_STATE -> "getInteger",
DATA_BOOLEAN_FROM_STATE -> "getBoolean",
DATA_BYTES_FROM_STATE -> "getBinary",
DATA_STRING_FROM_STATE -> "getString",
GETTRANSACTIONBYID -> "transactionById",
ADDRESSFROMRECIPIENT -> "addressFromRecipient",
ACCOUNTASSETBALANCE -> "assetBalance",
GETASSETINFOBYID -> "assetInfo",
TRANSFERTRANSACTIONBYID -> "transferTransactionById",
TRANSACTIONHEIGHTBYID -> "transactionHeightById",
BLOCKINFOBYHEIGHT -> "blockInfoByHeight"
).map(Function.tupled(inaccessibleFunction))
/** Returns a chain of blocks starting with the block with the given ID (from oldest to newest) */
override def blockIdsAfter(parentSignature: ByteStr, howMany: Int): Option[Seq[ByteStr]] = kill("blockIdsAfter")
override def parentHeader(block: BlockHeader, back: Int): Option[BlockHeader] = kill("parentHeader")
override def totalFee(height: Int): Option[Long] = kill("totalFee")

val userFunctions = Array(
"getIntegerValue",
"getBooleanValue",
"getBinaryValue",
"getStringValue",
"wavesBalance"
).map(inaccessibleUserFunction)
/** Features related */
override def approvedFeatures: Map[Short, Int] = kill("approvedFeatures")
override def activatedFeatures: Map[Short, Int] = kill("activatedFeatures")
override def featureVotes(height: Int): Map[Short, Int] = kill("featureVotes")
override def portfolio(a: Address): Portfolio = kill("portfolio")
override def transactionInfo(id: ByteStr): Option[(Int, Transaction)] = kill("transactionInfo")
override def transactionHeight(id: ByteStr): Option[Int] = kill("transactionHeight")
override def nftList(address: Address, from: Option[Asset.IssuedAsset]): CloseableIterator[IssueTransaction] = kill("nftList")
override def addressTransactions(address: Address,
types: Set[TransactionParser],
fromId: Option[ByteStr]): CloseableIterator[(Height, Transaction)] = kill("addressTransactions")
override def containsTransaction(tx: Transaction): Boolean = kill("containsTransaction")
override def assetDescription(id: Asset.IssuedAsset): Option[AssetDescription] = kill("assetDescription")
override def resolveAlias(a: Alias): Either[ValidationError, Address] = kill("resolveAlias")
override def leaseDetails(leaseId: ByteStr): Option[LeaseDetails] = kill("leaseDetails")
override def filledVolumeAndFee(orderId: ByteStr): VolumeAndFee = kill("filledVolumeAndFee")

val matcherContext = CTX(matcherTypes, vars, nativeFunctions ++ userFunctions).evaluationContext
/** Retrieves Waves balance snapshot in the [from, to] range (inclusive) */
override def balanceSnapshots(address: Address, from: Int, to: BlockId): Seq[BalanceSnapshot] = kill("balanceSnapshots")
override def accountScript(address: Address): Option[Script] = kill("accountScript")
override def hasScript(address: Address): Boolean = kill("hasScript")
override def assetScript(id: Asset.IssuedAsset): Option[Script] = kill("assetScript")
override def hasAssetScript(id: Asset.IssuedAsset): Boolean = kill("hasAssetScript")
override def accountDataKeys(address: Address): Seq[String] = kill("accountDataKeys")
override def accountData(acc: Address, key: String): Option[DataEntry[_]] = kill("accountData")
override def accountData(acc: Address): AccountDataInfo = kill("accountData")
override def leaseBalance(address: Address): LeaseBalance = kill("leaseBalance")
override def balance(address: Address, mayBeAssetId: Asset): Long = kill("balance")
override def assetDistribution(asset: Asset.IssuedAsset): AssetDistribution = kill("assetDistribution")
override def assetDistributionAtHeight(asset: Asset.IssuedAsset,
height: Int,
count: Int,
fromAddress: Option[Address]): Either[ValidationError, AssetDistributionPage] =
kill("assetDistributionAtHeight")
override def wavesDistribution(height: Int): Either[ValidationError, Map[Address, Long]] = kill("wavesDistribution")
override def allActiveLeases: CloseableIterator[LeaseTransaction] = kill("allActiveLeases")

baseContext |+| matcherContext
/** Builds a new portfolio map by applying a partial function to all portfolios on which the function is defined.
*
* @note Portfolios passed to `pf` only contain Waves and Leasing balances to improve performance */
override def collectLposPortfolios[A](pf: PartialFunction[(Address, Portfolio), A]): Map[Address, A] = kill("collectLposPortfolios")
override def invokeScriptResult(txId: TransactionId): Either[ValidationError, InvokeScriptResult] = kill("invokeScriptResult")
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -4,36 +4,42 @@ import cats.Eval
import cats.implicits._
import com.wavesplatform.account.AddressScheme
import com.wavesplatform.lang.contract.DApp
import com.wavesplatform.lang.script.v1.ExprScript
import com.wavesplatform.lang.script.{ContractScript, Script}
import com.wavesplatform.lang.v1.compiler.Terms.{EVALUATED, FALSE, TRUE}
import com.wavesplatform.lang.v1.evaluator.{ContractEvaluator, EvaluatorV1, Log}
import com.wavesplatform.transaction.assets.exchange.Order
import com.wavesplatform.lang.script.v1.ExprScript
import com.wavesplatform.transaction.smart.{RealTransactionWrapper, Verifier}
import com.wavesplatform.transaction.{Authorized, Proven}

object MatcherScriptRunner {

def apply(script: Script, order: Order, isTokenScript: Boolean): (Log, Either[String, EVALUATED]) = script match {
def apply(script: Script, order: Order): (Log, Either[String, EVALUATED]) = script match {
case s: ExprScript =>
val ctx = MatcherContext.build(script.stdLibVersion, AddressScheme.current.chainId, Eval.later(order), !isTokenScript)
EvaluatorV1.applyWithLogging(ctx, s.expr)
MatcherContext.build(script.stdLibVersion, AddressScheme.current.chainId, Eval.later(order), isDApp = false) match {
case Left(error) => (List.empty, Left(error))
case Right(ctx) => EvaluatorV1.applyWithLogging(ctx, s.expr)
}

case ContractScript.ContractScriptImpl(_, DApp(_, decls, _, Some(vf)), _) =>
val ctx = MatcherContext.build(
MatcherContext.build(
script.stdLibVersion,
AddressScheme.current.chainId,
Eval.later(???) /*order not used in global context where @Verifier annotation is used */,
proofsEnabled = true
)
val evalContract = ContractEvaluator.verify(decls, vf, RealTransactionWrapper.ord(order))
EvaluatorV1.evalWithLogging(ctx, evalContract)
Eval.later(order),
isDApp = true
) match {
case Left(error) => (List.empty, Left(error))
case Right(ctx) =>
val evalContract = ContractEvaluator.verify(decls, vf, RealTransactionWrapper.ord(order))
EvaluatorV1.evalWithLogging(ctx, evalContract)
}

case ContractScript.ContractScriptImpl(_, DApp(_, _, _, None), _) =>
(List.empty, Verifier.verifyAsEllipticCurveSignature[Proven with Authorized](order) match {
case Right(_) => Right(TRUE)
case Left(_) => Right(FALSE)
})

case _ => (List.empty, "Unsupported script version".asLeft[EVALUATED])
}
}
22 changes: 22 additions & 0 deletions dex/src/test/scala/com/wavesplatform/dex/error/ProduceError.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
package com.wavesplatform.dex.error

import org.scalatest.matchers.{MatchResult, Matcher}

import scala.util.matching.Regex
import scala.util.{Left, Right}

class ProduceError(errorPattern: Regex) extends Matcher[Either[_, _]] {
override def apply(ei: Either[_, _]): MatchResult =
ei match {
case r @ Right(_) => MatchResult(matches = false, "expecting {0} to be Left and match: {1}", "got expected error", IndexedSeq(r, errorPattern))
case Left(l) =>
MatchResult(matches = errorPattern.findFirstIn(l.toString).isDefined,
"expecting {0} to match: {1}",
"got expected error",
IndexedSeq(l, errorPattern))
}
}

object ProduceError {
def produce(errorPattern: Regex): Matcher[Either[_, _]] = new ProduceError(errorPattern)
}
Loading

0 comments on commit a7d7446

Please sign in to comment.