send and sendAndReceive use Party for the destination

This commit is contained in:
Shams Asari
2016-06-30 11:18:28 +01:00
parent aef111114f
commit a02263937c
39 changed files with 237 additions and 220 deletions

View File

@ -33,6 +33,7 @@ import com.r3corda.node.services.keys.E2ETestKeyManagementService
import com.r3corda.node.services.network.InMemoryNetworkMapCache
import com.r3corda.node.services.network.InMemoryNetworkMapService
import com.r3corda.node.services.network.NetworkMapService
import com.r3corda.node.services.network.NetworkMapService.Companion.REGISTER_PROTOCOL_TOPIC
import com.r3corda.node.services.network.NodeRegistration
import com.r3corda.node.services.persistence.*
import com.r3corda.node.services.statemachine.StateMachineManager
@ -50,7 +51,6 @@ import java.nio.file.FileAlreadyExistsException
import java.nio.file.Files
import java.nio.file.Path
import java.security.KeyPair
import java.security.Security
import java.time.Clock
import java.util.*
@ -160,8 +160,8 @@ abstract class AbstractNode(val dir: Path, val configuration: NodeConfiguration,
// This object doesn't need to be referenced from this class because it registers handlers on the network
// service and so that keeps it from being collected.
DataVendingService(net, storage)
NotaryChangeService(net, smm)
DataVendingService(net, storage, services.networkMapCache)
NotaryChangeService(net, smm, services.networkMapCache)
buildAdvertisedServices()
@ -231,9 +231,9 @@ abstract class AbstractNode(val dir: Path, val configuration: NodeConfiguration,
val reg = NodeRegistration(info, networkMapSeq++, type, expires)
val sessionID = random63BitValue()
val request = NetworkMapService.RegistrationRequest(reg.toWire(storage.myLegalIdentityKey.private), net.myAddress, sessionID)
val message = net.createMessage(NetworkMapService.REGISTER_PROTOCOL_TOPIC + ".0", request.serialize().bits)
val message = net.createMessage("$REGISTER_PROTOCOL_TOPIC.0", request.serialize().bits)
val future = SettableFuture.create<NetworkMapService.RegistrationResponse>()
val topic = NetworkMapService.REGISTER_PROTOCOL_TOPIC + "." + sessionID
val topic = "$REGISTER_PROTOCOL_TOPIC.$sessionID"
net.runOnNextMessage(topic, RunOnCallerThread) { message ->
future.set(message.data.deserialize())
@ -254,8 +254,8 @@ abstract class AbstractNode(val dir: Path, val configuration: NodeConfiguration,
val timestampChecker = TimestampChecker(platformClock, 30.seconds)
inNodeNotaryService = when (type) {
is SimpleNotaryService.Type -> SimpleNotaryService(smm, net, timestampChecker, uniquenessProvider)
is ValidatingNotaryService.Type -> ValidatingNotaryService(smm, net, timestampChecker, uniquenessProvider)
is SimpleNotaryService.Type -> SimpleNotaryService(smm, net, timestampChecker, uniquenessProvider, services.networkMapCache)
is ValidatingNotaryService.Type -> ValidatingNotaryService(smm, net, timestampChecker, uniquenessProvider, services.networkMapCache)
else -> null
}
}

View File

@ -130,8 +130,8 @@ class IRSSimulation(networkSendManuallyPumped: Boolean, runAsync: Boolean, laten
val sessionID = random63BitValue()
val instigator = TwoPartyDealProtocol.Instigator(node2.net.myAddress, notary.info.identity, irs, nodeAKey!!, sessionID)
val acceptor = TwoPartyDealProtocol.Acceptor(node1.net.myAddress, notary.info.identity, irs, sessionID)
val instigator = TwoPartyDealProtocol.Instigator(node2.info.identity, notary.info.identity, irs, nodeAKey!!, sessionID)
val acceptor = TwoPartyDealProtocol.Acceptor(node1.info.identity, notary.info.identity, irs, sessionID)
showProgressFor(listOf(node1, node2))
showConsensusFor(listOf(node1, node2, regulators[0]))

View File

@ -44,10 +44,19 @@ class TradeSimulation(runAsync: Boolean, latencyInjector: InMemoryMessagingNetwo
val cashIssuerKey = generateKeyPair()
val amount = 1000.DOLLARS `issued by` Party("Big friendly bank", cashIssuerKey.public).ref(1)
val sessionID = random63BitValue()
val buyerProtocol = TwoPartyTradeProtocol.Buyer(seller.net.myAddress, notary.info.identity,
amount, CommercialPaper.State::class.java, sessionID)
val sellerProtocol = TwoPartyTradeProtocol.Seller(buyer.net.myAddress, notary.info,
issuance.tx.outRef(0), amount, seller.storage.myLegalIdentityKey, sessionID)
val buyerProtocol = TwoPartyTradeProtocol.Buyer(
seller.info.identity,
notary.info.identity,
amount,
CommercialPaper.State::class.java,
sessionID)
val sellerProtocol = TwoPartyTradeProtocol.Seller(
buyer.info.identity,
notary.info,
issuance.tx.outRef(0),
amount,
seller.storage.myLegalIdentityKey,
sessionID)
showConsensusFor(listOf(buyer, seller, notary))
showProgressFor(listOf(buyer, seller))

View File

@ -2,7 +2,7 @@ package com.r3corda.node.services
import com.r3corda.core.messaging.Ack
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.node.services.api.AbstractNodeService
import com.r3corda.node.services.statemachine.StateMachineManager
import protocols.AbstractStateReplacementProtocol
@ -12,7 +12,7 @@ import protocols.NotaryChangeProtocol
* A service that monitors the network for requests for changing the notary of a state,
* and immediately runs the [NotaryChangeProtocol] if the auto-accept criteria are met.
*/
class NotaryChangeService(net: MessagingService, val smm: StateMachineManager) : AbstractNodeService(net) {
class NotaryChangeService(net: MessagingService, val smm: StateMachineManager, networkMapCache: NetworkMapCache) : AbstractNodeService(net, networkMapCache) {
init {
addMessageHandler(NotaryChangeProtocol.TOPIC_INITIATE,
{ req: AbstractStateReplacementProtocol.Handshake -> handleChangeNotaryRequest(req) }
@ -21,8 +21,8 @@ class NotaryChangeService(net: MessagingService, val smm: StateMachineManager) :
private fun handleChangeNotaryRequest(req: AbstractStateReplacementProtocol.Handshake): Ack {
val protocol = NotaryChangeProtocol.Acceptor(
req.replyTo as SingleMessageRecipient,
req.sessionID!!,
req.replyToParty,
req.sessionID,
req.sessionIdForSend)
smm.add(NotaryChangeProtocol.TOPIC_CHANGE, protocol)
return Ack

View File

@ -2,18 +2,19 @@ package com.r3corda.node.services.api
import com.r3corda.core.messaging.Message
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.core.node.services.TOPIC_DEFAULT_POSTFIX
import com.r3corda.core.serialization.SingletonSerializeAsToken
import com.r3corda.core.serialization.deserialize
import com.r3corda.core.serialization.serialize
import com.r3corda.protocols.AbstractRequestMessage
import com.r3corda.protocols.ServiceRequestMessage
import javax.annotation.concurrent.ThreadSafe
/**
* Abstract superclass for services that a node can host, which provides helper functions.
*/
@ThreadSafe
abstract class AbstractNodeService(val net: MessagingService) : SingletonSerializeAsToken() {
abstract class AbstractNodeService(val net: MessagingService, val networkMapCache: NetworkMapCache) : SingletonSerializeAsToken() {
/**
* Register a handler for a message topic. In comparison to using net.addMessageHandler() this manages a lot of
@ -24,18 +25,18 @@ abstract class AbstractNodeService(val net: MessagingService) : SingletonSeriali
* @param handler a function to handle the deserialised request and return an optional response (if return type not Unit)
* @param exceptionConsumer a function to which any thrown exception is passed.
*/
protected inline fun <reified Q : AbstractRequestMessage, reified R : Any>
protected inline fun <reified Q : ServiceRequestMessage, reified R : Any>
addMessageHandler(topic: String,
crossinline handler: (Q) -> R,
crossinline exceptionConsumer: (Message, Exception) -> Unit) {
net.addMessageHandler(topic + TOPIC_DEFAULT_POSTFIX, null) { message, r ->
try {
val req = message.data.deserialize<Q>()
val data = handler(req)
val request = message.data.deserialize<Q>()
val response = handler(request)
// If the return type R is Unit, then do not send a response
if (data.javaClass != Unit.javaClass) {
val msg = net.createMessage("$topic.${req.sessionID}", data.serialize().bits)
net.send(msg, req.replyTo)
if (response.javaClass != Unit.javaClass) {
val msg = net.createMessage("$topic.${request.sessionID}", response.serialize().bits)
net.send(msg, request.getReplyTo(networkMapCache))
}
} catch(e: Exception) {
exceptionConsumer(message, e)
@ -51,7 +52,7 @@ abstract class AbstractNodeService(val net: MessagingService) : SingletonSeriali
* @param topic the topic, without the default session ID postfix (".0)
* @param handler a function to handle the deserialised request and return an optional response (if return type not Unit)
*/
protected inline fun <reified Q : AbstractRequestMessage, reified R : Any>
protected inline fun <reified Q : ServiceRequestMessage, reified R : Any>
addMessageHandler(topic: String,
crossinline handler: (Q) -> R) {
addMessageHandler(topic, handler, { message: Message, exception: Exception -> throw exception })

View File

@ -41,7 +41,7 @@ object NodeInterestRates {
/**
* The Service that wraps [Oracle] and handles messages/network interaction/request scrubbing.
*/
class Service(node: AbstractNode) : AcceptsFileUpload, AbstractNodeService(node.services.networkService) {
class Service(node: AbstractNode) : AcceptsFileUpload, AbstractNodeService(node.services.networkService, node.services.networkMapCache) {
val ss = node.services.storageService
val oracle = Oracle(ss.myLegalIdentity, ss.myLegalIdentityKey, node.services.clock)
@ -84,7 +84,7 @@ object NodeInterestRates {
override fun call(): Unit {
val answers = service.oracle.query(request.queries, request.deadline)
progressTracker.currentStep = SENDING
send("${RatesFixProtocol.TOPIC}.query", request.replyTo, request.sessionID!!, answers)
send("${RatesFixProtocol.TOPIC}.query", request.replyToParty, request.sessionID, answers)
}
}

View File

@ -7,16 +7,16 @@ import com.r3corda.core.messaging.MessageRecipients
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.NodeInfo
import com.r3corda.core.node.services.ServiceType
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.core.node.services.ServiceType
import com.r3corda.core.node.services.TOPIC_DEFAULT_POSTFIX
import com.r3corda.core.serialization.SerializedBytes
import com.r3corda.core.serialization.deserialize
import com.r3corda.core.serialization.serialize
import com.r3corda.node.services.api.AbstractNodeService
import com.r3corda.node.utilities.AddOrRemove
import com.r3corda.protocols.ServiceRequestMessage
import org.slf4j.LoggerFactory
import com.r3corda.protocols.AbstractRequestMessage
import java.security.PrivateKey
import java.security.SignatureException
import java.time.Instant
@ -61,20 +61,25 @@ interface NetworkMapService {
val nodes: List<NodeInfo>
class FetchMapRequest(val subscribe: Boolean, val ifChangedSinceVersion: Int?, replyTo: MessageRecipients, sessionID: Long) : AbstractRequestMessage(replyTo, sessionID)
abstract class NetworkMapRequestMessage(val replyTo: MessageRecipients) : ServiceRequestMessage {
override fun getReplyTo(networkMapCache: NetworkMapCache): MessageRecipients = replyTo
}
class FetchMapRequest(val subscribe: Boolean, val ifChangedSinceVersion: Int?, replyTo: MessageRecipients, override val sessionID: Long) : NetworkMapRequestMessage(replyTo)
data class FetchMapResponse(val nodes: Collection<NodeRegistration>?, val version: Int)
class QueryIdentityRequest(val identity: Party, replyTo: MessageRecipients, sessionID: Long) : AbstractRequestMessage(replyTo, sessionID)
class QueryIdentityRequest(val identity: Party, replyTo: MessageRecipients, override val sessionID: Long) : NetworkMapRequestMessage(replyTo)
data class QueryIdentityResponse(val node: NodeInfo?)
class RegistrationRequest(val wireReg: WireNodeRegistration, replyTo: MessageRecipients, sessionID: Long) : AbstractRequestMessage(replyTo, sessionID)
class RegistrationRequest(val wireReg: WireNodeRegistration, replyTo: MessageRecipients, override val sessionID: Long) : NetworkMapRequestMessage(replyTo)
data class RegistrationResponse(val success: Boolean)
class SubscribeRequest(val subscribe: Boolean, replyTo: MessageRecipients, sessionID: Long) : AbstractRequestMessage(replyTo, sessionID)
class SubscribeRequest(val subscribe: Boolean, replyTo: MessageRecipients, override val sessionID: Long) : NetworkMapRequestMessage(replyTo)
data class SubscribeResponse(val confirmed: Boolean)
data class Update(val wireReg: WireNodeRegistration, val replyTo: MessageRecipients)
data class UpdateAcknowledge(val wireRegHash: SecureHash, val replyTo: MessageRecipients)
}
@ThreadSafe
class InMemoryNetworkMapService(net: MessagingService, home: NodeRegistration, val cache: NetworkMapCache) : NetworkMapService, AbstractNodeService(net) {
class InMemoryNetworkMapService(net: MessagingService, home: NodeRegistration, val cache: NetworkMapCache) : NetworkMapService, AbstractNodeService(net, cache) {
private val registeredNodes = ConcurrentHashMap<Party, NodeRegistration>()
// Map from subscriber address, to a list of unacknowledged updates
private val subscribers = ThreadBox(mutableMapOf<SingleMessageRecipient, MutableList<SecureHash>>())

View File

@ -2,6 +2,7 @@ package com.r3corda.node.services.persistence
import com.r3corda.core.contracts.SignedTransaction
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.core.node.services.StorageService
import com.r3corda.core.utilities.loggerFor
import com.r3corda.node.services.api.AbstractNodeService
@ -24,7 +25,7 @@ import javax.annotation.concurrent.ThreadSafe
* Additionally, because nodes do not store invalid transactions, requesting such a transaction will always yield null.
*/
@ThreadSafe
class DataVendingService(net: MessagingService, private val storage: StorageService) : AbstractNodeService(net) {
class DataVendingService(net: MessagingService, private val storage: StorageService, networkMapCache: NetworkMapCache) : AbstractNodeService(net, networkMapCache) {
companion object {
val logger = loggerFor<DataVendingService>()
}

View File

@ -5,7 +5,7 @@ import co.paralleluniverse.fibers.FiberScheduler
import co.paralleluniverse.fibers.Suspendable
import com.google.common.util.concurrent.ListenableFuture
import com.google.common.util.concurrent.SettableFuture
import com.r3corda.core.messaging.MessageRecipients
import com.r3corda.core.crypto.Party
import com.r3corda.core.protocols.ProtocolLogic
import com.r3corda.core.protocols.ProtocolStateMachine
import com.r3corda.core.utilities.UntrustworthyData
@ -81,9 +81,13 @@ class ProtocolStateMachineImpl<R>(val logic: ProtocolLogic<R>,
}
@Suspendable @Suppress("UNCHECKED_CAST")
override fun <T : Any> sendAndReceive(topic: String, destination: MessageRecipients, sessionIDForSend: Long, sessionIDForReceive: Long,
obj: Any, recvType: Class<T>): UntrustworthyData<T> {
val result = StateMachineManager.FiberRequest.ExpectingResponse(topic, destination, sessionIDForSend, sessionIDForReceive, obj, recvType)
override fun <T : Any> sendAndReceive(topic: String,
destination: Party,
sessionIDForSend: Long,
sessionIDForReceive: Long,
payload: Any,
recvType: Class<T>): UntrustworthyData<T> {
val result = StateMachineManager.FiberRequest.ExpectingResponse(topic, destination, sessionIDForSend, sessionIDForReceive, payload, recvType)
return suspendAndExpectReceive(result)
}
@ -94,8 +98,8 @@ class ProtocolStateMachineImpl<R>(val logic: ProtocolLogic<R>,
}
@Suspendable
override fun send(topic: String, destination: MessageRecipients, sessionID: Long, obj: Any) {
val result = StateMachineManager.FiberRequest.NotExpectingResponse(topic, destination, sessionID, obj)
override fun send(topic: String, destination: Party, sessionID: Long, payload: Any) {
val result = StateMachineManager.FiberRequest.NotExpectingResponse(topic, destination, sessionID, payload)
suspend(result)
}

View File

@ -8,7 +8,7 @@ import com.esotericsoftware.kryo.Kryo
import com.google.common.base.Throwables
import com.google.common.util.concurrent.ListenableFuture
import com.r3corda.core.abbreviate
import com.r3corda.core.messaging.MessageRecipients
import com.r3corda.core.crypto.Party
import com.r3corda.core.messaging.runOnNextMessage
import com.r3corda.core.messaging.send
import com.r3corda.core.protocols.ProtocolLogic
@ -253,7 +253,8 @@ class StateMachineManager(val serviceHub: ServiceHubInternal, tokenizableService
request.payload?.let {
val topic = "${request.topic}.${request.sessionIDForSend}"
psm.logger.trace { "Sending message of type ${it.javaClass.name} using topic $topic to ${request.destination} (${it.toString().abbreviate(50)})" }
serviceHub.networkService.send(topic, it, request.destination!!)
val address = serviceHub.networkMapCache.getNodeByLegalName(request.destination!!.name)!!.address
serviceHub.networkService.send(topic, it, address)
}
if (request is FiberRequest.NotExpectingResponse) {
// We sent a message, but don't expect a response, so re-enter the continuation to let it keep going.
@ -307,7 +308,7 @@ class StateMachineManager(val serviceHub: ServiceHubInternal, tokenizableService
// TODO: Clean this up
open class FiberRequest(val topic: String,
val destination: MessageRecipients?,
val destination: Party?,
val sessionIDForSend: Long,
val sessionIDForReceive: Long,
val payload: Any?) {
@ -317,7 +318,7 @@ class StateMachineManager(val serviceHub: ServiceHubInternal, tokenizableService
class ExpectingResponse<R : Any>(
topic: String,
destination: MessageRecipients?,
destination: Party?,
sessionIDForSend: Long,
sessionIDForReceive: Long,
obj: Any?,
@ -326,7 +327,7 @@ class StateMachineManager(val serviceHub: ServiceHubInternal, tokenizableService
class NotExpectingResponse(
topic: String,
destination: MessageRecipients,
destination: Party,
sessionIDForSend: Long,
obj: Any?
) : FiberRequest(topic, destination, sessionIDForSend, -1, obj)

View File

@ -2,7 +2,7 @@ package com.r3corda.node.services.transactions
import com.r3corda.core.messaging.Ack
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.core.node.services.ServiceType
import com.r3corda.core.node.services.TimestampChecker
import com.r3corda.core.node.services.UniquenessProvider
@ -22,7 +22,8 @@ import com.r3corda.protocols.NotaryProtocol
abstract class NotaryService(val smm: StateMachineManager,
net: MessagingService,
val timestampChecker: TimestampChecker,
val uniquenessProvider: UniquenessProvider) : AbstractNodeService(net) {
val uniquenessProvider: UniquenessProvider,
networkMapCache: NetworkMapCache) : AbstractNodeService(net, networkMapCache) {
object Type : ServiceType("corda.notary")
abstract val logger: org.slf4j.Logger
@ -37,8 +38,9 @@ abstract class NotaryService(val smm: StateMachineManager,
}
private fun processRequest(req: NotaryProtocol.Handshake): Ack {
val protocol = protocolFactory.create(req.replyTo as SingleMessageRecipient,
req.sessionID!!,
val protocol = protocolFactory.create(
req.replyToParty,
req.sessionID,
req.sendSessionID,
timestampChecker,
uniquenessProvider)

View File

@ -1,6 +1,7 @@
package com.r3corda.node.services.transactions
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.core.node.services.ServiceType
import com.r3corda.core.node.services.TimestampChecker
import com.r3corda.core.node.services.UniquenessProvider
@ -13,7 +14,8 @@ class SimpleNotaryService(
smm: StateMachineManager,
net: MessagingService,
timestampChecker: TimestampChecker,
uniquenessProvider: UniquenessProvider) : NotaryService(smm, net, timestampChecker, uniquenessProvider) {
uniquenessProvider: UniquenessProvider,
networkMapCache: NetworkMapCache) : NotaryService(smm, net, timestampChecker, uniquenessProvider, networkMapCache) {
object Type : ServiceType("corda.notary.simple")
override val logger = loggerFor<SimpleNotaryService>()

View File

@ -1,7 +1,8 @@
package com.r3corda.node.services.transactions
import com.r3corda.core.crypto.Party
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.services.NetworkMapCache
import com.r3corda.core.node.services.ServiceType
import com.r3corda.core.node.services.TimestampChecker
import com.r3corda.core.node.services.UniquenessProvider
@ -15,14 +16,15 @@ class ValidatingNotaryService(
smm: StateMachineManager,
net: MessagingService,
timestampChecker: TimestampChecker,
uniquenessProvider: UniquenessProvider
) : NotaryService(smm, net, timestampChecker, uniquenessProvider) {
uniquenessProvider: UniquenessProvider,
networkMapCache: NetworkMapCache
) : NotaryService(smm, net, timestampChecker, uniquenessProvider, networkMapCache) {
object Type : ServiceType("corda.notary.validating")
override val logger = loggerFor<ValidatingNotaryService>()
override val protocolFactory = object : NotaryProtocol.Factory {
override fun create(otherSide: SingleMessageRecipient,
override fun create(otherSide: Party,
sendSessionID: Long,
receiveSessionID: Long,
timestampChecker: TimestampChecker,

View File

@ -59,7 +59,8 @@ class AttachmentTests {
val id = n0.storage.attachments.importAttachment(ByteArrayInputStream(fakeAttachment()))
// Get node one to run a protocol to fetch it and insert it.
val f1 = n1.smm.add("tests.fetch1", FetchAttachmentsProtocol(setOf(id), n0.net.myAddress))
network.runNetwork()
val f1 = n1.smm.add("tests.fetch1", FetchAttachmentsProtocol(setOf(id), n0.info.identity))
network.runNetwork()
assertEquals(0, f1.get().fromDisk.size)
@ -70,7 +71,7 @@ class AttachmentTests {
// Shut down node zero and ensure node one can still resolve the attachment.
n0.stop()
val response: FetchDataProtocol.Result<Attachment> = n1.smm.add("tests.fetch1", FetchAttachmentsProtocol(setOf(id), n0.net.myAddress)).get()
val response: FetchDataProtocol.Result<Attachment> = n1.smm.add("tests.fetch1", FetchAttachmentsProtocol(setOf(id), n0.info.identity)).get()
assertEquals(attachment, response.fromDisk[0])
}
@ -80,14 +81,15 @@ class AttachmentTests {
// Get node one to fetch a non-existent attachment.
val hash = SecureHash.randomSHA256()
val f1 = n1.smm.add("tests.fetch2", FetchAttachmentsProtocol(setOf(hash), n0.net.myAddress))
network.runNetwork()
val f1 = n1.smm.add("tests.fetch2", FetchAttachmentsProtocol(setOf(hash), n0.info.identity))
network.runNetwork()
val e = assertFailsWith<FetchDataProtocol.HashNotFound> { rootCauseExceptions { f1.get() } }
assertEquals(hash, e.requested)
}
@Test
fun maliciousResponse() {
fun `malicious response`() {
// Make a node that doesn't do sanity checking at load time.
val n0 = network.createNode(null, -1, object : MockNetwork.Factory {
override fun create(dir: Path, config: NodeConfiguration, network: MockNetwork, networkMapAddr: NodeInfo?,
@ -112,7 +114,8 @@ class AttachmentTests {
writer.close()
// Get n1 to fetch the attachment. Should receive corrupted bytes.
val f1 = n1.smm.add("tests.fetch1", FetchAttachmentsProtocol(setOf(id), n0.net.myAddress))
network.runNetwork()
val f1 = n1.smm.add("tests.fetch1", FetchAttachmentsProtocol(setOf(id), n0.info.identity))
network.runNetwork()
assertFailsWith<FetchDataProtocol.DownloadedVsRequestedDataMismatch> {
rootCauseExceptions { f1.get() }

View File

@ -11,7 +11,6 @@ import com.r3corda.core.contracts.*
import com.r3corda.core.crypto.Party
import com.r3corda.core.crypto.SecureHash
import com.r3corda.core.days
import com.r3corda.core.messaging.SingleMessageRecipient
import com.r3corda.core.node.NodeInfo
import com.r3corda.core.node.ServiceHub
import com.r3corda.core.node.services.ServiceType
@ -56,14 +55,14 @@ class TwoPartyTradeProtocolTests {
lateinit var net: MockNetwork
private fun runSeller(smm: StateMachineManager, notary: NodeInfo,
otherSide: SingleMessageRecipient, assetToSell: StateAndRef<OwnableState>, price: Amount<Issued<Currency>>,
otherSide: Party, assetToSell: StateAndRef<OwnableState>, price: Amount<Issued<Currency>>,
myKeyPair: KeyPair, buyerSessionID: Long): ListenableFuture<SignedTransaction> {
val seller = TwoPartyTradeProtocol.Seller(otherSide, notary, assetToSell, price, myKeyPair, buyerSessionID)
return smm.add("${TwoPartyTradeProtocol.TRADE_TOPIC}.seller", seller)
}
private fun runBuyer(smm: StateMachineManager, notaryNode: NodeInfo,
otherSide: SingleMessageRecipient, acceptablePrice: Amount<Issued<Currency>>, typeToBuy: Class<out OwnableState>,
otherSide: Party, acceptablePrice: Amount<Issued<Currency>>, typeToBuy: Class<out OwnableState>,
sessionID: Long): ListenableFuture<SignedTransaction> {
val buyer = TwoPartyTradeProtocol.Buyer(otherSide, notaryNode.identity, acceptablePrice, typeToBuy, sessionID)
return smm.add("${TwoPartyTradeProtocol.TRADE_TOPIC}.buyer", buyer)
@ -105,7 +104,7 @@ class TwoPartyTradeProtocolTests {
val bobResult = runBuyer(
bobNode.smm,
notaryNode.info,
aliceNode.net.myAddress,
aliceNode.info.identity,
1000.DOLLARS `issued by` issuer,
CommercialPaper.State::class.java,
buyerSessionID
@ -113,7 +112,7 @@ class TwoPartyTradeProtocolTests {
val aliceResult = runSeller(
aliceNode.smm,
notaryNode.info,
bobNode.net.myAddress,
bobNode.info.identity,
lookup("alice's paper"),
1000.DOLLARS `issued by` issuer,
ALICE_KEY,
@ -139,7 +138,6 @@ class TwoPartyTradeProtocolTests {
val aliceNode = net.createPartyNode(notaryNode.info, ALICE.name, ALICE_KEY)
var bobNode = net.createPartyNode(notaryNode.info, BOB.name, BOB_KEY)
val aliceAddr = aliceNode.net.myAddress
val bobAddr = bobNode.net.myAddress as InMemoryMessagingNetwork.Handle
val networkMapAddr = notaryNode.info
val issuer = bobNode.services.storageService.myLegalIdentity.ref(0)
@ -156,7 +154,7 @@ class TwoPartyTradeProtocolTests {
val aliceFuture = runSeller(
aliceNode.smm,
notaryNode.info,
bobAddr,
bobNode.info.identity,
lookup("alice's paper"),
1000.DOLLARS `issued by` issuer,
ALICE_KEY,
@ -165,7 +163,7 @@ class TwoPartyTradeProtocolTests {
runBuyer(
bobNode.smm,
notaryNode.info,
aliceAddr,
aliceNode.info.identity,
1000.DOLLARS `issued by` issuer,
CommercialPaper.State::class.java,
buyerSessionID
@ -276,7 +274,7 @@ class TwoPartyTradeProtocolTests {
runSeller(
aliceNode.smm,
notaryNode.info,
bobNode.net.myAddress,
bobNode.info.identity,
lookup("alice's paper"),
1000.DOLLARS `issued by` issuer,
ALICE_KEY,
@ -285,7 +283,7 @@ class TwoPartyTradeProtocolTests {
runBuyer(
bobNode.smm,
notaryNode.info,
aliceNode.net.myAddress,
aliceNode.info.identity,
1000.DOLLARS `issued by` issuer,
CommercialPaper.State::class.java,
buyerSessionID
@ -371,9 +369,6 @@ class TwoPartyTradeProtocolTests {
val bobNode = net.createPartyNode(notaryNode.info, BOB.name, BOB_KEY)
val issuer = MEGA_CORP.ref(1, 2, 3)
val aliceAddr = aliceNode.net.myAddress
val bobAddr = bobNode.net.myAddress as InMemoryMessagingNetwork.Handle
val bobKey = bobNode.keyManagement.freshKey()
val bobsBadCash = fillUpForBuyer(bobError, bobKey.public).second
val alicesFakePaper = fillUpForSeller(aliceError, aliceNode.storage.myLegalIdentity.owningKey,
@ -389,7 +384,7 @@ class TwoPartyTradeProtocolTests {
val aliceResult = runSeller(
aliceNode.smm,
notaryNode.info,
bobAddr,
bobNode.info.identity,
lookup("alice's paper"),
1000.DOLLARS `issued by` issuer,
ALICE_KEY,
@ -398,7 +393,7 @@ class TwoPartyTradeProtocolTests {
val bobResult = runBuyer(
bobNode.smm,
notaryNode.info,
aliceAddr,
aliceNode.info.identity,
1000.DOLLARS `issued by` issuer,
CommercialPaper.State::class.java,
buyerSessionID

View File

@ -59,7 +59,7 @@ class InMemoryNetworkMapServiceTest {
assertEquals(2, service.nodes.count())
// Confirm that de-registering the node succeeds and drops it from the node lists
var removeChange = NodeRegistration(registerNode.info, seq, AddOrRemove.REMOVE, expires)
val removeChange = NodeRegistration(registerNode.info, seq, AddOrRemove.REMOVE, expires)
val removeWireChange = removeChange.toWire(nodeKey.private)
assert(service.processRegistrationChangeRequest(NetworkMapService.RegistrationRequest(removeWireChange, mapServiceNode.info.address, Long.MIN_VALUE)).success)
assertNull(service.processQueryRequest(NetworkMapService.QueryIdentityRequest(registerNode.info.identity, mapServiceNode.info.address, Long.MIN_VALUE)).node)
@ -73,7 +73,7 @@ class InMemoryNetworkMapServiceTest {
@Suspendable
override fun call() {
val req = NetworkMapService.UpdateAcknowledge(hash, serviceHub.networkService.myAddress)
send(NetworkMapService.PUSH_ACK_PROTOCOL_TOPIC, server.address, 0, req)
send(NetworkMapService.PUSH_ACK_PROTOCOL_TOPIC, server.identity, 0, req)
}
}
@ -84,7 +84,7 @@ class InMemoryNetworkMapServiceTest {
val sessionID = random63BitValue()
val req = NetworkMapService.FetchMapRequest(subscribe, ifChangedSinceVersion, serviceHub.networkService.myAddress, sessionID)
return sendAndReceive<NetworkMapService.FetchMapResponse>(
NetworkMapService.FETCH_PROTOCOL_TOPIC, server.address, 0, sessionID, req)
NetworkMapService.FETCH_PROTOCOL_TOPIC, server.identity, 0, sessionID, req)
.validate { it.nodes }
}
}
@ -97,7 +97,7 @@ class InMemoryNetworkMapServiceTest {
val req = NetworkMapService.RegistrationRequest(reg.toWire(privateKey), serviceHub.networkService.myAddress, sessionID)
return sendAndReceive<NetworkMapService.RegistrationResponse>(
NetworkMapService.REGISTER_PROTOCOL_TOPIC, server.address, 0, sessionID, req)
NetworkMapService.REGISTER_PROTOCOL_TOPIC, server.identity, 0, sessionID, req)
.validate { it }
}
}
@ -110,19 +110,20 @@ class InMemoryNetworkMapServiceTest {
val req = NetworkMapService.SubscribeRequest(subscribe, serviceHub.networkService.myAddress, sessionID)
return sendAndReceive<NetworkMapService.SubscribeResponse>(
NetworkMapService.SUBSCRIPTION_PROTOCOL_TOPIC, server.address, 0, sessionID, req)
NetworkMapService.SUBSCRIPTION_PROTOCOL_TOPIC, server.identity, 0, sessionID, req)
.validate { it }
}
}
@Test
fun successWithNetwork() {
fun `success with network`() {
val (mapServiceNode, registerNode) = network.createTwoNodes()
// Confirm there's a network map service on node 0
assertNotNull(mapServiceNode.inNodeNetworkMapService)
// Confirm all nodes have registered themselves
network.runNetwork()
var fetchPsm = registerNode.smm.add(NetworkMapService.FETCH_PROTOCOL_TOPIC, TestFetchPSM(mapServiceNode.info, false))
network.runNetwork()
assertEquals(2, fetchPsm.get()?.count())
@ -143,11 +144,12 @@ class InMemoryNetworkMapServiceTest {
}
@Test
fun subscribeWithNetwork() {
fun `subscribe with network`() {
val (mapServiceNode, registerNode) = network.createTwoNodes()
val service = (mapServiceNode.inNodeNetworkMapService as InMemoryNetworkMapService)
// Test subscribing to updates
network.runNetwork()
val subscribePsm = registerNode.smm.add(NetworkMapService.SUBSCRIPTION_PROTOCOL_TOPIC,
TestSubscribePSM(mapServiceNode.info, true))
network.runNetwork()

View File

@ -1,8 +1,8 @@
package com.r3corda.node.services
import com.codahale.metrics.MetricRegistry
import com.r3corda.core.contracts.SignedTransaction
import com.google.common.util.concurrent.ListenableFuture
import com.r3corda.core.contracts.SignedTransaction
import com.r3corda.core.messaging.MessagingService
import com.r3corda.core.node.services.*
import com.r3corda.core.node.services.testing.MockStorageService
@ -68,7 +68,7 @@ open class MockServices(
if (net != null && storage != null) {
// Creating this class is sufficient, we don't have to store it anywhere, because it registers a listener
// on the networking service, so that will keep it from being collected.
DataVendingService(net, storage)
DataVendingService(net, storage, networkMapCache)
}
}
}

View File

@ -8,9 +8,9 @@ import com.r3corda.contracts.testing.`with notary`
import com.r3corda.core.bd
import com.r3corda.core.contracts.DOLLARS
import com.r3corda.core.contracts.Fix
import com.r3corda.core.contracts.TransactionType
import com.r3corda.core.crypto.Party
import com.r3corda.core.crypto.generateKeyPair
import com.r3corda.core.contracts.TransactionType
import com.r3corda.core.testing.ALICE_PUBKEY
import com.r3corda.core.testing.DUMMY_NOTARY
import com.r3corda.core.testing.MEGA_CORP
@ -109,8 +109,9 @@ class NodeInterestRatesTest {
val tx = TransactionType.General.Builder()
val fixOf = NodeInterestRates.parseFixOf("LIBOR 2016-03-16 1M")
val protocol = RatesFixProtocol(tx, n2.info, fixOf, "0.675".bd, "0.1".bd, Duration.ofNanos(1))
val protocol = RatesFixProtocol(tx, n2.info.identity, fixOf, "0.675".bd, "0.1".bd, Duration.ofNanos(1))
BriefLogFormatter.initVerbose("rates")
net.runNetwork()
val future = n1.smm.add("rates", protocol)
net.runNetwork()
@ -123,4 +124,4 @@ class NodeInterestRatesTest {
}
private fun makeTX() = TransactionType.General.Builder().withItems(1000.DOLLARS.CASH `issued by` DUMMY_CASH_ISSUER `owned by` ALICE_PUBKEY `with notary` DUMMY_NOTARY)
}
}