From 8bb02c63f0c206efe09a2d3dea9e7957afd668aa Mon Sep 17 00:00:00 2001 From: Rick Parker Date: Wed, 13 Dec 2017 14:34:01 +0000 Subject: [PATCH] ENT-1161 Notary load testing flow (#175) --- .../persistence/DBCheckpointStorage.kt | 4 + .../enterprise/perftestcordapp/Whitelist.kt | 9 + .../LinearStateBatchNotariseContract.kt | 36 +++ .../flows/LinearStateBatchNotariseFlow.kt | 131 ++++++++ ....core.serialization.SerializationWhitelist | 2 + .../com/r3/corda/jmeter/BaseFlowSampler.kt | 9 +- .../kotlin/com/r3/corda/jmeter/Samplers.kt | 115 +++++++ .../resources/LocalBatchNotarise Request.jmx | 291 ++++++++++++++++++ .../r3/corda/jmeter/StartLocalPerfCorDapp.kt | 14 +- 9 files changed, 605 insertions(+), 6 deletions(-) create mode 100644 perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/Whitelist.kt create mode 100644 perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/contracts/LinearStateBatchNotariseContract.kt create mode 100644 perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/flows/LinearStateBatchNotariseFlow.kt create mode 100644 perftestcordapp/src/main/resources/META-INF/services/net.corda.core.serialization.SerializationWhitelist create mode 100644 tools/jmeter/src/main/resources/LocalBatchNotarise Request.jmx diff --git a/node/src/main/kotlin/net/corda/node/services/persistence/DBCheckpointStorage.kt b/node/src/main/kotlin/net/corda/node/services/persistence/DBCheckpointStorage.kt index e0db523b07..fcad94611e 100644 --- a/node/src/main/kotlin/net/corda/node/services/persistence/DBCheckpointStorage.kt +++ b/node/src/main/kotlin/net/corda/node/services/persistence/DBCheckpointStorage.kt @@ -2,11 +2,13 @@ package net.corda.node.services.persistence import net.corda.core.flows.StateMachineRunId import net.corda.core.serialization.SerializedBytes +import net.corda.core.utilities.debug import net.corda.node.services.api.CheckpointStorage import net.corda.node.services.statemachine.Checkpoint import net.corda.nodeapi.internal.persistence.DatabaseTransactionManager import net.corda.nodeapi.internal.persistence.NODE_DATABASE_PREFIX import net.corda.nodeapi.internal.persistence.currentDBSession +import org.slf4j.LoggerFactory import java.util.* import java.util.stream.Stream import javax.persistence.Column @@ -18,6 +20,7 @@ import javax.persistence.Lob * Simple checkpoint key value storage in DB. */ class DBCheckpointStorage : CheckpointStorage { + val log = LoggerFactory.getLogger(this::class.java) @Entity @javax.persistence.Table(name = "${NODE_DATABASE_PREFIX}checkpoints") @@ -35,6 +38,7 @@ class DBCheckpointStorage : CheckpointStorage { currentDBSession().saveOrUpdate(DBCheckpoint().apply { checkpointId = id.uuid.toString() this.checkpoint = checkpoint.bytes + log.debug { "Checkpoint $checkpointId, size=${this.checkpoint.size}" } }) } diff --git a/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/Whitelist.kt b/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/Whitelist.kt new file mode 100644 index 0000000000..829233a7a5 --- /dev/null +++ b/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/Whitelist.kt @@ -0,0 +1,9 @@ +package com.r3.corda.enterprise.perftestcordapp + +import net.corda.core.serialization.SerializationWhitelist +import java.util.* + + +class Whitelist : SerializationWhitelist { + override val whitelist: List> = listOf(LinkedList::class.java) +} \ No newline at end of file diff --git a/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/contracts/LinearStateBatchNotariseContract.kt b/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/contracts/LinearStateBatchNotariseContract.kt new file mode 100644 index 0000000000..7d477b25b8 --- /dev/null +++ b/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/contracts/LinearStateBatchNotariseContract.kt @@ -0,0 +1,36 @@ +package com.r3.corda.enterprise.perftestcordapp.contracts + +import net.corda.core.contracts.* +import net.corda.core.identity.AbstractParty +import net.corda.core.transactions.LedgerTransaction +import java.time.Instant + +/** + * A lightweight `LinearState` based contract and state for use with notary performance testing. + * + * The verify method is mostly empty. All it expects is a single command. No additional vault schemas are defined. + */ +class LinearStateBatchNotariseContract : Contract { + companion object { + const val CP_PROGRAM_ID: ContractClassName = "com.r3.corda.enterprise.perftestcordapp.contracts.LinearStateBatchNotariseContract" + } + + data class State( + override val linearId: UniqueIdentifier, + val creator: AbstractParty, + val creationStamp: Instant + ) : LinearState { + + override val participants = listOf(creator) + } + + interface Commands : CommandData { + class Create : TypeOnlyCommandData(), Commands + class Evolve : TypeOnlyCommandData(), Commands + } + + override fun verify(tx: LedgerTransaction) { + val command = tx.commands.requireSingleCommand() + val timeWindow: TimeWindow? = tx.timeWindow + } +} diff --git a/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/flows/LinearStateBatchNotariseFlow.kt b/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/flows/LinearStateBatchNotariseFlow.kt new file mode 100644 index 0000000000..3c58e2eadf --- /dev/null +++ b/perftestcordapp/src/main/kotlin/com/r3/corda/enterprise/perftestcordapp/flows/LinearStateBatchNotariseFlow.kt @@ -0,0 +1,131 @@ +package com.r3.corda.enterprise.perftestcordapp.flows + +import co.paralleluniverse.fibers.Suspendable +import com.r3.corda.enterprise.perftestcordapp.contracts.LinearStateBatchNotariseContract +import net.corda.core.contracts.TimeWindow +import net.corda.core.contracts.TransactionState +import net.corda.core.contracts.UniqueIdentifier +import net.corda.core.flows.* +import net.corda.core.identity.Party +import net.corda.core.internal.times +import net.corda.core.serialization.CordaSerializable +import net.corda.core.transactions.SignedTransaction +import net.corda.core.transactions.TransactionBuilder +import net.corda.core.utilities.ProgressTracker +import java.time.Duration +import java.time.Instant +import java.util.* +import java.util.concurrent.TimeUnit + +/** + * A flow that generates N linear states, and then evolves them X times, as close to the specified rate as possible. + * + * @property notary The notary to use for notarising the evolution transactions (not the initial, which is unnotarised). + * @property n The number of states per transaction. + * @property x The number of iterations to do (so overall, we generate x+1 transactions). + * @property logIterations If true, will log at info level the iteration the flow is on (helpful if trying to see progress in node logs). + * @property transactionsPerSecond A target number of transactions to generate per second. The target may not be achieved. + */ +@StartableByRPC +class LinearStateBatchNotariseFlow(private val notary: Party, + private val n: Int, + private val x: Int, + private val logIterations: Boolean, + private val transactionsPerSecond: Double +) : FlowLogic() { + companion object { + object GENERATING_INITIAL_TX : ProgressTracker.Step("Generating initial transaction") + object EVOLVING_STATES_TX : ProgressTracker.Step("Generating transaction to evolve states") + object SENDING_RESULTS : ProgressTracker.Step("Sending results") + + fun tracker() = ProgressTracker(GENERATING_INITIAL_TX, EVOLVING_STATES_TX, SENDING_RESULTS) + } + + override val progressTracker: ProgressTracker = tracker() + + @Suspendable + override fun call(): Result { + progressTracker.currentStep = GENERATING_INITIAL_TX + val us = serviceHub.myInfo.legalIdentities.first() + var inputTx = buildInitialTx(us) + progressTracker.currentStep = EVOLVING_STATES_TX + val durationOfEachIteration = Duration.ofHours(1).dividedBy((transactionsPerSecond * TimeUnit.SECONDS.convert(1, TimeUnit.HOURS)).toLong()) + val measurements = LinkedList() + val iterationStartTime = serviceHub.clock.instant() + (0 until x).forEach { iterationNumber -> + val expectedTimeOfNextIteration = iterationStartTime.plus(durationOfEachIteration.times(iterationNumber.toLong())) + val sleepDuration = Duration.between(serviceHub.clock.instant(), expectedTimeOfNextIteration) + if (!sleepDuration.isNegative && !sleepDuration.isZero) { + sleep(sleepDuration) + } + if (logIterations) { + logger.info("ITERATION ${iterationNumber + 1} of $x, with $n states. Slept for $sleepDuration") + } + buildEvolveTx(inputTx, us, iterationNumber, sleepDuration).apply { + inputTx = first + measurements += second + } + } + progressTracker.currentStep = SENDING_RESULTS + return Result(measurements) + } + + @Suspendable + private fun buildEvolveTx(inputTx: SignedTransaction, us: Party, iterationNumber: Int, sleepDuration: Duration): Pair { + val tx = assembleEvolveTx(inputTx, us) + val startTime = serviceHub.clock.instant() + val stx = finaliseTx(tx, "Unable to notarise initial evolution transaction, iteration $iterationNumber.") + val endTime = serviceHub.clock.instant() + return stx to Measurement(startTime, endTime, sleepDuration) + } + + @Suspendable + private fun assembleEvolveTx(inputTx: SignedTransaction, us: Party): SignedTransaction { + val wtx = inputTx.tx + val builder = TransactionBuilder(notary) + (0 until n).forEach { outputIndex -> + val input = wtx.outRef(outputIndex) + builder.addInputState(input) + builder.addOutputState(TransactionState(LinearStateBatchNotariseContract.State(input.state.data.linearId, us, serviceHub.clock.instant()), LinearStateBatchNotariseContract.CP_PROGRAM_ID, notary)) + } + builder.addCommand(LinearStateBatchNotariseContract.Commands.Evolve(), us.owningKey) + builder.setTimeWindow(TimeWindow.fromOnly(serviceHub.clock.instant())) + val tx = serviceHub.signInitialTransaction(builder, us.owningKey) + return tx + } + + @Suspendable + private fun buildInitialTx(us: Party): SignedTransaction { + val tx = assembleInitialTx(us) + return finaliseTx(tx, "Unable to notarise initial generation transaction.") + } + + @Suspendable + private fun assembleInitialTx(us: Party): SignedTransaction { + val builder = TransactionBuilder(notary) + (0 until n).forEach { outputIndex -> + builder.addOutputState(TransactionState(LinearStateBatchNotariseContract.State(UniqueIdentifier(), us, serviceHub.clock.instant()), LinearStateBatchNotariseContract.CP_PROGRAM_ID, notary)) + } + builder.addCommand(LinearStateBatchNotariseContract.Commands.Create(), us.owningKey) + builder.setTimeWindow(TimeWindow.fromOnly(serviceHub.clock.instant())) + val tx = serviceHub.signInitialTransaction(builder, us.owningKey) + return tx + } + + @Suspendable + protected fun finaliseTx(tx: SignedTransaction, message: String): SignedTransaction { + try { + return subFlow(FinalityFlow(tx)) + } catch (e: NotaryException) { + throw FlowException(message, e) + } + } + + @CordaSerializable + data class Result(val measurements: LinkedList) + + @CordaSerializable + data class Measurement(val start: Instant, val end: Instant, val delay: Duration) +} + + diff --git a/perftestcordapp/src/main/resources/META-INF/services/net.corda.core.serialization.SerializationWhitelist b/perftestcordapp/src/main/resources/META-INF/services/net.corda.core.serialization.SerializationWhitelist new file mode 100644 index 0000000000..0f42698536 --- /dev/null +++ b/perftestcordapp/src/main/resources/META-INF/services/net.corda.core.serialization.SerializationWhitelist @@ -0,0 +1,2 @@ +com.r3.corda.enterprise.perftestcordapp.Whitelist + diff --git a/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/BaseFlowSampler.kt b/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/BaseFlowSampler.kt index 34b17e4901..04f8b46e97 100644 --- a/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/BaseFlowSampler.kt +++ b/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/BaseFlowSampler.kt @@ -48,11 +48,15 @@ abstract class BaseFlowSampler() : AbstractJavaSamplerClient() { setupTest(rpcProxy!!, context) } + protected open fun additionalFlowResponseProcessing(context: JavaSamplerContext, sample: SampleResult, response: Any?) { + // Override this if you want to contribute things from the flow result to the sample. + } + override fun runTest(context: JavaSamplerContext): SampleResult { val flowInvoke = createFlowInvoke(rpcProxy!!, context) val result = SampleResult() result.sampleStart() - val handle = rpcProxy!!.startFlowDynamic(flowInvoke!!.flowLogicClass, *(flowInvoke!!.args)) + val handle = rpcProxy!!.startFlowDynamic(flowInvoke.flowLogicClass, *(flowInvoke.args)) result.sampleLabel = handle.id.toString() result.latencyEnd() try { @@ -60,11 +64,14 @@ abstract class BaseFlowSampler() : AbstractJavaSamplerClient() { result.sampleEnd() return result.apply { isSuccessful = true + additionalFlowResponseProcessing(context, this, flowResult) } } catch (e: Exception) { result.sampleEnd() + e.printStackTrace() return result.apply { isSuccessful = false + additionalFlowResponseProcessing(context, this, e) } } } diff --git a/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/Samplers.kt b/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/Samplers.kt index 6ab1a97b5d..0fc61cf256 100644 --- a/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/Samplers.kt +++ b/tools/jmeter/src/main/kotlin/com/r3/corda/jmeter/Samplers.kt @@ -5,12 +5,15 @@ import com.r3.corda.enterprise.perftestcordapp.POUNDS import com.r3.corda.enterprise.perftestcordapp.flows.CashIssueAndPaymentFlow import com.r3.corda.enterprise.perftestcordapp.flows.CashIssueAndPaymentNoSelection import com.r3.corda.enterprise.perftestcordapp.flows.CashIssueFlow +import com.r3.corda.enterprise.perftestcordapp.flows.LinearStateBatchNotariseFlow import net.corda.core.identity.CordaX500Name import net.corda.core.identity.Party import net.corda.core.messaging.CordaRPCOps import net.corda.core.utilities.OpaqueBytes import org.apache.jmeter.config.Argument import org.apache.jmeter.protocol.java.sampler.JavaSamplerContext +import org.apache.jmeter.samplers.SampleResult +import java.util.* /** * A base sampler that looks up identities via RPC ready for starting flows, to be extended and specialised as required. @@ -90,4 +93,116 @@ class CashIssueAndPaySampler : AbstractSampler() { override val additionalArgs: Set get() = setOf(notary, otherParty, coinSelection) +} + +/** + * A sampler that attempts to generate load on the Notary. + * + * It builds a transaction of multiple `LinearState`s and then for each iteration transitions each state as a batch in + * a single transaction and sends it to be notarised. That way the size of the transaction, both in bytes and number of + * states, can be varied (not independently currently). + * + * The requesting flow does this for a specified number of iterations, and returns a result that is then fed back to JMeter. + * So don't be surprised if it looks like JMeter is getting no results for a while. It only receives them when the flow + * finishes. + * If JMeter asks for more samples/iterations than that, then another flow is kicked off automatically if the repeat property + * is set to true, otherwise it will return an error/failure for the next sample. + * + * The flow will throttle (if necessary) to maintain a specified number of iterations/transactions per second. This is + * aggregated over all iterations, so GC pauses etc shouldn't reduce the overall number of transactions in a time period, + * unless the node / notary is unable to keep up. If falling behind, the flow will not pause between transactions. + */ +class LinearStateBatchNotariseSampler : AbstractSampler() { + companion object JMeterProperties { + val numberOfStates = Argument("numStates", "1", "", "Number of linear states to include in each transaction.") + val numberOfIterations = Argument("numIterations", "1", "", "Number of iterations / evolutions to do. Each iteration generates one transaction.") + val logIterations = Argument("enableLog", "false", "", "Print in the logs what iteration the test is on etc.") + val numberOfTps = Argument("transactionsPerSecond", "1.0", "", "Transaction per second target.") + val repeat = Argument("repeatInvoke", "false", "", "If true, invoke the flow again if JMeter expects more iterations.") + } + + var n: Int = 0 + var x: Int = 0 + var log: Boolean = false + var tps: Double = 1.0 + var reRequest: Boolean = false + + var measurements: LinkedList> = LinkedList() + var measurementsSize: Int = 0 + var nextIteration: Int = 0 + var sample: SampleResult? = null + + override val additionalArgs: Set = setOf(notary, numberOfStates, numberOfIterations, logIterations, numberOfTps, repeat) + + // At test setup, we fire off one big request via RPC. + override fun setupTest(context: JavaSamplerContext) { + measurements.clear() + super.setupTest(context) + println("Running test $context") + super.runTest(context) + } + + override fun setupTest(rpcProxy: CordaRPCOps, testContext: JavaSamplerContext) { + getNotaryIdentity(rpcProxy, testContext) + n = testContext.getParameter(numberOfStates.name, numberOfStates.value).toInt() + x = testContext.getParameter(numberOfIterations.name, numberOfIterations.value).toInt() + log = testContext.getParameter(logIterations.name, logIterations.value).toBoolean() + tps = testContext.getParameter(numberOfTps.name, numberOfTps.value).toDouble() + reRequest = testContext.getParameter(repeat.name, repeat.value).toBoolean() + } + + override fun teardownTest(rpcProxy: CordaRPCOps, testContext: JavaSamplerContext) { + } + + override fun createFlowInvoke(rpcProxy: CordaRPCOps, testContext: JavaSamplerContext): FlowInvoke { + return FlowInvoke(LinearStateBatchNotariseFlow::class.java, arrayOf(notaryIdentity, n, x, log, tps)) + } + + override fun additionalFlowResponseProcessing(context: JavaSamplerContext, sample: SampleResult, response: Any?) { + if (response is LinearStateBatchNotariseFlow.Result && response.measurements.isNotEmpty()) { + measurements.add(response.measurements) + measurementsSize += response.measurements.size + } + this.sample = sample + } + + private fun nextMeasurement(context: JavaSamplerContext): LinearStateBatchNotariseFlow.Measurement { + val firstList = measurements.first() + val measurement = firstList.remove() + measurementsSize-- + if (firstList.isEmpty()) { + measurements.remove() + nextIteration = 0 + // if a flag is set, run the flow again + if (reRequest) { + println("Re-running test $context") + super.runTest(context) + } + } + return measurement + } + + // Each iteration of the test returns the next measurement from the large batch request. + override fun runTest(context: JavaSamplerContext): SampleResult { + val topLevelSample = sample ?: SampleResult().apply { isSuccessful = false } + val currentIteration = nextIteration++ + // Build samples based on the response. + val result = if (topLevelSample.isSuccessful && measurementsSize > 0) { + val measurement = nextMeasurement(context) + val result = SampleResult(measurement.end.toEpochMilli(), measurement.end.toEpochMilli() - measurement.start.toEpochMilli()) + val delay = measurement.delay.toMillis() + if (delay < 0) { + result.latency = -delay + } + result.isSuccessful = true + result.sampleLabel = "${topLevelSample.sampleLabel}-$currentIteration" + result + } else { + val result = SampleResult(topLevelSample.timeStamp, 0) + result.isSuccessful = false + result.sampleLabel = if (!topLevelSample.isSuccessful) "${topLevelSample.sampleLabel}-$currentIteration" else "${topLevelSample.sampleLabel}-END" + result + } + return result + } } \ No newline at end of file diff --git a/tools/jmeter/src/main/resources/LocalBatchNotarise Request.jmx b/tools/jmeter/src/main/resources/LocalBatchNotarise Request.jmx new file mode 100644 index 0000000000..ba7995e318 --- /dev/null +++ b/tools/jmeter/src/main/resources/LocalBatchNotarise Request.jmx @@ -0,0 +1,291 @@ + + + + + + false + false + + + + + + + + false + + saveConfig + + + true + true + true + + true + true + true + true + false + true + true + false + false + false + true + false + false + false + true + 0 + true + true + true + true + true + + + + true + + + + stopthread + + false + -1 + + 3 + + 1509455820000 + 1509455820000 + false + + + + + + + + + host + localhost + = + + + port + 10004 + = + + + username + perf + = + + + password + perf + = + + + notaryName + O=Notary Service,L=Zurich,C=CH,CN=corda.notary.simple + = + + + numStates + 100 + = + + + numIterations + 1 + = + + + enableLog + true + = + + + transactionsPerMinute + 10 + = + + + repeatInvoke + true + = + + + + com.r3.corda.jmeter.LinearStateBatchNotariseSampler + + + + + + + host + localhost + = + + + port + 10004 + = + + + username + perf + = + + + password + perf + = + + + notaryName + O=Notary Service,L=Zurich,C=CH,CN=corda.notary.simple + = + + + otherPartyName + O=Bank B,L=New York,C=US + = + + + useCoinSelection + true + = + + + + com.r3.corda.jmeter.CashIssueAndPaySampler + + + + + stopthread + + false + -1 + + 3 + + 1509455820000 + 1509455820000 + false + + + + + + + + + host + localhost + = + + + port + 10007 + = + + + username + perf + = + + + password + perf + = + + + notaryName + O=Notary Service,L=Zurich,C=CH,CN=corda.notary.simple + = + + + numStates + 100 + = + + + numIterations + 1 + = + + + enableLog + true + = + + + transactionsPerMinute + 10 + = + + + repeatInvoke + true + = + + + + com.r3.corda.jmeter.LinearStateBatchNotariseSampler + + + + + + + host + localhost + = + + + port + 10004 + = + + + username + perf + = + + + password + perf + = + + + notaryName + O=Notary Service,L=Zurich,C=CH,CN=corda.notary.simple + = + + + otherPartyName + O=Bank B,L=New York,C=US + = + + + useCoinSelection + true + = + + + + com.r3.corda.jmeter.CashIssueAndPaySampler + + + + + + true + + + + diff --git a/tools/jmeter/src/test/kotlin/com/r3/corda/jmeter/StartLocalPerfCorDapp.kt b/tools/jmeter/src/test/kotlin/com/r3/corda/jmeter/StartLocalPerfCorDapp.kt index 397cb0ba4e..6feea7f126 100644 --- a/tools/jmeter/src/test/kotlin/com/r3/corda/jmeter/StartLocalPerfCorDapp.kt +++ b/tools/jmeter/src/test/kotlin/com/r3/corda/jmeter/StartLocalPerfCorDapp.kt @@ -5,29 +5,33 @@ import net.corda.node.services.Permissions import net.corda.nodeapi.internal.config.User import net.corda.testing.DUMMY_NOTARY import net.corda.testing.node.NotarySpec +import org.slf4j.LoggerFactory import java.io.BufferedReader import java.io.InputStreamReader class StartLocalPerfCorDapp { companion object { + val log = LoggerFactory.getLogger(this::class.java) + @JvmStatic fun main(args: Array) { // Typically the RPC port of Bank A is 10004. val demoUser = User("perf", "perf", setOf(Permissions.all())) net.corda.testing.driver.driver(startNodesInProcess = false, waitForAllNodesToFinish = true, + //isDebug = true, notarySpecs = listOf(NotarySpec(DUMMY_NOTARY.name, validating = false)), extraCordappPackagesToScan = listOf("com.r3.corda.enterprise.perftestcordapp")) { val (nodeA, nodeB) = listOf( - startNode(providedName = net.corda.testing.DUMMY_BANK_A.name, rpcUsers = listOf(demoUser)), - startNode(providedName = net.corda.testing.DUMMY_BANK_B.name, rpcUsers = listOf(demoUser)) + startNode(providedName = net.corda.testing.DUMMY_BANK_A.name, rpcUsers = listOf(demoUser), maximumHeapSize = "1G"), + startNode(providedName = net.corda.testing.DUMMY_BANK_B.name, rpcUsers = listOf(demoUser), maximumHeapSize = "1G") ).map { it.getOrThrow() } - println("Nodes started!") + log.info("Nodes started!") val input = BufferedReader(InputStreamReader(System.`in`)) do { - Ssh.log.info("Type 'quit' to exit cleanly.") + log.info("Type 'quit' to exit cleanly.") } while (input.readLine() != "quit") - println("Quitting... (this sometimes takes a while)") + log.info("Quitting... (this sometimes takes a while)") nodeA.stop() nodeB.stop() defaultNotaryHandle.nodeHandles.getOrThrow().map { it.stop() }