ledger-api-test-tool: Port LotsOfPartiesIT. (#3328)

* ledger-api-test-tool: Port LotsOfPartiesIT.

* ledger-api-test-tool: Run LotsOfPartiesIT with multiple participants.

Odds of me getting this wrong: 80%. Let's find out!

* ledger-api-test-tool: More underscores, methinks.

* ledger-api-test-tool: Improve the readability of the LotsOfParties test.

Instead of all the functional magic, just do things twice. It's even
shorter in some cases.

* ledger-api-test-tool: Not so many parties. :-(

* sandbox: Extract duplication between conformance test definitions.

* sandbox: Run the LotsOfPartiesIT conformance test separately.

* Revert "ledger-api-test-tool: Not so many parties. :-("

This reverts commit 8039a7cc9b.

* sandbox: Add the copyright header to sandbox_conformance_test.bzl.

* Run `./fmt.sh`.

* ledger-api-test-tool: Generalize `sandbox_conformance_test`.

I want to use it with the DAML-on-X reference v2.

* api-server-damlonx/reference-v2: Use `server_conformance_test`.

* api-server-damlonx/reference-v2: Run LotsOfPartiesIT separately.
This commit is contained in:
Samir Talwar 2019-11-05 20:54:21 +01:00 committed by mergify[bot]
parent ac8bf9f062
commit fddaeb1c4a
7 changed files with 324 additions and 240 deletions

View File

@ -8,6 +8,7 @@ load(
load(
"//ledger/ledger-api-test-tool:conformance.bzl",
"conformance_test",
"server_conformance_test",
)
da_scala_binary(
@ -62,38 +63,51 @@ da_scala_binary(
],
)
conformance_test(
SERVERS = {
"h2database": {
"binary": ":reference-v2",
"server_args": [
"--port=6865",
# "db_close_delay=-1" is needed so that the in-memory database is not closed
# (and therefore lost) after the flyway migration
"--jdbc-url=jdbc:h2:mem:daml_on_x;db_close_delay=-1;db_close_on_exit=false",
],
"test_tool_args": [
"--timeout-scale-factor=10",
],
},
"postgres": {
"binary": ":ephemeral-postgres-reference-server",
"extra_data": [
"@postgresql_dev_env//:all",
"@postgresql_dev_env//:createdb",
"@postgresql_dev_env//:initdb",
"@postgresql_dev_env//:pg_ctl",
],
"server_args": [
"--port=6865",
],
"test_tool_args": [
"--timeout-scale-factor=10",
],
},
}
server_conformance_test(
name = "conformance-test",
extra_data = [
"@postgresql_dev_env//:all",
"@postgresql_dev_env//:createdb",
"@postgresql_dev_env//:initdb",
"@postgresql_dev_env//:pg_ctl",
],
server = ":ephemeral-postgres-reference-server",
server_args = [
"--port 6865",
],
servers = SERVERS,
test_tool_args = [
"--all-tests",
"--exclude TimeIT",
"--timeout-scale-factor=10",
"--exclude=LotsOfPartiesIT",
"--exclude=TimeIT",
],
)
conformance_test(
name = "conformance-test-h2database",
server = ":reference-v2",
server_args = [
"--port 6865",
# db_close_delay=-1 is needed so that the in-memory database is not closed (and therefore lost)
# after the flyway migration
"--jdbc-url jdbc:h2:mem:daml_on_x;db_close_delay=-1;db_close_on_exit=false",
],
server_conformance_test(
name = "conformance-test-lots-of-parties",
servers = SERVERS,
test_tool_args = [
"--all-tests",
"--exclude TimeIT",
"--timeout-scale-factor=10",
"--include=LotsOfPartiesIT",
],
)
@ -105,11 +119,11 @@ conformance_test(
],
server = ":reference-v2",
server_args = [
"--port 6865",
# db_close_delay=-1 is needed so that the in-memory database is not closed (and therefore lost)
# after the flyway migration
"--jdbc-url jdbc:h2:mem:daml_on_x;db_close_delay=-1;db_close_on_exit=false",
"--extra-participant second-participant,6866,jdbc:h2:mem:daml_on_x2;db_close_delay=-1;db_close_on_exit=false",
"--port=6865",
# "db_close_delay=-1" is needed so that the in-memory database is not closed
# (and therefore lost) after the flyway migration
"--jdbc-url=jdbc:h2:mem:daml_on_x;db_close_delay=-1;db_close_on_exit=false",
"--extra-participant=second-participant,6866,jdbc:h2:mem:daml_on_x2;db_close_delay=-1;db_close_on_exit=false",
],
test_tool_args = [
"--include=SemanticTests,TransactionServiceIT",

View File

@ -1,129 +0,0 @@
// Copyright (c) 2019 The DAML Authors. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.digitalasset.platform.tests.integration.ledger.api
import akka.stream.scaladsl.{Sink, Source}
import com.digitalasset.ledger.api.testing.utils.SuiteResourceManagementAroundAll
import com.digitalasset.ledger.api.v1.commands.CreateCommand
import com.digitalasset.ledger.api.v1.ledger_offset.LedgerOffset
import com.digitalasset.ledger.api.v1.transaction.Transaction
import com.digitalasset.ledger.api.v1.transaction_filter.{Filters, TransactionFilter}
import com.digitalasset.ledger.api.v1.value.Record
import com.digitalasset.platform.apitesting.{LedgerContext, MultiLedgerFixture, TestTemplateIds}
import com.digitalasset.platform.participant.util.ValueConversions._
import com.digitalasset.util.Ctx
import org.scalatest.concurrent.AsyncTimeLimitedTests
import org.scalatest.{Assertion, AsyncWordSpec, Matchers, OptionValues}
import scala.collection.breakOut
import scala.concurrent.Future
class LotsOfPartiesIT
extends AsyncWordSpec
with MultiLedgerFixture
with SuiteResourceManagementAroundAll
with AsyncTimeLimitedTests
with Matchers
with OptionValues {
protected lazy val testTemplateIds = new TestTemplateIds(config)
protected lazy val templateIds = testTemplateIds.templateIds
private val numParties = 1024
private val allParties: List[String] =
Range.inclusive(1, numParties).map(i => s"party$i")(breakOut)
private val submittingParty :: observerParties = allParties
override protected def config: Config =
Config.default.withParties(submittingParty, observerParties: _*)
private val commandId = this.getClass.getSimpleName
"lots of registered parties" when {
"submitting commands" should {
"submit successfully and track the completion" in allFixtures { c =>
val giver = "giver" -> submittingParty.asParty
val observers = "observers" -> observerParties.map(_.asParty).asList
val cmd =
c.command(
commandId,
submittingParty,
List(
CreateCommand(
Some(templateIds.withObservers),
Some(Record(
Some(templateIds.withObservers),
List(giver, observers).asRecordFields))).wrap)
)
for {
cc <- c.commandClient()
tracker <- cc.trackCommands[Unit](List(submittingParty))
result <- Source.single(Ctx.unit(cmd)).via(tracker).runWith(Sink.head)
} yield {
result.value.status.value should have('code (0))
}
}
}
"reading transactions" should {
"see the transaction in a single multi-party subscription" in allFixtures { c =>
for {
tx <- getTx(c, allParties)
} yield {
assertExpectedWitnesses(createdEventWitnesses(tx))
}
}
"see the transaction in multiple single-party subscriptions" in allFixtures { c =>
for {
txs <- Future.sequence(allParties.map(p => getTx(c, List(p))))
} yield {
assertExpectedWitnesses(txs.flatMap(tx => createdEventWitnesses(tx)))
}
}
}
"reading ACS" should {
"see the contract in a single multi-party subscription" in allFixtures { c =>
for {
responses <- c.acsClient
.getActiveContracts(filterForParties(allParties))
.runWith(Sink.seq)
} yield {
val contracts = responses.flatMap(_.activeContracts)
contracts should have length 1
assertExpectedWitnesses(contracts.headOption.value.witnessParties)
}
}
"see the contract in multiple single-party subscriptions" in allFixtures { c =>
for {
responses <- Future.sequence(allParties.map(p =>
c.acsClient.getActiveContracts(filterForParties(List(p))).runWith(Sink.seq)))
} yield {
val contracts = responses.flatten.flatMap(_.activeContracts)
assertExpectedWitnesses(contracts.flatMap(_.witnessParties))
}
}
}
}
private def assertExpectedWitnesses(witnesses: Seq[String]): Assertion = {
witnesses should have length numParties.toLong
witnesses should contain theSameElementsAs allParties
}
private def createdEventWitnesses(tx: Transaction): Seq[String] = {
tx.events should have length 1
tx.events.headOption.value.event.created.value.witnessParties
}
private def getTx(c: LedgerContext, parties: List[String]): Future[Transaction] = {
c.transactionClient
.getTransactions(
LedgerOffset(LedgerOffset.Value.Boundary(LedgerOffset.LedgerBoundary.LEDGER_BEGIN)),
None,
filterForParties(parties))
.runWith(Sink.head)
}
private def filterForParties(parties: List[String]) = {
TransactionFilter(parties.map(_ -> Filters())(breakOut))
}
}

View File

@ -26,3 +26,14 @@ def conformance_test(name, server, server_args = [], extra_data = [], ports = [6
"exclusive",
],
) if not is_windows else None
def server_conformance_test(name, servers, server_args = [], test_tool_args = []):
for server_name, server in servers.items():
test_name = "-".join([segment for segment in [name, server_name] if segment])
conformance_test(
name = test_name,
extra_data = server.get("extra_data", []),
server = server["binary"],
server_args = server["server_args"] + server_args,
test_tool_args = server["test_tool_args"] + test_tool_args,
)

View File

@ -11,6 +11,7 @@ import com.digitalasset.ledger.api.v1.transaction.{Transaction, TransactionTree,
import com.digitalasset.ledger.test_stable.Test.AgreementFactory
import com.digitalasset.ledger.test_stable.Test.AgreementFactory._
import io.grpc.{Status, StatusException, StatusRuntimeException}
import scalaz.{@@, Tag}
import scala.concurrent.duration.DurationInt
import scala.concurrent.{ExecutionContext, Future}
@ -126,6 +127,14 @@ private[testtool] abstract class LedgerTestSuite(val session: LedgerSession) {
}
}
implicit def diffShowTag[A, T](implicit diffShowA: DiffShow[A]): DiffShow[A @@ T] =
new DiffShow[A @@ T] {
override def show(t: A @@ T): String = diffShowA.show(Tag.unwrap(t))
override def diff(left: A @@ T, right: A @@ T): Comparison =
diffShowA.diff(Tag.unwrap(left), Tag.unwrap(right))
}
implicit def diffShowSeq[T](implicit diffShowT: DiffShow[T]): DiffShow[Seq[T]] =
new DiffShow[Seq[T]] {
override def show(t: Seq[T]): String = t.toString

View File

@ -0,0 +1,194 @@
// Copyright (c) 2019 The DAML Authors. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.ledger.api.testtool.tests
import com.daml.ledger.api.testtool.infrastructure.participant.ParticipantTestContext
import com.daml.ledger.api.testtool.infrastructure.{LedgerSession, LedgerTest, LedgerTestSuite}
import com.digitalasset.ledger.api.v1.event.CreatedEvent
import com.digitalasset.ledger.api.v1.transaction.Transaction
import com.digitalasset.ledger.client.binding.Primitive.{ContractId, Party}
import com.digitalasset.ledger.test_stable.Test.WithObservers
import scala.concurrent.Future
final class LotsOfParties(session: LedgerSession) extends LedgerTestSuite(session) {
type Parties = Set[Party]
type PartyMap[T] = Map[Party, T]
private val partyCount = 1024
private[this] val seeTransactionsInMultipleSinglePartySubscriptions = LedgerTest(
"LOPseeTransactionsInMultipleSinglePartySubscriptions",
"Observers should see transactions in multiple single-party subscriptions"
) { context =>
for {
alpha <- context.participant()
beta <- context.participant()
giver <- alpha.allocateParty()
alphaObservers <- alpha.allocateParties(partyCount / 2 - 1)
betaObservers <- beta.allocateParties(partyCount / 2)
observers = alphaObservers ++ betaObservers
alphaParties = giver +: alphaObservers
contractId <- alpha.create(giver, WithObservers(giver, observers))
alphaTransactionsByParty <- transactionsForEachParty(alpha, alphaParties)
betaTransactionsByParty <- transactionsForEachParty(beta, betaObservers)
} yield {
assertWitnessesOfSinglePartySubscriptions(
alphaParties.toSet,
contractId,
activeContractsFrom(alphaTransactionsByParty))
assertWitnessesOfSinglePartySubscriptions(
betaObservers.toSet,
contractId,
activeContractsFrom(betaTransactionsByParty))
}
}
private[this] val seeTransactionsInSingleMultiPartySubscription = LedgerTest(
"LOPseeTransactionsInSingleMultiPartySubscription",
"Observers should see transactions in a single multi-party subscription"
) { context =>
for {
alpha <- context.participant()
beta <- context.participant()
giver <- alpha.allocateParty()
alphaObservers <- alpha.allocateParties(partyCount / 2 - 1)
betaObservers <- beta.allocateParties(partyCount / 2)
observers = alphaObservers ++ betaObservers
alphaParties = giver +: alphaObservers
contractId <- alpha.create(giver, WithObservers(giver, observers))
alphaTransactions <- alpha.flatTransactions(alphaParties: _*)
betaTransactions <- beta.flatTransactions(betaObservers: _*)
} yield {
assertWitnessesOfAMultiPartySubscription(
alphaParties.toSet,
contractId,
activeContractsFrom(alphaTransactions))
assertWitnessesOfAMultiPartySubscription(
betaObservers.toSet,
contractId,
activeContractsFrom(betaTransactions))
}
}
private[this] val seeActiveContractsInMultipleSinglePartySubscriptions = LedgerTest(
"LOPseeActiveContractsInMultipleSinglePartySubscriptions",
"Observers should see active contracts in multiple single-party subscriptions"
) { context =>
for {
alpha <- context.participant()
beta <- context.participant()
giver <- alpha.allocateParty()
alphaObservers <- alpha.allocateParties(partyCount / 2 - 1)
betaObservers <- beta.allocateParties(partyCount / 2)
observers = alphaObservers ++ betaObservers
alphaParties = giver +: alphaObservers
contractId <- alpha.create(giver, WithObservers(giver, observers))
alphaContractsByParty <- activeContractsForEachParty(alpha, alphaParties)
betaContractsByParty <- activeContractsForEachParty(beta, betaObservers)
} yield {
assertWitnessesOfSinglePartySubscriptions(
alphaParties.toSet,
contractId,
alphaContractsByParty)
assertWitnessesOfSinglePartySubscriptions(
betaObservers.toSet,
contractId,
betaContractsByParty)
}
}
private[this] val seeActiveContractsInSingleMultiPartySubscription = LedgerTest(
"LOPseeActiveContractsInSingleMultiPartySubscription",
"Observers should see active contracts in a single multi-party subscription"
) { context =>
for {
alpha <- context.participant()
beta <- context.participant()
giver <- alpha.allocateParty()
alphaObservers <- alpha.allocateParties(partyCount / 2 - 1)
betaObservers <- beta.allocateParties(partyCount / 2)
observers = alphaObservers ++ betaObservers
alphaParties = giver +: alphaObservers
contractId <- alpha.create(giver, WithObservers(giver, observers))
alphaContracts <- alpha.activeContracts(alphaParties: _*)
betaContracts <- beta.activeContracts(betaObservers: _*)
} yield {
assertWitnessesOfAMultiPartySubscription(alphaParties.toSet, contractId, alphaContracts)
assertWitnessesOfAMultiPartySubscription(betaObservers.toSet, contractId, betaContracts)
}
}
override val tests: Vector[LedgerTest] =
Vector(
seeTransactionsInMultipleSinglePartySubscriptions,
seeTransactionsInSingleMultiPartySubscription,
seeActiveContractsInMultipleSinglePartySubscriptions,
seeActiveContractsInSingleMultiPartySubscription,
)
private def transactionsForEachParty(
ledger: ParticipantTestContext,
observers: Vector[Party]): Future[PartyMap[Vector[Transaction]]] = {
Future
.sequence(observers.map(observer => ledger.flatTransactions(observer).map(observer -> _)))
.map(_.toMap)
}
private def activeContractsForEachParty(
ledger: ParticipantTestContext,
observers: Vector[Party]): Future[PartyMap[Vector[CreatedEvent]]] = {
Future
.sequence(observers.map(observer => ledger.activeContracts(observer).map(observer -> _)))
.map(_.toMap)
}
private def activeContractsFrom(
transactionsByParty: PartyMap[Vector[Transaction]]): PartyMap[Vector[CreatedEvent]] = {
transactionsByParty.mapValues(transactions => activeContractsFrom(transactions))
}
private def activeContractsFrom(transactions: Vector[Transaction]): Vector[CreatedEvent] = {
transactions.map(transaction => {
val event = assertSingleton("single transaction event", transaction.events)
event.event.created.get
})
}
private def assertWitnessesOfSinglePartySubscriptions(
observers: Set[Party],
contractId: ContractId[WithObservers],
activeContracts: PartyMap[Seq[CreatedEvent]]
): Unit = {
val expectedContractIds: PartyMap[Seq[ContractId[WithObservers]]] =
observers.map(observer => observer -> Seq(contractId)).toMap
val actualContractIds: PartyMap[Seq[ContractId[WithObservers]]] =
activeContracts.mapValues(_.map(e => ContractId(e.contractId)))
assertEquals("single-party contract IDs", actualContractIds, expectedContractIds)
val expectedWitnesses: Set[Seq[Parties]] =
observers.map(observer => Seq(Set(observer)))
val actualWitnesses: Set[Seq[Parties]] =
activeContracts.values.map(_.map(_.witnessParties.map(Party(_)).toSet)).toSet
assertEquals("single-party witnesses", actualWitnesses, expectedWitnesses)
}
private def assertWitnessesOfAMultiPartySubscription(
observers: Set[Party],
contractId: ContractId[WithObservers],
activeContracts: Seq[CreatedEvent]
): Unit = {
val expectedContractIds: Seq[ContractId[WithObservers]] =
Seq(contractId)
val actualContractIds: Seq[ContractId[WithObservers]] =
activeContracts.map(e => ContractId(e.contractId))
assertEquals("multi-party contract IDs", actualContractIds, expectedContractIds)
val expectedWitnesses: Seq[Parties] =
Seq(observers)
val actualWitnesses: Seq[Parties] =
activeContracts.map(_.witnessParties.map(Party(_)).toSet)
assertEquals("multi-party witnesses", actualWitnesses, expectedWitnesses)
}
}

View File

@ -28,6 +28,7 @@ package object tests {
"DivulgenceIT" -> (new Divulgence(_)),
"IdentityIT" -> (new Identity(_)),
"LedgerConfigurationServiceIT" -> (new LedgerConfigurationService(_)),
"LotsOfPartiesIT" -> (new LotsOfParties(_)),
"PackageManagementServiceIT" -> (new PackageManagement(_)),
"PackageServiceIT" -> (new Packages(_)),
"PartyManagementServiceIT" -> (new PartyManagement(_)),

View File

@ -14,7 +14,7 @@ load(
load("//bazel_tools:pom_file.bzl", "pom_file")
load(
"//ledger/ledger-api-test-tool:conformance.bzl",
"conformance_test",
"server_conformance_test",
)
compileDependencies = [
@ -202,107 +202,91 @@ da_scala_test_suite(
] + testDependencies,
)
conformance_test(
SERVERS = {
"memory": {
"binary": ":sandbox-binary",
"server_args": [
"--port=6865",
"--eager-package-loading",
],
"test_tool_args": [
"--timeout-scale-factor=10",
],
},
"h2database": {
"binary": ":sandbox-binary",
"server_args": [
"--port=6865",
"--eager-package-loading",
# "db_close_delay=-1" is needed so that the in-memory database is not closed
# (and therefore lost) after the flyway migration
"--sql-backend-jdbcurl=jdbc:h2:mem:static_time;db_close_delay=-1",
],
"test_tool_args": [
"--timeout-scale-factor=10",
],
},
"postgres": {
"binary": ":sandbox-ephemeral-postgres",
"extra_data": [
"@postgresql_dev_env//:all",
"@postgresql_dev_env//:createdb",
"@postgresql_dev_env//:initdb",
"@postgresql_dev_env//:pg_ctl",
],
"server_args": [
"--port=6865",
"--eager-package-loading",
],
"test_tool_args": [
"--timeout-scale-factor=10",
],
},
}
server_conformance_test(
name = "conformance-test-static-time",
server = ":sandbox-binary",
server_args = [
"--port 6865",
"--static-time",
"--eager-package-loading",
],
servers = SERVERS,
test_tool_args = [
"--all-tests",
"--timeout-scale-factor=10",
"--exclude=LotsOfPartiesIT",
],
)
conformance_test(
server_conformance_test(
name = "conformance-test-static-time-lots-of-parties",
server_args = [
"--static-time",
],
servers = SERVERS,
test_tool_args = [
"--include=LotsOfPartiesIT",
],
)
server_conformance_test(
name = "conformance-test-wall-clock",
server = ":sandbox-binary",
server_args = [
"--port 6865",
"--wall-clock-time",
"--eager-package-loading",
],
servers = SERVERS,
test_tool_args = [
"--all-tests",
"--exclude TimeIT",
"--timeout-scale-factor=10",
"--exclude=LotsOfPartiesIT",
"--exclude=TimeIT",
],
)
conformance_test(
name = "conformance-test-static-time-postgres",
extra_data = [
"@postgresql_dev_env//:all",
"@postgresql_dev_env//:createdb",
"@postgresql_dev_env//:initdb",
"@postgresql_dev_env//:pg_ctl",
],
server = ":sandbox-ephemeral-postgres",
server_conformance_test(
name = "conformance-test-wall-clock-lots-of-parties",
server_args = [
"--port 6865",
"--static-time",
"--eager-package-loading",
],
test_tool_args = [
"--all-tests",
"--timeout-scale-factor=10",
],
)
conformance_test(
name = "conformance-test-wall-clock-postgres",
extra_data = [
"@postgresql_dev_env//:all",
"@postgresql_dev_env//:createdb",
"@postgresql_dev_env//:initdb",
"@postgresql_dev_env//:pg_ctl",
],
server = ":sandbox-ephemeral-postgres",
server_args = [
"--port 6865",
"--wall-clock-time",
"--eager-package-loading",
],
servers = SERVERS,
test_tool_args = [
"--all-tests",
"--exclude TimeIT",
"--timeout-scale-factor=10",
],
)
conformance_test(
name = "conformance-test-static-time-h2database",
server = ":sandbox-binary",
server_args = [
"--port 6865",
"--static-time",
"--eager-package-loading",
# db_close_delay=-1 is needed so that the in-memory database is not closed (and therefore lost)
# after the flyway migration
"--sql-backend-jdbcurl jdbc:h2:mem:static_time;db_close_delay=-1",
],
test_tool_args = [
"--all-tests",
"--timeout-scale-factor=10",
],
)
conformance_test(
name = "conformance-test-wall-clock-h2database",
server = ":sandbox-binary",
server_args = [
"--port 6865",
"--wall-clock-time",
"--eager-package-loading",
# db_close_delay=-1 is needed so that the in-memory database is not closed (and therefore lost)
# after the flyway migration
"--sql-backend-jdbcurl jdbc:h2:mem:wall_clock_time;db_close_delay=-1",
],
test_tool_args = [
"--all-tests",
"--exclude TimeIT",
"--timeout-scale-factor=10",
"--include=LotsOfPartiesIT",
],
)