Return archived events from /command/exercise (#3036)

* Adding `domain.ArchivedContract`

* Adding `domain.Contract`

* Returning archived and active contracts from `/command/exercise`

Improving integration tests, asserting values in the response JSON

* Updating documentation

* Do not populate workflowId if it is not provided

set it to empty string (default), it is optional in the Ledger API
This commit is contained in:
Leonid Shlyapnikov 2019-09-26 16:06:18 -04:00 committed by GitHub
parent bb309794db
commit 1fa170366d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 342 additions and 81 deletions

View File

@ -135,10 +135,14 @@ output, each contract formatted according to :doc:`lf-value-specification`::
"status": 200, "status": 200,
"result": [ "result": [
{ {
"observers": [],
"agreementText": "", "agreementText": "",
"contractId": "#237:0", "signatories": [
"Alice"
],
"contractId": "#489:0",
"templateId": { "templateId": {
"packageId": "cde2c40565fd8962eaebae7584ae89ba12d301d4c683189dccbbf0d0d67afc05", "packageId": "ac3a64908d9f6b4453329b3d7d8ddea44c83f4f5469de5f7ae19158c69bf8473",
"moduleName": "Iou", "moduleName": "Iou",
"entityName": "Iou" "entityName": "Iou"
}, },
@ -183,10 +187,14 @@ output::
{ {
"status": 200, "status": 200,
"result": { "result": {
"observers": [],
"agreementText": "", "agreementText": "",
"contractId": "#237:0", "signatories": [
"Alice"
],
"contractId": "#56:0",
"templateId": { "templateId": {
"packageId": "cde2c40565fd8962eaebae7584ae89ba12d301d4c683189dccbbf0d0d67afc05", "packageId": "ac3a64908d9f6b4453329b3d7d8ddea44c83f4f5469de5f7ae19158c69bf8473",
"moduleName": "Iou", "moduleName": "Iou",
"entityName": "Iou" "entityName": "Iou"
}, },
@ -209,7 +217,7 @@ POST http://localhost:44279/command/exercise
Exercise a choice on a contract. Exercise a choice on a contract.
``"contractId": "#237:0"`` is the value from the create output ``"contractId": "#56:0"`` is the value from the create output
application/json body:: application/json body::
{ {
@ -217,7 +225,7 @@ application/json body::
"moduleName": "Iou", "moduleName": "Iou",
"entityName": "Iou" "entityName": "Iou"
}, },
"contractId": "#237:0", "contractId": "#56:0",
"choice": "Iou_Transfer", "choice": "Iou_Transfer",
"argument": { "argument": {
"newOwner": "Alice" "newOwner": "Alice"
@ -230,27 +238,47 @@ output::
"status": 200, "status": 200,
"result": [ "result": [
{ {
"agreementText": "", "archived": {
"contractId": "#441:1", "workflowId": "Alice Workflow",
"templateId": { "contractId": "#56:0",
"packageId": "cde2c40565fd8962eaebae7584ae89ba12d301d4c683189dccbbf0d0d67afc05", "templateId": {
"moduleName": "Iou", "packageId": "ac3a64908d9f6b4453329b3d7d8ddea44c83f4f5469de5f7ae19158c69bf8473",
"entityName": "IouTransfer" "moduleName": "Iou",
}, "entityName": "Iou"
"witnessParties": [
"Alice"
],
"argument": {
"iou": {
"observers": [],
"issuer": "Alice",
"amount": "999.99",
"currency": "USD",
"owner": "Alice"
}, },
"newOwner": "Alice" "witnessParties": [
}, "Alice"
"workflowId": "Alice Workflow" ]
}
},
{
"active": {
"observers": [],
"agreementText": "",
"signatories": [
"Alice"
],
"contractId": "#301:1",
"templateId": {
"packageId": "ac3a64908d9f6b4453329b3d7d8ddea44c83f4f5469de5f7ae19158c69bf8473",
"moduleName": "Iou",
"entityName": "IouTransfer"
},
"witnessParties": [
"Alice"
],
"argument": {
"iou": {
"observers": [],
"issuer": "Alice",
"amount": "999.99",
"currency": "USD",
"owner": "Alice"
},
"newOwner": "Alice"
},
"workflowId": "Alice Workflow"
}
} }
] ]
} }

View File

@ -1,3 +1,9 @@
# HTTP JSON Service # HTTP JSON Service
See "HTTP JSON API Service" on docs.daml.com for usage information. See "HTTP JSON API Service" on docs.daml.com for usage information.
Documentation can also be found in the RST format:
- [HTTP JSON API Service](/docs/source/json-api/index.rst)
- [DAML-LF JSON Encoding](/docs/source/json-api/lf-value-specification.rst)
- [Search Query Language](/docs/source/json-api/search-query-language.rst)

View File

@ -8,8 +8,14 @@ import java.time.Instant
import com.digitalasset.api.util.TimeProvider import com.digitalasset.api.util.TimeProvider
import com.digitalasset.daml.lf.data.ImmArray.ImmArraySeq import com.digitalasset.daml.lf.data.ImmArray.ImmArraySeq
import com.digitalasset.http.CommandService.Error import com.digitalasset.http.CommandService.Error
import com.digitalasset.http.domain.{ActiveContract, CreateCommand, ExerciseCommand, JwtPayload} import com.digitalasset.http.domain.{
import com.digitalasset.http.util.ClientUtil.{uniqueCommandId, workflowIdFromParty} ActiveContract,
Contract,
CreateCommand,
ExerciseCommand,
JwtPayload
}
import com.digitalasset.http.util.ClientUtil.uniqueCommandId
import com.digitalasset.http.util.IdentifierConverters.refApiIdentifier import com.digitalasset.http.util.IdentifierConverters.refApiIdentifier
import com.digitalasset.http.util.{Commands, Transactions} import com.digitalasset.http.util.{Commands, Transactions}
import com.digitalasset.jwt.domain.Jwt import com.digitalasset.jwt.domain.Jwt
@ -19,6 +25,7 @@ import com.typesafe.scalalogging.StrictLogging
import scalaz.std.scalaFuture._ import scalaz.std.scalaFuture._
import scalaz.syntax.show._ import scalaz.syntax.show._
import scalaz.syntax.std.option._ import scalaz.syntax.std.option._
import scalaz.syntax.traverse._
import scalaz.{-\/, EitherT, Show, \/, \/-} import scalaz.{-\/, EitherT, Show, \/, \/-}
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -50,13 +57,13 @@ class CommandService(
@SuppressWarnings(Array("org.wartremover.warts.Any")) @SuppressWarnings(Array("org.wartremover.warts.Any"))
def exercise(jwt: Jwt, jwtPayload: JwtPayload, input: ExerciseCommand[lav1.value.Record]) def exercise(jwt: Jwt, jwtPayload: JwtPayload, input: ExerciseCommand[lav1.value.Record])
: Future[Error \/ ImmArraySeq[ActiveContract[lav1.value.Value]]] = { : Future[Error \/ ImmArraySeq[Contract[lav1.value.Value]]] = {
val et: EitherT[Future, Error, ImmArraySeq[ActiveContract[lav1.value.Value]]] = for { val et: EitherT[Future, Error, ImmArraySeq[Contract[lav1.value.Value]]] = for {
command <- EitherT.either(exerciseCommand(input)) command <- EitherT.either(exerciseCommand(input))
request = submitAndWaitRequest(jwtPayload, input.meta, command) request = submitAndWaitRequest(jwtPayload, input.meta, command)
response <- liftET(logResult('exercise, submitAndWaitForTransaction(jwt, request))) response <- liftET(logResult('exercise, submitAndWaitForTransaction(jwt, request)))
contracts <- EitherT.either(activeContracts(response)) contracts <- EitherT.either(contracts(response))
} yield contracts } yield contracts
et.run et.run
@ -98,8 +105,7 @@ class CommandService(
val maximumRecordTime: Instant = meta val maximumRecordTime: Instant = meta
.flatMap(_.maximumRecordTime) .flatMap(_.maximumRecordTime)
.getOrElse(ledgerEffectiveTime.plusNanos(defaultTimeToLive.toNanos)) .getOrElse(ledgerEffectiveTime.plusNanos(defaultTimeToLive.toNanos))
val workflowId: domain.WorkflowId = val workflowId: Option[domain.WorkflowId] = meta.flatMap(_.workflowId)
meta.flatMap(_.workflowId).getOrElse(workflowIdFromParty(jwtPayload.party))
val commandId: lar.CommandId = meta.flatMap(_.commandId).getOrElse(uniqueCommandId()) val commandId: lar.CommandId = meta.flatMap(_.commandId).getOrElse(uniqueCommandId())
Commands.submitAndWaitRequest( Commands.submitAndWaitRequest(
@ -133,16 +139,28 @@ class CommandService(
@SuppressWarnings(Array("org.wartremover.warts.Any")) @SuppressWarnings(Array("org.wartremover.warts.Any"))
private def activeContracts( private def activeContracts(
tx: lav1.transaction.Transaction): Error \/ ImmArraySeq[ActiveContract[lav1.value.Value]] = { tx: lav1.transaction.Transaction): Error \/ ImmArraySeq[ActiveContract[lav1.value.Value]] = {
import scalaz.syntax.traverse._
val workflowId = domain.WorkflowId.fromLedgerApi(tx) val workflowId = domain.WorkflowId.fromLedgerApi(tx)
Transactions Transactions
.decodeAllCreatedEvents(tx) .decodeAllCreatedEvents(tx)
.traverse(domain.ActiveContract.fromLedgerApi(workflowId)(_)) .traverse(ActiveContract.fromLedgerApi(workflowId)(_))
.leftMap(e => Error('activeContracts, e.shows)) .leftMap(e => Error('activeContracts, e.shows))
} }
private def contracts(response: lav1.command_service.SubmitAndWaitForTransactionResponse)
: Error \/ ImmArraySeq[Contract[lav1.value.Value]] =
response.transaction
.toRightDisjunction(Error('contracts, s"Received response without transaction: $response"))
.flatMap(contracts)
@SuppressWarnings(Array("org.wartremover.warts.Any"))
private def contracts(
tx: lav1.transaction.Transaction): Error \/ ImmArraySeq[Contract[lav1.value.Value]] = {
val workflowId = domain.WorkflowId.fromLedgerApi(tx)
tx.events.iterator
.to[ImmArraySeq]
.traverse(Contract.fromLedgerApi(workflowId)(_))
.leftMap(e => Error('contracts, e.shows))
}
} }
object CommandService { object CommandService {

View File

@ -66,7 +66,7 @@ class ContractsService(
jwt: Jwt, jwt: Jwt,
party: lar.Party, party: lar.Party,
templateId: Option[TemplateId.OptionalPkg], templateId: Option[TemplateId.OptionalPkg],
contractId: String): Future[Option[ActiveContract]] = contractId: domain.ContractId): Future[Option[ActiveContract]] =
for { for {
(as, _) <- search(jwt, party, templateIds(templateId), Map.empty) (as, _) <- search(jwt, party, templateIds(templateId), Map.empty)
a = findByContractId(contractId)(as) a = findByContractId(contractId)(as)
@ -75,8 +75,9 @@ class ContractsService(
private def templateIds(a: Option[TemplateId.OptionalPkg]): Set[TemplateId.OptionalPkg] = private def templateIds(a: Option[TemplateId.OptionalPkg]): Set[TemplateId.OptionalPkg] =
a.toList.toSet a.toList.toSet
private def findByContractId(k: String)(as: Seq[ActiveContract]): Option[ActiveContract] = private def findByContractId(k: domain.ContractId)(
as.find(x => (x.contractId: String) == k) as: Seq[ActiveContract]): Option[ActiveContract] =
as.find(x => (x.contractId: domain.ContractId) == k)
def search(jwt: Jwt, jwtPayload: JwtPayload, request: GetActiveContractsRequest): Future[Result] = def search(jwt: Jwt, jwtPayload: JwtPayload, request: GetActiveContractsRequest): Future[Result] =
search(jwt, jwtPayload.party, request.templateIds, request.query) search(jwt, jwtPayload.party, request.templateIds, request.query)

View File

@ -85,12 +85,12 @@ class Endpoints(
.leftMap(e => InvalidUserInput(e.shows)) .leftMap(e => InvalidUserInput(e.shows))
): ET[domain.ExerciseCommand[lav1.value.Record]] ): ET[domain.ExerciseCommand[lav1.value.Record]]
as <- eitherT( cs <- eitherT(
handleFutureFailure(commandService.exercise(jwt, jwtPayload, cmd)) handleFutureFailure(commandService.exercise(jwt, jwtPayload, cmd))
): ET[ImmArraySeq[domain.ActiveContract[lav1.value.Value]]] ): ET[ImmArraySeq[domain.Contract[lav1.value.Value]]]
jsVal <- either( jsVal <- either(
as.traverse(a => encoder.encodeV(a)) cs.traverse(a => encoder.encodeV(a))
.leftMap(e => ServerError(e.shows)) .leftMap(e => ServerError(e.shows))
.flatMap(as => encodeList(as))): ET[JsValue] .flatMap(as => encodeList(as))): ET[JsValue]

View File

@ -30,22 +30,32 @@ object domain {
} }
} }
case class JwtPayload(ledgerId: lar.LedgerId, applicationId: lar.ApplicationId, party: lar.Party) case class JwtPayload(ledgerId: lar.LedgerId, applicationId: lar.ApplicationId, party: Party)
case class TemplateId[+PkgId](packageId: PkgId, moduleName: String, entityName: String) case class TemplateId[+PkgId](packageId: PkgId, moduleName: String, entityName: String)
case class Contract[+LfV](value: ArchivedContract \/ ActiveContract[LfV])
case class ActiveContract[+LfV]( case class ActiveContract[+LfV](
workflowId: Option[WorkflowId], workflowId: Option[WorkflowId],
contractId: String, contractId: ContractId,
templateId: TemplateId.RequiredPkg, templateId: TemplateId.RequiredPkg,
key: Option[LfV], key: Option[LfV],
argument: LfV, argument: LfV,
witnessParties: Seq[String], witnessParties: Seq[Party],
signatories: Seq[Party],
observers: Seq[Party],
agreementText: String) agreementText: String)
case class ArchivedContract(
workflowId: Option[WorkflowId],
contractId: ContractId,
templateId: TemplateId.RequiredPkg,
witnessParties: Seq[Party])
case class ContractLookupRequest[+LfV]( case class ContractLookupRequest[+LfV](
ledgerId: Option[String], ledgerId: Option[String],
id: (TemplateId.OptionalPkg, LfV) \/ (Option[TemplateId.OptionalPkg], String)) id: (TemplateId.OptionalPkg, LfV) \/ (Option[TemplateId.OptionalPkg], ContractId))
case class GetActiveContractsRequest( case class GetActiveContractsRequest(
templateIds: Set[TemplateId.OptionalPkg], templateIds: Set[TemplateId.OptionalPkg],
@ -54,6 +64,14 @@ object domain {
type WorkflowIdTag = lar.WorkflowIdTag type WorkflowIdTag = lar.WorkflowIdTag
type WorkflowId = lar.WorkflowId type WorkflowId = lar.WorkflowId
type ContractIdTag = lar.ContractIdTag
type ContractId = lar.ContractId
val ContractId = lar.ContractId
type PartyTag = lar.PartyTag
type Party = lar.Party
val Party = lar.Party
object WorkflowId { object WorkflowId {
def apply(s: String): WorkflowId = lar.WorkflowId(s) def apply(s: String): WorkflowId = lar.WorkflowId(s)
@ -77,21 +95,52 @@ object domain {
TemplateId(in.packageId, in.moduleName, in.entityName) TemplateId(in.packageId, in.moduleName, in.entityName)
} }
object Contract {
def fromLedgerApi(workflowId: Option[WorkflowId])(
event: lav1.event.Event): Error \/ Contract[lav1.value.Value] = event.event match {
case lav1.event.Event.Event.Created(created) =>
ActiveContract.fromLedgerApi(workflowId)(created).map(a => Contract(\/-(a)))
case lav1.event.Event.Event.Archived(archived) =>
ArchivedContract.fromLedgerApi(workflowId)(archived).map(a => Contract(-\/(a)))
case lav1.event.Event.Event.Empty =>
val errorMsg = s"Expected either Created or Archived event, got: Empty"
-\/(Error('Contract_fromLedgerApi, errorMsg))
}
implicit val covariant: Traverse[Contract] = new Traverse[Contract] {
override def map[A, B](fa: Contract[A])(f: A => B): Contract[B] = {
val valueB: ArchivedContract \/ ActiveContract[B] = fa.value.map(a => a.map(f))
Contract(valueB)
}
override def traverseImpl[G[_]: Applicative, A, B](fa: Contract[A])(
f: A => G[B]): G[Contract[B]] = {
val valueB: G[ArchivedContract \/ ActiveContract[B]] = fa.value.traverse(a => a.traverse(f))
valueB.map(x => Contract[B](x))
}
}
}
def boxedRecord(a: lav1.value.Record): lav1.value.Value =
lav1.value.Value(lav1.value.Value.Sum.Record(a))
object ActiveContract { object ActiveContract {
def fromLedgerApi(workflowId: Option[WorkflowId])( def fromLedgerApi(workflowId: Option[WorkflowId])(
in: lav1.event.CreatedEvent): Error \/ ActiveContract[lav1.value.Value] = in: lav1.event.CreatedEvent): Error \/ ActiveContract[lav1.value.Value] =
for { for {
templateId <- in.templateId required "templateId" templateId <- in.templateId required "templateId"
argument <- in.createArguments required "createArguments" argument <- in.createArguments required "createArguments"
boxedArgument = lav1.value.Value(lav1.value.Value.Sum.Record(argument))
} yield } yield
ActiveContract( ActiveContract(
workflowId = workflowId, workflowId = workflowId,
contractId = in.contractId, contractId = ContractId(in.contractId),
templateId = TemplateId fromLedgerApi templateId, templateId = TemplateId fromLedgerApi templateId,
key = in.contractKey, key = in.contractKey,
argument = boxedArgument, argument = boxedRecord(argument),
witnessParties = in.witnessParties, witnessParties = Party.subst(in.witnessParties),
signatories = Party.subst(in.signatories),
observers = Party.subst(in.observers),
agreementText = in.agreementText getOrElse "" agreementText = in.agreementText getOrElse ""
) )
@ -123,6 +172,20 @@ object domain {
} }
} }
object ArchivedContract {
def fromLedgerApi(workflowId: Option[WorkflowId])(
in: lav1.event.ArchivedEvent): Error \/ ArchivedContract =
for {
templateId <- in.templateId required "templateId"
} yield
ArchivedContract(
workflowId = workflowId,
contractId = ContractId(in.contractId),
templateId = TemplateId fromLedgerApi templateId,
witnessParties = Party.subst(in.witnessParties)
)
}
object ContractLookupRequest { object ContractLookupRequest {
implicit val covariant: Traverse[ContractLookupRequest] = new Traverse[ContractLookupRequest] { implicit val covariant: Traverse[ContractLookupRequest] = new Traverse[ContractLookupRequest] {
override def map[A, B](fa: ContractLookupRequest[A])(f: A => B) = override def map[A, B](fa: ContractLookupRequest[A])(f: A => B) =

View File

@ -24,16 +24,16 @@ object JsonProtocol extends DefaultJsonProtocol {
implicit val WorkflowIdFormat: JsonFormat[domain.WorkflowId] = implicit val WorkflowIdFormat: JsonFormat[domain.WorkflowId] =
taggedJsonFormat[String, domain.WorkflowIdTag] taggedJsonFormat[String, domain.WorkflowIdTag]
implicit val PartyFormat: JsonFormat[lar.Party] = implicit val PartyFormat: JsonFormat[domain.Party] =
taggedJsonFormat[String, lar.PartyTag] taggedJsonFormat[String, domain.PartyTag]
implicit val CommandIdFormat: JsonFormat[lar.CommandId] = implicit val CommandIdFormat: JsonFormat[lar.CommandId] =
taggedJsonFormat[String, lar.CommandIdTag] taggedJsonFormat[String, lar.CommandIdTag]
implicit val ChoiceFormat: JsonFormat[lar.Choice] = taggedJsonFormat[String, lar.ChoiceTag] implicit val ChoiceFormat: JsonFormat[lar.Choice] = taggedJsonFormat[String, lar.ChoiceTag]
implicit val ContractIdFormat: JsonFormat[lar.ContractId] = implicit val ContractIdFormat: JsonFormat[domain.ContractId] =
taggedJsonFormat[String, lar.ContractIdTag] taggedJsonFormat[String, domain.ContractIdTag]
object LfValueCodec object LfValueCodec
extends ApiCodecCompressed[AbsoluteContractId]( extends ApiCodecCompressed[AbsoluteContractId](
@ -69,7 +69,9 @@ object JsonProtocol extends DefaultJsonProtocol {
case (Some(templateId), Some(key), None) => case (Some(templateId), Some(key), None) =>
-\/((templateId.convertTo[domain.TemplateId.OptionalPkg], key)) -\/((templateId.convertTo[domain.TemplateId.OptionalPkg], key))
case (otid, None, Some(contractId)) => case (otid, None, Some(contractId)) =>
\/-((otid map (_.convertTo[domain.TemplateId.OptionalPkg]), contractId.convertTo[String])) val a = otid map (_.convertTo[domain.TemplateId.OptionalPkg])
val b = contractId.convertTo[domain.ContractId]
\/-((a, b))
case (None, Some(_), None) => case (None, Some(_), None) =>
deserializationError( deserializationError(
"ContractLookupRequest requires key to be accompanied by a templateId") "ContractLookupRequest requires key to be accompanied by a templateId")
@ -80,8 +82,36 @@ object JsonProtocol extends DefaultJsonProtocol {
case _ => deserializationError("ContractLookupRequest must be an object") case _ => deserializationError("ContractLookupRequest must be an object")
} }
implicit val ContractFormat: RootJsonFormat[domain.Contract[JsValue]] =
new RootJsonFormat[domain.Contract[JsValue]] {
private val archivedKey = "archived"
private val activeKey = "active"
override def read(json: JsValue): domain.Contract[JsValue] = json match {
case JsObject(fields) =>
fields.toList match {
case List((`archivedKey`, archived)) =>
domain.Contract(-\/(ArchivedContractFormat.read(archived)))
case List((`activeKey`, active)) =>
domain.Contract(\/-(ActiveContractFormat.read(active)))
case _ =>
deserializationError(
s"Contract must be either {$archivedKey: obj} or {$activeKey: obj}, got: $fields")
}
case _ => deserializationError("Contract must be an object")
}
override def write(obj: domain.Contract[JsValue]): JsValue = obj.value match {
case -\/(archived) => JsObject(archivedKey -> ArchivedContractFormat.write(archived))
case \/-(active) => JsObject(activeKey -> ActiveContractFormat.write(active))
}
}
implicit val ActiveContractFormat: RootJsonFormat[domain.ActiveContract[JsValue]] = implicit val ActiveContractFormat: RootJsonFormat[domain.ActiveContract[JsValue]] =
jsonFormat7(domain.ActiveContract.apply[JsValue]) jsonFormat9(domain.ActiveContract.apply[JsValue])
implicit val ArchivedContractFormat: RootJsonFormat[domain.ArchivedContract] =
jsonFormat4(domain.ArchivedContract.apply)
private val templatesKey = "%templates" private val templatesKey = "%templates"

View File

@ -8,14 +8,8 @@ import akka.stream.Materializer
import akka.stream.scaladsl.{Sink, Source} import akka.stream.scaladsl.{Sink, Source}
import akka.{Done, NotUsed} import akka.{Done, NotUsed}
import com.digitalasset.api.util.TimeProvider import com.digitalasset.api.util.TimeProvider
import com.digitalasset.http.domain
import com.digitalasset.http.util.FutureUtil.toFuture import com.digitalasset.http.util.FutureUtil.toFuture
import com.digitalasset.ledger.api.refinements.ApiTypes.{ import com.digitalasset.ledger.api.refinements.ApiTypes.{ApplicationId, CommandId, Party}
ApplicationId,
CommandId,
Party,
WorkflowId
}
import com.digitalasset.ledger.api.v1.ledger_offset.LedgerOffset import com.digitalasset.ledger.api.v1.ledger_offset.LedgerOffset
import com.digitalasset.ledger.api.v1.transaction.Transaction import com.digitalasset.ledger.api.v1.transaction.Transaction
import com.digitalasset.ledger.api.v1.transaction_filter.{Filters, TransactionFilter} import com.digitalasset.ledger.api.v1.transaction_filter.{Filters, TransactionFilter}
@ -65,8 +59,5 @@ object ClientUtil {
def uniqueId(): String = UUID.randomUUID.toString def uniqueId(): String = UUID.randomUUID.toString
def workflowIdFromParty(p: Party): domain.WorkflowId =
WorkflowId(s"${Party.unwrap(p)} Workflow")
def uniqueCommandId(): CommandId = CommandId(uniqueId()) def uniqueCommandId(): CommandId = CommandId(uniqueId())
} }

View File

@ -53,16 +53,18 @@ object Commands {
def submitAndWaitRequest( def submitAndWaitRequest(
ledgerId: lar.LedgerId, ledgerId: lar.LedgerId,
applicationId: lar.ApplicationId, applicationId: lar.ApplicationId,
workflowId: domain.WorkflowId, workflowId: Option[domain.WorkflowId],
commandId: lar.CommandId, commandId: lar.CommandId,
ledgerEffectiveTime: Instant, ledgerEffectiveTime: Instant,
maximumRecordTime: Instant, maximumRecordTime: Instant,
party: lar.Party, party: lar.Party,
command: lav1.commands.Command.Command): lav1.command_service.SubmitAndWaitRequest = { command: lav1.commands.Command.Command): lav1.command_service.SubmitAndWaitRequest = {
val workflowIdStr: String = workflowId.map(domain.WorkflowId.unwrap).getOrElse("")
val commands = lav1.commands.Commands( val commands = lav1.commands.Commands(
ledgerId = lar.LedgerId.unwrap(ledgerId), ledgerId = lar.LedgerId.unwrap(ledgerId),
workflowId = domain.WorkflowId.unwrap(workflowId), workflowId = workflowIdStr,
applicationId = lar.ApplicationId.unwrap(applicationId), applicationId = lar.ApplicationId.unwrap(applicationId),
commandId = lar.CommandId.unwrap(commandId), commandId = lar.CommandId.unwrap(commandId),
party = lar.Party.unwrap(party), party = lar.Party.unwrap(party),

View File

@ -5,6 +5,8 @@ package com.digitalasset.http
import com.digitalasset.ledger.api.{v1 => lav1} import com.digitalasset.ledger.api.{v1 => lav1}
import org.scalacheck.Gen import org.scalacheck.Gen
import scalaz.{-\/, \/, \/-}
import spray.json.{JsString, JsValue}
object Generators { object Generators {
def genApiIdentifier: Gen[lav1.value.Identifier] = def genApiIdentifier: Gen[lav1.value.Identifier] =
@ -51,4 +53,52 @@ object Generators {
implicit object OptionalPackageIdGen extends PackageIdGen[Option[String]] { implicit object OptionalPackageIdGen extends PackageIdGen[Option[String]] {
override def gen: Gen[Option[String]] = Gen.option(RequiredPackageIdGen.gen) override def gen: Gen[Option[String]] = Gen.option(RequiredPackageIdGen.gen)
} }
def workflowIdGen: Gen[domain.WorkflowId] = Gen.identifier.map(domain.WorkflowId(_))
def contractIdGen: Gen[domain.ContractId] = Gen.identifier.map(domain.ContractId(_))
def partyGen: Gen[domain.Party] = Gen.identifier.map(domain.Party(_))
def scalazEitherGen[A, B](a: Gen[A], b: Gen[B]): Gen[\/[A, B]] =
Gen.oneOf(a.map(-\/(_)), b.map(\/-(_)))
def contractGen: Gen[domain.Contract[JsValue]] =
scalazEitherGen(archivedContractGen, activeContractGen).map(domain.Contract(_))
def activeContractGen: Gen[domain.ActiveContract[JsValue]] =
for {
workflowId <- Gen.option(workflowIdGen)
contractId <- contractIdGen
templateId <- Generators.genDomainTemplateId
key <- Gen.option(Gen.identifier.map(JsString(_)))
argument <- Gen.identifier.map(JsString(_))
witnessParties <- Gen.listOf(partyGen)
signatories <- Gen.listOf(partyGen)
observers <- Gen.listOf(partyGen)
agreementText <- Gen.identifier
} yield
domain.ActiveContract[JsValue](
workflowId = workflowId,
contractId = contractId,
templateId = templateId,
key = key,
argument = argument,
witnessParties = witnessParties,
signatories = signatories,
observers = observers,
agreementText = agreementText
)
def archivedContractGen: Gen[domain.ArchivedContract] =
for {
workflowId <- Gen.option(workflowIdGen)
contractId <- contractIdGen
templateId <- Generators.genDomainTemplateId
witnessParties <- Gen.listOf(partyGen)
} yield
domain.ArchivedContract(
workflowId = workflowId,
contractId = contractId,
templateId = templateId,
witnessParties = witnessParties,
)
} }

View File

@ -271,8 +271,15 @@ class HttpServiceIntegrationTest
inside(exerciseOutput) { inside(exerciseOutput) {
case JsObject(fields) => case JsObject(fields) =>
inside(fields.get("result")) { inside(fields.get("result")) {
case Some(JsArray(Vector(activeContract: JsObject))) => case Some(JsArray(Vector(contract1: JsObject, contract2: JsObject))) =>
assertActiveContract(decoder, activeContract, create, exercise) inside(contract1.fields.toList) {
case List(("archived", archived: JsObject)) =>
assertArchivedContract(archived, contractId)
}
inside(contract2.fields.toList) {
case List(("active", active: JsObject)) =>
assertActiveContract(decoder, active, create, exercise)
}
} }
} }
} }
@ -296,20 +303,35 @@ class HttpServiceIntegrationTest
}: Future[Assertion] }: Future[Assertion]
} }
private def assertArchivedContract(
jsObject: JsObject,
contractId: domain.ContractId): Assertion = {
import JsonProtocol._
val archived = SprayJson.decode[domain.ArchivedContract](jsObject).valueOr(e => fail(e.shows))
archived.contractId shouldBe contractId
}
private def assertActiveContract( private def assertActiveContract(
decoder: DomainJsonDecoder, decoder: DomainJsonDecoder,
jsObject: JsObject, jsObject: JsObject,
create: domain.CreateCommand[v.Record], create: domain.CreateCommand[v.Record],
exercise: domain.ExerciseCommand[v.Record]): Assertion = { exercise: domain.ExerciseCommand[v.Record]): Assertion = {
// TODO(Leo): check the jsObject.argument is the same as createCommand.argument val expectedContractFields: Seq[v.RecordField] = create.argument.fields
println(s"------- jsObject: $jsObject") val expectedNewOwner: v.Value = exercise.argument.fields.headOption
println(s"------- create: $create") .flatMap(_.value)
println(s"------- exercise: $exercise") .getOrElse(fail("Cannot extract expected newOwner"))
inside(jsObject.fields.get("argument")) { val active = decoder.decodeV[domain.ActiveContract](jsObject).valueOr(e => fail(e.shows))
case Some(JsObject(fields)) => inside(active.argument.sum.record.map(_.fields)) {
fields.size shouldBe (exercise.argument.fields.size + 1) // +1 for the original "iou" from create case Some(
Seq(
v.RecordField("iou", Some(contractRecord)),
v.RecordField("newOwner", Some(newOwner)))) =>
val contractFields: Seq[v.RecordField] =
contractRecord.sum.record.map(_.fields).getOrElse(Seq.empty)
(contractFields: Seq[v.RecordField]) shouldBe (expectedContractFields: Seq[v.RecordField])
(newOwner: v.Value) shouldBe (expectedNewOwner: v.Value)
} }
} }
@ -454,13 +476,13 @@ class HttpServiceIntegrationTest
} }
} }
private def getContractId(output: JsValue): lar.ContractId = private def getContractId(output: JsValue): domain.ContractId =
inside(output) { inside(output) {
case JsObject(topFields) => case JsObject(topFields) =>
inside(topFields.get("result")) { inside(topFields.get("result")) {
case Some(JsObject(fields)) => case Some(JsObject(fields)) =>
inside(fields.get("contractId")) { inside(fields.get("contractId")) {
case Some(JsString(contractId)) => lar.ContractId(contractId) case Some(JsString(contractId)) => domain.ContractId(contractId)
} }
} }
} }

View File

@ -0,0 +1,48 @@
// Copyright (c) 2019 The DAML Authors. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.digitalasset.http.json
import com.digitalasset.http.Generators.contractGen
import com.digitalasset.http.domain
import org.scalatest.prop.GeneratorDrivenPropertyChecks
import org.scalatest.{FreeSpec, Inside, Matchers}
import scalaz.{\/, \/-}
import spray.json.{JsObject, JsValue}
class JsonProtocolTest
extends FreeSpec
with Matchers
with Inside
with GeneratorDrivenPropertyChecks {
import JsonProtocol._
implicit override val generatorDrivenConfig: PropertyCheckConfiguration =
PropertyCheckConfiguration(minSuccessful = 100)
"domain.Contract" - {
"can be serialized to JSON" in forAll(contractGen) { contract =>
inside(SprayJson.encode(contract)) {
case \/-(JsObject(fields)) =>
inside(fields.toList) {
case List(("archived", JsObject(_))) =>
case List(("active", JsObject(_))) =>
}
}
}
"can be serialized and deserialized back to the same object" in forAll(contractGen) {
contract0 =>
val actual: SprayJson.Error \/ domain.Contract[JsValue] = for {
jsValue <- SprayJson.encode(contract0)
contract <- SprayJson.decode[domain.Contract[JsValue]](jsValue)
} yield contract
inside(actual) {
case \/-(contract1) => contract1 shouldBe contract0
}
}
}
}

View File

@ -9,6 +9,8 @@ This page contains release notes for the SDK.
HEAD — ongoing HEAD — ongoing
-------------- --------------
+ [JSON API - Experimental] Returning archived and active contracts from ``/command/exercise``
enpoint. See `issue #2925 <https://github.com/digital-asset/daml/issues/2925>`_.
+ [JSON API - Experimental] Flattening the output of the ``/contracts/search`` endpoint. + [JSON API - Experimental] Flattening the output of the ``/contracts/search`` endpoint.
The endpoint returns ``ActiveContract`` objects without ``GetActiveContractsResponse`` wrappers. The endpoint returns ``ActiveContract`` objects without ``GetActiveContractsResponse`` wrappers.
See `issue #2987 <https://github.com/digital-asset/daml/pull/2987>`_. See `issue #2987 <https://github.com/digital-asset/daml/pull/2987>`_.