Removed the ability to manually start notary nodes from the driver and MockNetwork. Instead by default a single notary is automatically started. This can be customised at creation time of the driver and MockNetwork. This more accurately models the concept of network parameters in a CZ.

Also added helper methods to retrieve this default notary.
This commit is contained in:
Shams Asari
2017-11-05 12:34:42 +00:00
parent 9be37c2b88
commit 3bb018a5ce
64 changed files with 637 additions and 643 deletions

View File

@ -15,9 +15,9 @@ import net.corda.finance.flows.CashPaymentFlow
import net.corda.node.services.Permissions.Companion.startFlow
import net.corda.nodeapi.User
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.chooseIdentity
import net.corda.testing.driver.NodeHandle
import net.corda.testing.driver.driver
import net.corda.testing.node.NotarySpec
import net.corda.testing.performance.div
import net.corda.testing.performance.startPublishingFixedRateInjector
import net.corda.testing.performance.startReporter
@ -102,26 +102,25 @@ class NodePerformanceTests {
@Test
fun `self pay rate`() {
driver(startNodesInProcess = true, extraCordappPackagesToScan = listOf("net.corda.finance")) {
val a = startNotaryNode(
DUMMY_NOTARY.name,
rpcUsers = listOf(User("A", "A", setOf(startFlow<CashIssueFlow>(), startFlow<CashPaymentFlow>())))
).getOrThrow()
a as NodeHandle.InProcess
val metricRegistry = startReporter(shutdownManager, a.node.services.monitoringService.metrics)
a.rpcClientToNode().use("A", "A") { connection ->
val notary = connection.proxy.notaryIdentities().first()
val user = User("A", "A", setOf(startFlow<CashIssueFlow>(), startFlow<CashPaymentFlow>()))
driver(
notarySpecs = listOf(NotarySpec(DUMMY_NOTARY.name, rpcUsers = listOf(user))),
startNodesInProcess = true,
extraCordappPackagesToScan = listOf("net.corda.finance")
) {
val notary = defaultNotaryNode.getOrThrow() as NodeHandle.InProcess
val metricRegistry = startReporter(shutdownManager, notary.node.services.monitoringService.metrics)
notary.rpcClientToNode().use("A", "A") { connection ->
println("ISSUING")
val doneFutures = (1..100).toList().parallelStream().map {
connection.proxy.startFlow(::CashIssueFlow, 1.DOLLARS, OpaqueBytes.of(0), notary).returnValue
connection.proxy.startFlow(::CashIssueFlow, 1.DOLLARS, OpaqueBytes.of(0), defaultNotaryIdentity).returnValue
}.toList()
doneFutures.transpose().get()
println("STARTING PAYMENT")
startPublishingFixedRateInjector(metricRegistry, 8, 5.minutes, 100L / TimeUnit.SECONDS) {
connection.proxy.startFlow(::CashPaymentFlow, 1.DOLLARS, a.nodeInfo.chooseIdentity()).returnValue.get()
connection.proxy.startFlow(::CashPaymentFlow, 1.DOLLARS, defaultNotaryIdentity).returnValue.get()
}
}
}
}
}

View File

@ -13,20 +13,16 @@ import net.corda.core.internal.div
import net.corda.core.internal.toLedgerTransaction
import net.corda.core.serialization.SerializationFactory
import net.corda.core.transactions.TransactionBuilder
import net.corda.core.utilities.OpaqueBytes
import net.corda.core.utilities.getOrThrow
import net.corda.core.utilities.loggerFor
import net.corda.core.utilities.seconds
import net.corda.node.internal.cordapp.CordappLoader
import net.corda.node.internal.cordapp.CordappProviderImpl
import net.corda.nodeapi.User
import net.corda.testing.DUMMY_BANK_A
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.SerializationEnvironmentRule
import net.corda.testing.driver.DriverDSLExposedInterface
import net.corda.testing.driver.NodeHandle
import net.corda.testing.driver.driver
import net.corda.testing.eventually
import net.corda.testing.node.MockServices
import org.junit.Assert.assertEquals
import org.junit.Before
@ -56,19 +52,15 @@ class AttachmentLoadingTests {
val bankAName = CordaX500Name("BankA", "Zurich", "CH")
val bankBName = CordaX500Name("BankB", "Zurich", "CH")
val notaryName = CordaX500Name("Notary", "Zurich", "CH")
val flowInitiatorClass: Class<out FlowLogic<*>> =
Class.forName("net.corda.finance.contracts.isolated.IsolatedDummyFlow\$Initiator", true, URLClassLoader(arrayOf(isolatedJAR)))
.asSubclass(FlowLogic::class.java)
private fun DriverDSLExposedInterface.createNotaryAndTwoNodes(): List<NodeHandle> {
val adminUser = User("admin", "admin", permissions = setOf("ALL"))
val nodes = listOf(
startNotaryNode(providedName = notaryName, rpcUsers = listOf(adminUser), validating = false),
startNode(providedName = bankAName, rpcUsers = listOf(adminUser)),
startNode(providedName = bankBName, rpcUsers = listOf(adminUser))
).transpose().getOrThrow() // Wait for all nodes to start up.
return nodes
private fun DriverDSLExposedInterface.createTwoNodes(): List<NodeHandle> {
return listOf(
startNode(providedName = bankAName),
startNode(providedName = bankBName)
).transpose().getOrThrow()
}
private fun DriverDSLExposedInterface.installIsolatedCordappTo(nodeName: CordaX500Name) {
@ -81,15 +73,6 @@ class AttachmentLoadingTests {
}
}
}
// Due to cluster instability after nodes been started it may take some time to all the nodes to become available
// *and* discover each other to reliably communicate. Hence, eventual nature of the test.
// TODO: Remove this method and usages of it once NetworkMap service been re-worked
private fun eventuallyPassingTest(block: () -> Unit) {
eventually<Throwable, Unit>(30.seconds) {
block()
}
}
}
private lateinit var services: Services
@ -105,9 +88,8 @@ class AttachmentLoadingTests {
val contractClass = appClassLoader.loadClass(ISOLATED_CONTRACT_ID).asSubclass(Contract::class.java)
val generateInitialMethod = contractClass.getDeclaredMethod("generateInitial", PartyAndReference::class.java, Integer.TYPE, Party::class.java)
val contract = contractClass.newInstance()
val txBuilder = generateInitialMethod.invoke(contract, PartyAndReference(DUMMY_BANK_A, OpaqueBytes(kotlin.ByteArray(1))), 1, DUMMY_NOTARY) as TransactionBuilder
val context = SerializationFactory.defaultFactory.defaultContext
.withClassLoader(appClassLoader)
val txBuilder = generateInitialMethod.invoke(contract, DUMMY_BANK_A.ref(1), 1, DUMMY_NOTARY) as TransactionBuilder
val context = SerializationFactory.defaultFactory.defaultContext.withClassLoader(appClassLoader)
val ledgerTx = txBuilder.toLedgerTransaction(services, context)
contract.verify(ledgerTx)
@ -121,11 +103,9 @@ class AttachmentLoadingTests {
fun `test that attachments retrieved over the network are not used for code`() {
driver(initialiseSerialization = false) {
installIsolatedCordappTo(bankAName)
val (_, bankA, bankB) = createNotaryAndTwoNodes()
eventuallyPassingTest {
assertFailsWith<UnexpectedFlowEndException>("Party C=CH,L=Zurich,O=BankB rejected session request: Don't know net.corda.finance.contracts.isolated.IsolatedDummyFlow\$Initiator") {
bankA.rpc.startFlowDynamic(flowInitiatorClass, bankB.nodeInfo.legalIdentities.first()).returnValue.getOrThrow()
}
val (bankA, bankB) = createTwoNodes()
assertFailsWith<UnexpectedFlowEndException>("Party C=CH,L=Zurich,O=BankB rejected session request: Don't know net.corda.finance.contracts.isolated.IsolatedDummyFlow\$Initiator") {
bankA.rpc.startFlowDynamic(flowInitiatorClass, bankB.nodeInfo.legalIdentities.first()).returnValue.getOrThrow()
}
}
}
@ -135,10 +115,8 @@ class AttachmentLoadingTests {
driver(initialiseSerialization = false) {
installIsolatedCordappTo(bankAName)
installIsolatedCordappTo(bankBName)
val (_, bankA, bankB) = createNotaryAndTwoNodes()
eventuallyPassingTest {
bankA.rpc.startFlowDynamic(flowInitiatorClass, bankB.nodeInfo.legalIdentities.first()).returnValue.getOrThrow()
}
val (bankA, bankB) = createTwoNodes()
bankA.rpc.startFlowDynamic(flowInitiatorClass, bankB.nodeInfo.legalIdentities.first()).returnValue.getOrThrow()
}
}
}

View File

@ -11,60 +11,64 @@ import net.corda.core.utilities.getOrThrow
import net.corda.finance.POUNDS
import net.corda.finance.flows.CashIssueFlow
import net.corda.finance.flows.CashPaymentFlow
import net.corda.node.services.Permissions.Companion.startFlow
import net.corda.node.services.Permissions.Companion.invokeRpc
import net.corda.node.services.transactions.RaftValidatingNotaryService
import net.corda.node.services.Permissions.Companion.startFlow
import net.corda.nodeapi.User
import net.corda.testing.*
import net.corda.testing.driver.NodeHandle
import net.corda.testing.driver.driver
import net.corda.testing.node.ClusterSpec
import net.corda.testing.node.NotarySpec
import org.assertj.core.api.Assertions.assertThat
import org.junit.Test
import rx.Observable
import java.util.*
import kotlin.test.assertEquals
class DistributedServiceTests {
lateinit var alice: NodeHandle
lateinit var notaries: List<NodeHandle.OutOfProcess>
lateinit var aliceProxy: CordaRPCOps
lateinit var raftNotaryIdentity: Party
lateinit var notaryStateMachines: Observable<Pair<Party, StateMachineUpdate>>
private fun setup(runTest: () -> Unit) = driver(extraCordappPackagesToScan = listOf("net.corda.finance.contracts")) {
// Start Alice and 3 notaries in a RAFT cluster
val clusterSize = 3
private lateinit var alice: NodeHandle
private lateinit var notaryNodes: List<NodeHandle.OutOfProcess>
private lateinit var aliceProxy: CordaRPCOps
private lateinit var raftNotaryIdentity: Party
private lateinit var notaryStateMachines: Observable<Pair<Party, StateMachineUpdate>>
private fun setup(testBlock: () -> Unit) {
val testUser = User("test", "test", permissions = setOf(
startFlow<CashIssueFlow>(),
startFlow<CashPaymentFlow>(),
invokeRpc(CordaRPCOps::nodeInfo),
invokeRpc(CordaRPCOps::stateMachinesFeed))
)
val notariesFuture = startNotaryCluster(
DUMMY_NOTARY.name.copy(commonName = RaftValidatingNotaryService.id),
rpcUsers = listOf(testUser),
clusterSize = clusterSize
)
val aliceFuture = startNode(providedName = ALICE.name, rpcUsers = listOf(testUser))
alice = aliceFuture.get()
val (notaryIdentity, notaryNodes) = notariesFuture.get()
raftNotaryIdentity = notaryIdentity
notaries = notaryNodes.map { it as NodeHandle.OutOfProcess }
driver(
extraCordappPackagesToScan = listOf("net.corda.finance.contracts"),
notarySpecs = listOf(NotarySpec(DUMMY_NOTARY.name, rpcUsers = listOf(testUser), cluster = ClusterSpec.Raft(clusterSize = 3))))
{
alice = startNode(providedName = ALICE.name, rpcUsers = listOf(testUser)).getOrThrow()
raftNotaryIdentity = defaultNotaryIdentity
notaryNodes = defaultNotaryHandle.nodeHandles.getOrThrow().map { it as NodeHandle.OutOfProcess }
assertEquals(notaries.size, clusterSize)
// Check that each notary has different identity as a node.
assertEquals(notaries.size, notaries.map { it.nodeInfo.chooseIdentity() }.toSet().size)
// Connect to Alice and the notaries
fun connectRpc(node: NodeHandle): CordaRPCOps {
val client = node.rpcClientToNode()
return client.start("test", "test").proxy
assertThat(notaryNodes).hasSize(3)
for (notaryNode in notaryNodes) {
assertThat(notaryNode.nodeInfo.legalIdentities).contains(raftNotaryIdentity)
}
// Check that each notary has different identity as a node.
assertThat(notaryNodes.flatMap { it.nodeInfo.legalIdentities - raftNotaryIdentity }.toSet()).hasSameSizeAs(notaryNodes)
// Connect to Alice and the notaries
fun connectRpc(node: NodeHandle): CordaRPCOps {
val client = node.rpcClientToNode()
return client.start("test", "test").proxy
}
aliceProxy = connectRpc(alice)
val rpcClientsToNotaries = notaryNodes.map(::connectRpc)
notaryStateMachines = Observable.from(rpcClientsToNotaries.map { proxy ->
proxy.stateMachinesFeed().updates.map { Pair(proxy.nodeInfo().chooseIdentity(), it) }
}).flatMap { it.onErrorResumeNext(Observable.empty()) }.bufferUntilSubscribed()
testBlock()
}
aliceProxy = connectRpc(alice)
val rpcClientsToNotaries = notaries.map(::connectRpc)
notaryStateMachines = Observable.from(rpcClientsToNotaries.map { proxy ->
proxy.stateMachinesFeed().updates.map { Pair(proxy.nodeInfo().chooseIdentity(), it) }
}).flatMap { it.onErrorResumeNext(Observable.empty()) }.bufferUntilSubscribed()
runTest()
}
// TODO Use a dummy distributed service rather than a Raft Notary Service as this test is only about Artemis' ability
@ -106,8 +110,8 @@ class DistributedServiceTests {
paySelf(5.POUNDS)
}
// Now kill a notary
with(notaries[0].process) {
// Now kill a notary node
with(notaryNodes[0].process) {
destroy()
waitFor()
}

View File

@ -18,6 +18,8 @@ import net.corda.testing.contracts.DummyContract
import net.corda.testing.driver.NodeHandle
import net.corda.testing.driver.driver
import net.corda.testing.dummyCommand
import net.corda.testing.node.ClusterSpec
import net.corda.testing.node.NotarySpec
import org.junit.Test
import java.util.*
import kotlin.test.assertEquals
@ -28,13 +30,16 @@ class RaftNotaryServiceTests {
@Test
fun `detect double spend`() {
driver(startNodesInProcess = true, extraCordappPackagesToScan = listOf("net.corda.testing.contracts")) {
val (notaryParty) = startNotaryCluster(notaryName, 3).getOrThrow()
driver(
startNodesInProcess = true,
extraCordappPackagesToScan = listOf("net.corda.testing.contracts"),
notarySpecs = listOf(NotarySpec(notaryName, cluster = ClusterSpec.Raft(clusterSize = 3))))
{
val bankA = startNode(providedName = DUMMY_BANK_A.name).map { (it as NodeHandle.InProcess).node }.getOrThrow()
val inputState = issueState(bankA, notaryParty)
val inputState = issueState(bankA, defaultNotaryIdentity)
val firstTxBuilder = TransactionBuilder(notaryParty)
val firstTxBuilder = TransactionBuilder(defaultNotaryIdentity)
.addInputState(inputState)
.addCommand(dummyCommand(bankA.services.myInfo.chooseIdentity().owningKey))
val firstSpendTx = bankA.services.signInitialTransaction(firstTxBuilder)
@ -42,7 +47,7 @@ class RaftNotaryServiceTests {
val firstSpend = bankA.services.startFlow(NotaryFlow.Client(firstSpendTx))
firstSpend.resultFuture.getOrThrow()
val secondSpendBuilder = TransactionBuilder(notaryParty).withItems(inputState).run {
val secondSpendBuilder = TransactionBuilder(defaultNotaryIdentity).withItems(inputState).run {
val dummyState = DummyContract.SingleOwnerState(0, bankA.info.chooseIdentity())
addOutputState(dummyState, DummyContract.PROGRAM_ID)
addCommand(dummyCommand(bankA.services.myInfo.chooseIdentity().owningKey))

View File

@ -8,11 +8,11 @@ import net.corda.core.messaging.startFlow
import net.corda.core.transactions.TransactionBuilder
import net.corda.testing.BOB
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.aliceAndBob
import net.corda.testing.contracts.DummyContract
import net.corda.testing.contracts.DummyState
import net.corda.testing.driver.driver
import net.corda.testing.dummyCommand
import net.corda.testing.notaryAliceAndBob
import org.junit.Test
import kotlin.test.assertEquals
@ -65,7 +65,7 @@ class LargeTransactionsTest {
val bigFile3 = InputStreamAndHash.createInMemoryTestZip(1024 * 1024 * 3, 2)
val bigFile4 = InputStreamAndHash.createInMemoryTestZip(1024 * 1024 * 3, 3)
driver(startNodesInProcess = true, extraCordappPackagesToScan = listOf("net.corda.testing.contracts")) {
val (_, alice) = notaryAliceAndBob()
val (alice, _) = aliceAndBob()
alice.useRPC {
val hash1 = it.uploadAttachment(bigFile1.inputStream)
val hash2 = it.uploadAttachment(bigFile2.inputStream)

View File

@ -21,6 +21,8 @@ import net.corda.testing.chooseIdentity
import net.corda.testing.driver.DriverDSLExposedInterface
import net.corda.testing.driver.NodeHandle
import net.corda.testing.driver.driver
import net.corda.testing.node.ClusterSpec
import net.corda.testing.node.NotarySpec
import org.assertj.core.api.Assertions.assertThat
import org.junit.Test
import java.util.*
@ -35,16 +37,14 @@ class P2PMessagingTest {
@Test
fun `communicating with a distributed service which we're part of`() {
driver(startNodesInProcess = true) {
val distributedService = startDistributedService()
startDriverWithDistributedService { distributedService ->
assertAllNodesAreUsed(distributedService, DISTRIBUTED_SERVICE_NAME, distributedService[0])
}
}
@Test
fun `distributed service requests are retried if one of the nodes in the cluster goes down without sending a response`() {
driver(startNodesInProcess = true) {
val distributedServiceNodes = startDistributedService()
startDriverWithDistributedService { distributedServiceNodes ->
val alice = startAlice()
val serviceAddress = alice.services.networkMapCache.run {
val notaryParty = notaryIdentities.randomOrNull()!!
@ -77,8 +77,7 @@ class P2PMessagingTest {
@Test
fun `distributed service request retries are persisted across client node restarts`() {
driver(startNodesInProcess = true) {
val distributedServiceNodes = startDistributedService()
startDriverWithDistributedService { distributedServiceNodes ->
val alice = startAlice()
val serviceAddress = alice.services.networkMapCache.run {
val notaryParty = notaryIdentities.randomOrNull()!!
@ -117,11 +116,10 @@ class P2PMessagingTest {
}
}
private fun DriverDSLExposedInterface.startDistributedService(): List<StartedNode<Node>> {
return startNotaryCluster(DISTRIBUTED_SERVICE_NAME, 2)
.getOrThrow()
.second
.map { (it as NodeHandle.InProcess).node }
private fun startDriverWithDistributedService(dsl: DriverDSLExposedInterface.(List<StartedNode<Node>>) -> Unit) {
driver(startNodesInProcess = true, notarySpecs = listOf(NotarySpec(DISTRIBUTED_SERVICE_NAME, cluster = ClusterSpec.Raft(clusterSize = 2)))) {
dsl(defaultNotaryHandle.nodeHandles.getOrThrow().map { (it as NodeHandle.InProcess).node })
}
}
private fun DriverDSLExposedInterface.startAlice(): StartedNode<Node> {

View File

@ -17,10 +17,9 @@ import net.corda.core.transactions.SignedTransaction
import net.corda.core.transactions.TransactionBuilder
import net.corda.core.utilities.ProgressTracker
import net.corda.core.utilities.getOrThrow
import net.corda.node.services.Permissions.Companion.startFlow
import net.corda.node.services.Permissions.Companion.invokeRpc
import net.corda.node.services.Permissions.Companion.startFlow
import net.corda.nodeapi.User
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.chooseIdentity
import net.corda.testing.driver.driver
import org.junit.Assume.assumeFalse
@ -43,7 +42,6 @@ class NodeStatePersistenceTests {
val message = Message("Hello world!")
driver(isDebug = true, startNodesInProcess = isQuasarAgentSpecified()) {
val nodeName = {
startNotaryNode(DUMMY_NOTARY.name, validating = false).getOrThrow()
val nodeHandle = startNode(rpcUsers = listOf(user)).getOrThrow()
val nodeName = nodeHandle.nodeInfo.chooseIdentity().name
nodeHandle.rpcClientToNode().start(user.username, user.password).use {

View File

@ -31,6 +31,7 @@ import java.security.PublicKey
import java.util.*
import javax.annotation.concurrent.ThreadSafe
import kotlin.collections.HashMap
import kotlin.collections.HashSet
class NetworkMapCacheImpl(
networkMapCacheBase: NetworkMapCacheBaseInternal,
@ -85,7 +86,7 @@ open class PersistentNetworkMapCache(
override val loadDBSuccess get() = _loadDBSuccess
override val notaryIdentities: List<Party> = notaries.map { it.identity }
private val validatingNotaries = notaries.mapNotNull { if (it.validating) it.identity else null }
private val validatingNotaries = notaries.mapNotNullTo(HashSet()) { if (it.validating) it.identity else null }
private val nodeInfoSerializer = NodeInfoWatcher(configuration.baseDirectory,
configuration.additionalNodeInfoPollingFrequencyMsec)

View File

@ -50,14 +50,12 @@ import kotlin.test.assertNull
import kotlin.test.assertTrue
class CordaRPCOpsImplTest {
private companion object {
val testJar = "net/corda/node/testing/test.jar"
}
private lateinit var mockNet: MockNetwork
private lateinit var aliceNode: StartedNode<MockNode>
private lateinit var notaryNode: StartedNode<MockNode>
private lateinit var notary: Party
private lateinit var rpc: CordaRPCOps
private lateinit var stateMachineUpdates: Observable<StateMachineUpdate>
@ -69,7 +67,6 @@ class CordaRPCOpsImplTest {
@Before
fun setup() {
mockNet = MockNetwork(cordappPackages = listOf("net.corda.finance.contracts.asset"))
notaryNode = mockNet.createNotaryNode(validating = false)
aliceNode = mockNet.createNode()
rpc = SecureCordaRPCOps(aliceNode.services, aliceNode.smm, aliceNode.database, aliceNode.services)
CURRENT_RPC_CONTEXT.set(RpcContext(user))

View File

@ -74,14 +74,12 @@ class TestCordaService2(val appServiceHub: AppServiceHub): SingletonSerializeAsT
class LegacyCordaService(@Suppress("UNUSED_PARAMETER") simpleServiceHub: ServiceHub) : SingletonSerializeAsToken()
class CordaServiceTest {
lateinit var mockNet: MockNetwork
lateinit var notaryNode: StartedNode<MockNetwork.MockNode>
lateinit var nodeA: StartedNode<MockNetwork.MockNode>
private lateinit var mockNet: MockNetwork
private lateinit var nodeA: StartedNode<MockNetwork.MockNode>
@Before
fun start() {
mockNet = MockNetwork(threadPerNode = true, cordappPackages = listOf("net.corda.node.internal","net.corda.finance"))
notaryNode = mockNet.createNotaryNode()
nodeA = mockNet.createNode()
mockNet.startNodes()
}

View File

@ -63,7 +63,7 @@ import kotlin.test.assertTrue
* We assume that Alice and Bob already found each other via some market, and have agreed the details already.
*/
@RunWith(Parameterized::class)
class TwoPartyTradeFlowTests(val anonymous: Boolean) {
class TwoPartyTradeFlowTests(private val anonymous: Boolean) {
companion object {
private val cordappPackages = listOf("net.corda.finance.contracts")
@JvmStatic
@ -93,7 +93,7 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
// allow interruption half way through.
mockNet = MockNetwork(threadPerNode = true, cordappPackages = cordappPackages)
ledger(MockServices(cordappPackages), initialiseSerialization = false) {
val notaryNode = mockNet.createNotaryNode()
val notaryNode = mockNet.defaultNotaryNode
val aliceNode = mockNet.createPartyNode(ALICE_NAME)
val bobNode = mockNet.createPartyNode(BOB_NAME)
val bankNode = mockNet.createPartyNode(BOC_NAME)
@ -143,7 +143,7 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
fun `trade cash for commercial paper fails using soft locking`() {
mockNet = MockNetwork(threadPerNode = true, cordappPackages = cordappPackages)
ledger(MockServices(cordappPackages), initialiseSerialization = false) {
val notaryNode = mockNet.createNotaryNode()
val notaryNode = mockNet.defaultNotaryNode
val aliceNode = mockNet.createPartyNode(ALICE_NAME)
val bobNode = mockNet.createPartyNode(BOB_NAME)
val bankNode = mockNet.createPartyNode(BOC_NAME)
@ -199,7 +199,7 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
fun `shutdown and restore`() {
mockNet = MockNetwork(cordappPackages = cordappPackages)
ledger(MockServices(cordappPackages), initialiseSerialization = false) {
val notaryNode = mockNet.createNotaryNode()
val notaryNode = mockNet.defaultNotaryNode
val aliceNode = mockNet.createPartyNode(ALICE_NAME)
var bobNode = mockNet.createPartyNode(BOB_NAME)
val bankNode = mockNet.createPartyNode(BOC_NAME)
@ -292,7 +292,8 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
// Creates a mock node with an overridden storage service that uses a RecordingMap, that lets us test the order
// of gets and puts.
private fun makeNodeWithTracking(name: CordaX500Name): StartedNode<MockNetwork.MockNode> {
private fun makeNodeWithTracking(
name: CordaX500Name): StartedNode<MockNetwork.MockNode> {
// Create a node in the mock network ...
return mockNet.createNode(MockNodeParameters(legalName = name), nodeFactory = { args ->
object : MockNetwork.MockNode(args) {
@ -307,7 +308,7 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
@Test
fun `check dependencies of sale asset are resolved`() {
mockNet = MockNetwork(cordappPackages = cordappPackages)
val notaryNode = mockNet.createNotaryNode()
val notaryNode = mockNet.defaultNotaryNode
val aliceNode = makeNodeWithTracking(ALICE_NAME)
val bobNode = makeNodeWithTracking(BOB_NAME)
val bankNode = makeNodeWithTracking(BOC_NAME)
@ -413,7 +414,7 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
@Test
fun `track works`() {
mockNet = MockNetwork(cordappPackages = cordappPackages)
val notaryNode = mockNet.createNotaryNode()
val notaryNode = mockNet.defaultNotaryNode
val aliceNode = makeNodeWithTracking(ALICE_NAME)
val bobNode = makeNodeWithTracking(BOB_NAME)
val bankNode = makeNodeWithTracking(BOC_NAME)
@ -568,7 +569,7 @@ class TwoPartyTradeFlowTests(val anonymous: Boolean) {
aliceError: Boolean,
expectedMessageSubstring: String
) {
val notaryNode = mockNet.createNotaryNode()
val notaryNode = mockNet.defaultNotaryNode
val aliceNode = mockNet.createPartyNode(ALICE_NAME)
val bobNode = mockNet.createPartyNode(BOB_NAME)
val bankNode = mockNet.createPartyNode(BOC_NAME)

View File

@ -1,7 +1,5 @@
package net.corda.node.services
import com.nhaarman.mockito_kotlin.doReturn
import com.nhaarman.mockito_kotlin.whenever
import net.corda.core.contracts.*
import net.corda.core.crypto.generateKeyPair
import net.corda.core.flows.NotaryChangeFlow
@ -15,14 +13,10 @@ import net.corda.core.transactions.WireTransaction
import net.corda.core.utilities.getOrThrow
import net.corda.core.utilities.seconds
import net.corda.node.internal.StartedNode
import net.corda.node.services.config.NotaryConfig
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.chooseIdentity
import net.corda.testing.*
import net.corda.testing.contracts.DummyContract
import net.corda.testing.dummyCommand
import net.corda.testing.getTestPartyAndCertificate
import net.corda.testing.node.MockNetwork
import net.corda.testing.node.MockNodeParameters
import net.corda.testing.node.MockNetwork.NotarySpec
import org.assertj.core.api.Assertions.assertThatExceptionOfType
import org.junit.After
import org.junit.Before
@ -42,23 +36,17 @@ class NotaryChangeTests {
@Before
fun setUp() {
mockNet = MockNetwork(cordappPackages = listOf("net.corda.testing.contracts"))
val (oldNotaryNode, newNotaryNode) = listOf(
createUnstartedNotary(DUMMY_NOTARY.name),
createUnstartedNotary(DUMMY_NOTARY.name.copy(organisation = "Dummy Notary 2"))
).map { it.start() }
this.oldNotaryNode = oldNotaryNode
val oldNotaryName = DUMMY_REGULATOR.name
mockNet = MockNetwork(
notarySpecs = listOf(NotarySpec(DUMMY_NOTARY.name), NotarySpec(oldNotaryName)),
cordappPackages = listOf("net.corda.testing.contracts")
)
clientNodeA = mockNet.createNode()
clientNodeB = mockNet.createNode()
oldNotaryParty = newNotaryNode.services.networkMapCache.getNotary(DUMMY_NOTARY.name)!!
newNotaryParty = newNotaryNode.services.networkMapCache.getNotary(DUMMY_NOTARY.name.copy(organisation = "Dummy Notary 2"))!!
}
private fun createUnstartedNotary(name: CordaX500Name): MockNetwork.MockNode {
return mockNet.createUnstartedNode(MockNodeParameters(
legalName = name,
configOverrides = { doReturn(NotaryConfig(validating = true)).whenever(it).notary }
))
oldNotaryNode = mockNet.notaryNodes[1]
mockNet.runNetwork() // Clear network map registration messages
newNotaryParty = clientNodeA.services.networkMapCache.getNotary(DUMMY_NOTARY.name)!!
oldNotaryParty = clientNodeA.services.networkMapCache.getNotary(oldNotaryName)!!
}
@After

View File

@ -35,7 +35,6 @@ class ScheduledFlowTests {
}
lateinit var mockNet: MockNetwork
lateinit var notaryNode: StartedNode<MockNetwork.MockNode>
lateinit var nodeA: StartedNode<MockNetwork.MockNode>
lateinit var nodeB: StartedNode<MockNetwork.MockNode>
@ -94,7 +93,6 @@ class ScheduledFlowTests {
@Before
fun setup() {
mockNet = MockNetwork(threadPerNode = true, cordappPackages = listOf("net.corda.testing.contracts"))
notaryNode = mockNet.createNotaryNode()
val a = mockNet.createUnstartedNode()
val b = mockNet.createUnstartedNode()

View File

@ -38,13 +38,13 @@ class NetworkMapCacheTest {
@Test
fun `getNodeByLegalIdentity`() {
val notaryNode = mockNet.createNotaryNode()
val aliceNode = mockNet.createPartyNode(ALICE.name)
val notaryCache: NetworkMapCache = notaryNode.services.networkMapCache
val bobNode = mockNet.createPartyNode(BOB.name)
val bobCache: NetworkMapCache = bobNode.services.networkMapCache
val expected = aliceNode.info
mockNet.runNetwork()
val actual = notaryNode.database.transaction { notaryCache.getNodeByLegalIdentity(aliceNode.info.chooseIdentity()) }
val actual = bobNode.database.transaction { bobCache.getNodeByLegalIdentity(aliceNode.info.chooseIdentity()) }
assertEquals(expected, actual)
// TODO: Should have a test case with anonymous lookup
@ -52,30 +52,30 @@ class NetworkMapCacheTest {
@Test
fun `getPeerByLegalName`() {
val notaryNode = mockNet.createNotaryNode()
val aliceNode = mockNet.createPartyNode(ALICE.name)
val notaryCache: NetworkMapCache = notaryNode.services.networkMapCache
val bobNode = mockNet.createPartyNode(BOB.name)
val bobCache: NetworkMapCache = bobNode.services.networkMapCache
val expected = aliceNode.info.legalIdentities.single()
mockNet.runNetwork()
val actual = notaryNode.database.transaction { notaryCache.getPeerByLegalName(ALICE.name) }
val actual = bobNode.database.transaction { bobCache.getPeerByLegalName(ALICE.name) }
assertEquals(expected, actual)
}
@Test
fun `remove node from cache`() {
val notaryNode = mockNet.createNotaryNode()
val aliceNode = mockNet.createPartyNode(ALICE.name)
val notaryLegalIdentity = notaryNode.info.chooseIdentity()
val bobNode = mockNet.createPartyNode(BOB.name)
val bobLegalIdentity = bobNode.info.chooseIdentity()
val alice = aliceNode.info.chooseIdentity()
val notaryCache = notaryNode.services.networkMapCache
val bobCache = bobNode.services.networkMapCache
mockNet.runNetwork()
notaryNode.database.transaction {
assertThat(notaryCache.getNodeByLegalIdentity(alice) != null)
notaryCache.removeNode(aliceNode.info)
assertThat(notaryCache.getNodeByLegalIdentity(alice) == null)
assertThat(notaryCache.getNodeByLegalIdentity(notaryLegalIdentity) != null)
assertThat(notaryCache.getNodeByLegalName(alice.name) == null)
bobNode.database.transaction {
assertThat(bobCache.getNodeByLegalIdentity(alice) != null)
bobCache.removeNode(aliceNode.info)
assertThat(bobCache.getNodeByLegalIdentity(alice) == null)
assertThat(bobCache.getNodeByLegalIdentity(bobLegalIdentity) != null)
assertThat(bobCache.getNodeByLegalName(alice.name) == null)
}
}
}

View File

@ -43,9 +43,7 @@ class NodeSchemaServiceTest {
@Test
fun `auto scanning of custom schemas for testing with Driver`() {
driver(startNodesInProcess = true) {
val node = startNode()
val nodeHandle = node.getOrThrow()
val result = nodeHandle.rpc.startFlow(::MappedSchemasFlow)
val result = defaultNotaryNode.getOrThrow().rpc.startFlow(::MappedSchemasFlow)
val mappedSchemas = result.returnValue.getOrThrow()
assertTrue(mappedSchemas.contains(TestSchema.name))
}
@ -54,11 +52,12 @@ class NodeSchemaServiceTest {
@Test
fun `custom schemas are loaded eagerly`() {
val expected = setOf("PARENTS", "CHILDREN")
assertEquals<Set<*>>(expected, driver {
(startNode(startInSameProcess = true).getOrThrow() as NodeHandle.InProcess).node.database.transaction {
val tables = driver(startNodesInProcess = true) {
(defaultNotaryNode.getOrThrow() as NodeHandle.InProcess).node.database.transaction {
session.createNativeQuery("SELECT TABLE_NAME FROM INFORMATION_SCHEMA.TABLES").list()
}
}.toMutableSet().apply { retainAll(expected) })
}
assertEquals<Set<*>>(expected, tables.toMutableSet().apply { retainAll(expected) })
}
@StartableByRPC

View File

@ -74,8 +74,8 @@ class FlowFrameworkTests {
fun start() {
mockNet = MockNetwork(
servicePeerAllocationStrategy = RoundRobin(),
cordappPackages = listOf("net.corda.finance.contracts", "net.corda.testing.contracts"))
val notary = mockNet.createNotaryNode()
cordappPackages = listOf("net.corda.finance.contracts", "net.corda.testing.contracts")
)
aliceNode = mockNet.createNode(MockNodeParameters(legalName = ALICE_NAME))
bobNode = mockNet.createNode(MockNodeParameters(legalName = BOB_NAME))
@ -84,7 +84,7 @@ class FlowFrameworkTests {
// Extract identities
alice = aliceNode.info.singleIdentity()
bob = bobNode.info.singleIdentity()
notaryIdentity = notary.services.getDefaultNotary()
notaryIdentity = aliceNode.services.getDefaultNotary()
}
@After

View File

@ -14,10 +14,13 @@ import net.corda.core.transactions.TransactionBuilder
import net.corda.core.utilities.getOrThrow
import net.corda.core.utilities.seconds
import net.corda.node.services.api.StartedNodeServices
import net.corda.testing.*
import net.corda.testing.ALICE_NAME
import net.corda.testing.contracts.DummyContract
import net.corda.testing.dummyCommand
import net.corda.testing.getDefaultNotary
import net.corda.testing.node.MockNetwork
import net.corda.testing.node.MockNodeParameters
import net.corda.testing.singleIdentity
import org.assertj.core.api.Assertions.assertThat
import org.junit.After
import org.junit.Before
@ -28,20 +31,19 @@ import kotlin.test.assertEquals
import kotlin.test.assertFailsWith
class NotaryServiceTests {
lateinit var mockNet: MockNetwork
lateinit var notaryServices: StartedNodeServices
lateinit var aliceServices: StartedNodeServices
lateinit var notary: Party
lateinit var alice: Party
private lateinit var mockNet: MockNetwork
private lateinit var notaryServices: StartedNodeServices
private lateinit var aliceServices: StartedNodeServices
private lateinit var notary: Party
private lateinit var alice: Party
@Before
fun setup() {
mockNet = MockNetwork(cordappPackages = listOf("net.corda.testing.contracts"))
val notaryNode = mockNet.createNotaryNode(validating = false)
aliceServices = mockNet.createNode(MockNodeParameters(legalName = ALICE_NAME)).services
mockNet.runNetwork() // Clear network map registration messages
notaryServices = notaryNode.services
notary = notaryServices.getDefaultNotary()
notaryServices = mockNet.defaultNotaryNode.services //TODO get rid of that
notary = aliceServices.getDefaultNotary()
alice = aliceServices.myInfo.singleIdentity()
}
@ -155,7 +157,7 @@ class NotaryServiceTests {
return future
}
fun issueState(services: ServiceHub, identity: Party): StateAndRef<*> {
private fun issueState(services: ServiceHub, identity: Party): StateAndRef<*> {
val tx = DummyContract.generateInitial(Random().nextInt(), notary, identity.ref(0))
val signedByNode = services.signInitialTransaction(tx)
val stx = notaryServices.addSignature(signedByNode, notary.owningKey)

View File

@ -15,10 +15,13 @@ import net.corda.core.transactions.TransactionBuilder
import net.corda.core.utilities.getOrThrow
import net.corda.node.services.api.StartedNodeServices
import net.corda.node.services.issueInvalidState
import net.corda.testing.*
import net.corda.testing.ALICE_NAME
import net.corda.testing.MEGA_CORP_KEY
import net.corda.testing.contracts.DummyContract
import net.corda.testing.dummyCommand
import net.corda.testing.node.MockNetwork
import net.corda.testing.node.MockNodeParameters
import net.corda.testing.singleIdentity
import org.assertj.core.api.Assertions.assertThat
import org.junit.After
import org.junit.Before
@ -28,21 +31,20 @@ import kotlin.test.assertEquals
import kotlin.test.assertFailsWith
class ValidatingNotaryServiceTests {
lateinit var mockNet: MockNetwork
lateinit var notaryServices: StartedNodeServices
lateinit var aliceServices: StartedNodeServices
lateinit var notary: Party
lateinit var alice: Party
private lateinit var mockNet: MockNetwork
private lateinit var notaryServices: StartedNodeServices
private lateinit var aliceServices: StartedNodeServices
private lateinit var notary: Party
private lateinit var alice: Party
@Before
fun setup() {
mockNet = MockNetwork(cordappPackages = listOf("net.corda.testing.contracts"))
val notaryNode = mockNet.createNotaryNode()
val aliceNode = mockNet.createNode(MockNodeParameters(legalName = ALICE_NAME))
mockNet.runNetwork() // Clear network map registration messages
notaryServices = notaryNode.services
notaryServices = mockNet.defaultNotaryNode.services
aliceServices = aliceNode.services
notary = notaryServices.getDefaultNotary()
notary = mockNet.defaultNotaryIdentity
alice = aliceNode.info.singleIdentity()
}
@ -97,7 +99,7 @@ class ValidatingNotaryServiceTests {
return future
}
fun issueState(serviceHub: ServiceHub, identity: Party): StateAndRef<*> {
private fun issueState(serviceHub: ServiceHub, identity: Party): StateAndRef<*> {
val tx = DummyContract.generateInitial(Random().nextInt(), notary, identity.ref(0))
val signedByNode = serviceHub.signInitialTransaction(tx)
val stx = notaryServices.addSignature(signedByNode, notary.owningKey)