CORDA-1353: Notaries should reject transactions that contain too… (#2986)

CORDA-1353: Notaries should reject transactions that contain too many inputs. Otherwise, it may cause a slowdown or make the service hang.
This commit is contained in:
Andrius Dagys
2018-04-25 17:29:19 +01:00
committed by GitHub
parent f89b3b3e0f
commit 7ad19af93f
5 changed files with 268 additions and 229 deletions

View File

@ -131,19 +131,21 @@ class NotaryFlow {
*/ */
// See AbstractStateReplacementFlow.Acceptor for why it's Void? // See AbstractStateReplacementFlow.Acceptor for why it's Void?
abstract class Service(val otherSideSession: FlowSession, val service: TrustedAuthorityNotaryService) : FlowLogic<Void?>() { abstract class Service(val otherSideSession: FlowSession, val service: TrustedAuthorityNotaryService) : FlowLogic<Void?>() {
companion object {
// TODO: Determine an appropriate limit and also enforce in the network parameters and the transaction builder.
private const val maxAllowedInputs = 10_000
}
@Suspendable @Suspendable
override fun call(): Void? { override fun call(): Void? {
check(serviceHub.myInfo.legalIdentities.any { serviceHub.networkMapCache.isNotary(it) }) { check(serviceHub.myInfo.legalIdentities.any { serviceHub.networkMapCache.isNotary(it) }) {
"We are not a notary on the network" "We are not a notary on the network"
} }
val requestPayload = otherSideSession.receive<NotarisationPayload>().unwrap { it } val requestPayload = otherSideSession.receive<NotarisationPayload>().unwrap { it }
var txId: SecureHash? = null var txId: SecureHash? = null
try { try {
val parts = validateRequest(requestPayload) val parts = validateRequest(requestPayload)
txId = parts.id txId = parts.id
checkNotary(parts.notary)
service.validateTimeWindow(parts.timestamp) service.validateTimeWindow(parts.timestamp)
service.commitInputStates(parts.inputs, txId, otherSideSession.counterparty, requestPayload.requestSignature) service.commitInputStates(parts.inputs, txId, otherSideSession.counterparty, requestPayload.requestSignature)
signTransactionAndSendResponse(txId) signTransactionAndSendResponse(txId)
@ -153,6 +155,16 @@ class NotaryFlow {
return null return null
} }
/** Checks whether the number of input states is too large. */
protected fun checkInputs(inputs: List<StateRef>) {
if (inputs.size > maxAllowedInputs) {
val error = NotaryError.TransactionInvalid(
IllegalArgumentException("A transaction cannot have more than $maxAllowedInputs inputs, received: ${inputs.size}")
)
throw NotaryInternalException(error)
}
}
/** /**
* Implement custom logic to perform transaction verification based on validity and privacy requirements. * Implement custom logic to perform transaction verification based on validity and privacy requirements.
*/ */

View File

@ -22,9 +22,12 @@ class NonValidatingNotaryFlow(otherSideSession: FlowSession, service: TrustedAut
@Suspendable @Suspendable
override fun validateRequest(requestPayload: NotarisationPayload): TransactionParts { override fun validateRequest(requestPayload: NotarisationPayload): TransactionParts {
val transaction = requestPayload.coreTransaction val transaction = requestPayload.coreTransaction
checkInputs(transaction.inputs)
val request = NotarisationRequest(transaction.inputs, transaction.id) val request = NotarisationRequest(transaction.inputs, transaction.id)
validateRequestSignature(request, requestPayload.requestSignature) validateRequestSignature(request, requestPayload.requestSignature)
return extractParts(transaction) val parts = extractParts(transaction)
checkNotary(parts.notary)
return parts
} }
private fun extractParts(tx: CoreTransaction): TransactionParts { private fun extractParts(tx: CoreTransaction): TransactionParts {
@ -35,8 +38,7 @@ class NonValidatingNotaryFlow(otherSideSession: FlowSession, service: TrustedAut
checkAllComponentsVisible(ComponentGroupEnum.INPUTS_GROUP) checkAllComponentsVisible(ComponentGroupEnum.INPUTS_GROUP)
checkAllComponentsVisible(ComponentGroupEnum.TIMEWINDOW_GROUP) checkAllComponentsVisible(ComponentGroupEnum.TIMEWINDOW_GROUP)
} }
val notary = tx.notary TransactionParts(tx.id, tx.inputs, tx.timeWindow, tx.notary)
TransactionParts(tx.id, tx.inputs, tx.timeWindow, notary)
} }
is ContractUpgradeFilteredTransaction -> TransactionParts(tx.id, tx.inputs, null, tx.notary) is ContractUpgradeFilteredTransaction -> TransactionParts(tx.id, tx.inputs, null, tx.notary)
is NotaryChangeWireTransaction -> TransactionParts(tx.id, tx.inputs, null, tx.notary) is NotaryChangeWireTransaction -> TransactionParts(tx.id, tx.inputs, null, tx.notary)

View File

@ -27,6 +27,7 @@ class ValidatingNotaryFlow(otherSideSession: FlowSession, service: TrustedAuthor
override fun validateRequest(requestPayload: NotarisationPayload): TransactionParts { override fun validateRequest(requestPayload: NotarisationPayload): TransactionParts {
try { try {
val stx = requestPayload.signedTransaction val stx = requestPayload.signedTransaction
checkInputs(stx.inputs)
validateRequestSignature(NotarisationRequest(stx.inputs, stx.id), requestPayload.requestSignature) validateRequestSignature(NotarisationRequest(stx.inputs, stx.id), requestPayload.requestSignature)
val notary = stx.notary val notary = stx.notary
checkNotary(notary) checkNotary(notary)

View File

@ -1,38 +1,27 @@
package net.corda.node.services.transactions package net.corda.node.services.transactions
import net.corda.core.concurrent.CordaFuture import co.paralleluniverse.fibers.Suspendable
import net.corda.core.contracts.StateAndRef
import net.corda.core.contracts.StateRef import net.corda.core.contracts.StateRef
import net.corda.core.crypto.* import net.corda.core.crypto.*
import net.corda.core.flows.* import net.corda.core.flows.NotaryException
import net.corda.core.flows.NotaryFlow
import net.corda.core.identity.Party import net.corda.core.identity.Party
import net.corda.core.internal.generateSignature import net.corda.core.internal.NotaryChangeTransactionBuilder
import net.corda.core.messaging.MessageRecipients
import net.corda.core.node.ServiceHub import net.corda.core.node.ServiceHub
import net.corda.core.serialization.deserialize
import net.corda.core.serialization.serialize
import net.corda.core.transactions.SignedTransaction import net.corda.core.transactions.SignedTransaction
import net.corda.core.transactions.TransactionBuilder
import net.corda.core.utilities.OpaqueBytes
import net.corda.core.utilities.getOrThrow import net.corda.core.utilities.getOrThrow
import net.corda.core.utilities.seconds
import net.corda.node.internal.StartedNode import net.corda.node.internal.StartedNode
import net.corda.node.services.messaging.Message
import net.corda.node.services.statemachine.InitialSessionMessage
import net.corda.testing.contracts.DummyContract
import net.corda.testing.core.ALICE_NAME import net.corda.testing.core.ALICE_NAME
import net.corda.testing.core.dummyCommand import net.corda.testing.core.DUMMY_NOTARY_NAME
import net.corda.testing.core.singleIdentity import net.corda.testing.core.singleIdentity
import net.corda.testing.node.internal.* import net.corda.testing.node.MockNetworkNotarySpec
import org.assertj.core.api.Assertions.assertThat import net.corda.testing.node.internal.InternalMockNetwork
import net.corda.testing.node.internal.InternalMockNodeParameters
import net.corda.testing.node.internal.startFlow
import org.junit.After import org.junit.After
import org.junit.Before import org.junit.Before
import org.junit.Test import org.junit.Test
import java.time.Instant
import java.util.*
import kotlin.test.assertEquals
import kotlin.test.assertFailsWith import kotlin.test.assertFailsWith
import kotlin.test.assertTrue
class NotaryServiceTests { class NotaryServiceTests {
private lateinit var mockNet: InternalMockNetwork private lateinit var mockNet: InternalMockNetwork
@ -43,7 +32,10 @@ class NotaryServiceTests {
@Before @Before
fun setup() { fun setup() {
mockNet = InternalMockNetwork(cordappPackages = listOf("net.corda.testing.contracts")) mockNet = InternalMockNetwork(
cordappPackages = listOf("net.corda.testing.contracts"),
notarySpecs = listOf(MockNetworkNotarySpec(DUMMY_NOTARY_NAME, validating = false))
)
aliceNode = mockNet.createNode(InternalMockNodeParameters(legalName = ALICE_NAME)) aliceNode = mockNet.createNode(InternalMockNodeParameters(legalName = ALICE_NAME))
notaryServices = mockNet.defaultNotaryNode.services //TODO get rid of that notaryServices = mockNet.defaultNotaryNode.services //TODO get rid of that
notary = mockNet.defaultNotaryIdentity notary = mockNet.defaultNotaryIdentity
@ -56,201 +48,38 @@ class NotaryServiceTests {
} }
@Test @Test
fun `should sign a unique transaction with a valid time-window`() { fun `should reject a transaction with too many inputs`() {
val stx = run { notariseWithTooManyInputs(aliceNode, alice, notary, mockNet)
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
.setTimeWindow(Instant.now(), 30.seconds)
aliceNode.services.signInitialTransaction(tx)
}
val future = runNotaryClient(stx)
val signatures = future.getOrThrow()
signatures.forEach { it.verify(stx.id) }
} }
@Test internal companion object {
fun `should sign a unique transaction without a time-window`() { /** This is used by both [NotaryServiceTests] and [ValidatingNotaryServiceTests]. */
val stx = run { fun notariseWithTooManyInputs(node: StartedNode<InternalMockNetwork.MockNode>, party: Party, notary: Party, network: InternalMockNetwork) {
val inputState = issueState(aliceNode.services, alice) val stx = generateTransaction(node, party, notary)
val tx = TransactionBuilder(notary)
.addInputState(inputState) val future = node.services.startFlow(DummyClientFlow(stx, notary)).resultFuture
.addCommand(dummyCommand(alice.owningKey)) network.runNetwork()
aliceNode.services.signInitialTransaction(tx) assertFailsWith<NotaryException> { future.getOrThrow() }
} }
val future = runNotaryClient(stx) private fun generateTransaction(node: StartedNode<InternalMockNetwork.MockNode>, party: Party, notary: Party): SignedTransaction {
val signatures = future.getOrThrow() val inputs = (1..10_005).map { StateRef(SecureHash.randomSHA256(), 0) }
signatures.forEach { it.verify(stx.id) } val tx = NotaryChangeTransactionBuilder(inputs, notary, party).build()
}
@Test return node.services.run {
fun `should report error for transaction with an invalid time-window`() { val myKey = myInfo.legalIdentities.first().owningKey
val stx = run { val signableData = SignableData(tx.id, SignatureMetadata(myInfo.platformVersion, Crypto.findSignatureScheme(myKey).schemeNumberID))
val inputState = issueState(aliceNode.services, alice) val mySignature = keyManagementService.sign(signableData, myKey)
val tx = TransactionBuilder(notary) SignedTransaction(tx, listOf(mySignature))
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
.setTimeWindow(Instant.now().plusSeconds(3600), 30.seconds)
aliceNode.services.signInitialTransaction(tx)
}
val future = runNotaryClient(stx)
val ex = assertFailsWith(NotaryException::class) { future.getOrThrow() }
assertThat(ex.error).isInstanceOf(NotaryError.TimeWindowInvalid::class.java)
}
@Test
fun `should sign identical transaction multiple times (notarisation is idempotent)`() {
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
val firstAttempt = NotaryFlow.Client(stx)
val secondAttempt = NotaryFlow.Client(stx)
val f1 = aliceNode.services.startFlow(firstAttempt).resultFuture
val f2 = aliceNode.services.startFlow(secondAttempt).resultFuture
mockNet.runNetwork()
// Note that the notary will only return identical signatures when using deterministic signature
// schemes (e.g. EdDSA) and when deterministic metadata is attached (no timestamps or nonces).
// We only really care that both signatures are over the same transaction and by the same notary.
val sig1 = f1.getOrThrow().single()
assertEquals(sig1.by, notary.owningKey)
assertTrue(sig1.isValid(stx.id))
val sig2 = f2.getOrThrow().single()
assertEquals(sig2.by, notary.owningKey)
assertTrue(sig2.isValid(stx.id))
}
@Test
fun `should report conflict when inputs are reused across transactions`() {
val firstState = issueState(aliceNode.services, alice)
val secondState = issueState(aliceNode.services, alice)
fun spendState(state: StateAndRef<*>): SignedTransaction {
val stx = run {
val tx = TransactionBuilder(notary)
.addInputState(state)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
} }
aliceNode.services.startFlow(NotaryFlow.Client(stx))
mockNet.runNetwork()
return stx
} }
val firstSpendTx = spendState(firstState) private class DummyClientFlow(stx: SignedTransaction, val notary: Party) : NotaryFlow.Client(stx) {
val secondSpendTx = spendState(secondState) @Suspendable
override fun call(): List<TransactionSignature> {
val doubleSpendTx = run { notarise(notary)
val tx = TransactionBuilder(notary) throw UnsupportedOperationException()
.addInputState(issueState(aliceNode.services, alice))
.addInputState(firstState)
.addInputState(secondState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
val doubleSpend = NotaryFlow.Client(doubleSpendTx) // Double spend the inputState in a second transaction.
val future = aliceNode.services.startFlow(doubleSpend)
mockNet.runNetwork()
val ex = assertFailsWith(NotaryException::class) { future.resultFuture.getOrThrow() }
val notaryError = ex.error as NotaryError.Conflict
assertEquals(notaryError.txId, doubleSpendTx.id)
with(notaryError) {
assertEquals(consumedStates.size, 2)
assertEquals(consumedStates[firstState.ref]!!.hashOfTransactionId, firstSpendTx.id.sha256())
assertEquals(consumedStates[secondState.ref]!!.hashOfTransactionId, secondSpendTx.id.sha256())
}
}
@Test
fun `should reject when notarisation request not signed by the requesting party`() {
runNotarisationAndInterceptClientPayload { originalPayload ->
val transaction = originalPayload.signedTransaction
val randomKeyPair = Crypto.generateKeyPair()
val bytesToSign = NotarisationRequest(transaction.inputs, transaction.id).serialize().bytes
val modifiedSignature = NotarisationRequestSignature(randomKeyPair.sign(bytesToSign), aliceNode.services.myInfo.platformVersion)
originalPayload.copy(requestSignature = modifiedSignature)
}
}
@Test
fun `should reject when incorrect notarisation request signed - inputs don't match`() {
runNotarisationAndInterceptClientPayload { originalPayload ->
val transaction = originalPayload.signedTransaction
val wrongInputs = listOf(StateRef(SecureHash.randomSHA256(), 0))
val request = NotarisationRequest(wrongInputs, transaction.id)
val modifiedSignature = request.generateSignature(aliceNode.services)
originalPayload.copy(requestSignature = modifiedSignature)
}
}
@Test
fun `should reject when incorrect notarisation request signed - transaction id doesn't match`() {
runNotarisationAndInterceptClientPayload { originalPayload ->
val transaction = originalPayload.signedTransaction
val wrongTransactionId = SecureHash.randomSHA256()
val request = NotarisationRequest(transaction.inputs, wrongTransactionId)
val modifiedSignature = request.generateSignature(aliceNode.services)
originalPayload.copy(requestSignature = modifiedSignature)
}
}
private fun runNotarisationAndInterceptClientPayload(payloadModifier: (NotarisationPayload) -> NotarisationPayload) {
aliceNode.setMessagingServiceSpy(object : MessagingServiceSpy(aliceNode.network) {
override fun send(message: Message, target: MessageRecipients, retryId: Long?, sequenceKey: Any) {
val messageData = message.data.deserialize<Any>() as? InitialSessionMessage
val payload = messageData?.firstPayload!!.deserialize()
if (payload is NotarisationPayload) {
val alteredPayload = payloadModifier(payload)
val alteredMessageData = messageData.copy(firstPayload = alteredPayload.serialize())
val alteredMessage = InMemoryMessage(message.topic, OpaqueBytes(alteredMessageData.serialize().bytes), message.uniqueMessageId)
messagingService.send(alteredMessage, target, retryId)
} else {
messagingService.send(message, target, retryId)
}
} }
})
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
} }
val future = runNotaryClient(stx)
val ex = assertFailsWith(NotaryException::class) { future.getOrThrow() }
assertThat(ex.error).isInstanceOf(NotaryError.RequestSignatureInvalid::class.java)
}
private fun runNotaryClient(stx: SignedTransaction): CordaFuture<List<TransactionSignature>> {
val flow = NotaryFlow.Client(stx)
val future = aliceNode.services.startFlow(flow).resultFuture
mockNet.runNetwork()
return future
}
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)
services.recordTransactions(stx)
return StateAndRef(tx.outputStates().first(), StateRef(stx.id, 0))
} }
} }

View File

@ -4,43 +4,49 @@ import net.corda.core.concurrent.CordaFuture
import net.corda.core.contracts.Command import net.corda.core.contracts.Command
import net.corda.core.contracts.StateAndRef import net.corda.core.contracts.StateAndRef
import net.corda.core.contracts.StateRef import net.corda.core.contracts.StateRef
import net.corda.core.crypto.TransactionSignature import net.corda.core.crypto.*
import net.corda.core.crypto.generateKeyPair import net.corda.core.flows.*
import net.corda.core.flows.NotaryError
import net.corda.core.flows.NotaryException
import net.corda.core.flows.NotaryFlow
import net.corda.core.identity.Party import net.corda.core.identity.Party
import net.corda.core.internal.generateSignature
import net.corda.core.messaging.MessageRecipients
import net.corda.core.node.ServiceHub import net.corda.core.node.ServiceHub
import net.corda.core.serialization.deserialize
import net.corda.core.serialization.serialize
import net.corda.core.transactions.SignedTransaction import net.corda.core.transactions.SignedTransaction
import net.corda.core.transactions.TransactionBuilder import net.corda.core.transactions.TransactionBuilder
import net.corda.core.utilities.OpaqueBytes
import net.corda.core.utilities.getOrThrow import net.corda.core.utilities.getOrThrow
import net.corda.core.utilities.seconds
import net.corda.node.internal.StartedNode
import net.corda.node.services.issueInvalidState import net.corda.node.services.issueInvalidState
import net.corda.node.services.messaging.Message
import net.corda.node.services.statemachine.InitialSessionMessage
import net.corda.testing.contracts.DummyContract import net.corda.testing.contracts.DummyContract
import net.corda.testing.core.ALICE_NAME import net.corda.testing.core.ALICE_NAME
import net.corda.testing.core.dummyCommand import net.corda.testing.core.dummyCommand
import net.corda.testing.core.singleIdentity import net.corda.testing.core.singleIdentity
import net.corda.testing.node.MockNetwork import net.corda.testing.node.internal.*
import net.corda.testing.node.MockNodeParameters
import net.corda.testing.node.StartedMockNode
import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.Assertions.assertThat
import org.junit.After import org.junit.After
import org.junit.Before import org.junit.Before
import org.junit.Test import org.junit.Test
import java.time.Instant
import java.util.* import java.util.*
import kotlin.test.assertEquals import kotlin.test.assertEquals
import kotlin.test.assertFailsWith import kotlin.test.assertFailsWith
import kotlin.test.assertTrue
class ValidatingNotaryServiceTests { class ValidatingNotaryServiceTests {
private lateinit var mockNet: MockNetwork private lateinit var mockNet: InternalMockNetwork
private lateinit var notaryNode: StartedMockNode private lateinit var notaryNode: StartedNode<InternalMockNetwork.MockNode>
private lateinit var aliceNode: StartedMockNode private lateinit var aliceNode: StartedNode<InternalMockNetwork.MockNode>
private lateinit var notary: Party private lateinit var notary: Party
private lateinit var alice: Party private lateinit var alice: Party
@Before @Before
fun setup() { fun setup() {
mockNet = MockNetwork(cordappPackages = listOf("net.corda.testing.contracts")) mockNet = InternalMockNetwork(cordappPackages = listOf("net.corda.testing.contracts"))
aliceNode = mockNet.createNode(MockNodeParameters(legalName = ALICE_NAME)) aliceNode = mockNet.createNode(InternalMockNodeParameters(legalName = ALICE_NAME))
notaryNode = mockNet.defaultNotaryNode notaryNode = mockNet.defaultNotaryNode
notary = mockNet.defaultNotaryIdentity notary = mockNet.defaultNotaryIdentity
alice = aliceNode.info.singleIdentity() alice = aliceNode.info.singleIdentity()
@ -61,7 +67,7 @@ class ValidatingNotaryServiceTests {
aliceNode.services.signInitialTransaction(tx) aliceNode.services.signInitialTransaction(tx)
} }
val future = runClient(stx) val future = runNotaryClient(stx)
val ex = assertFailsWith(NotaryException::class) { future.getOrThrow() } val ex = assertFailsWith(NotaryException::class) { future.getOrThrow() }
val notaryError = ex.error as NotaryError.TransactionInvalid val notaryError = ex.error as NotaryError.TransactionInvalid
@ -82,16 +88,205 @@ class ValidatingNotaryServiceTests {
// Expecting SignaturesMissingException instead of NotaryException, since the exception should originate from // Expecting SignaturesMissingException instead of NotaryException, since the exception should originate from
// the client flow. // the client flow.
val ex = assertFailsWith<SignedTransaction.SignaturesMissingException> { val ex = assertFailsWith<SignedTransaction.SignaturesMissingException> {
val future = runClient(stx) val future = runNotaryClient(stx)
future.getOrThrow() future.getOrThrow()
} }
val missingKeys = ex.missing val missingKeys = ex.missing
assertEquals(setOf(expectedMissingKey), missingKeys) assertEquals(setOf(expectedMissingKey), missingKeys)
} }
private fun runClient(stx: SignedTransaction): CordaFuture<List<TransactionSignature>> { @Test
fun `should sign a unique transaction with a valid time-window`() {
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
.setTimeWindow(Instant.now(), 30.seconds)
aliceNode.services.signInitialTransaction(tx)
}
val future = runNotaryClient(stx)
val signatures = future.getOrThrow()
signatures.forEach { it.verify(stx.id) }
}
@Test
fun `should sign a unique transaction without a time-window`() {
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
val future = runNotaryClient(stx)
val signatures = future.getOrThrow()
signatures.forEach { it.verify(stx.id) }
}
@Test
fun `should report error for transaction with an invalid time-window`() {
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
.setTimeWindow(Instant.now().plusSeconds(3600), 30.seconds)
aliceNode.services.signInitialTransaction(tx)
}
val future = runNotaryClient(stx)
val ex = assertFailsWith(NotaryException::class) { future.getOrThrow() }
assertThat(ex.error).isInstanceOf(NotaryError.TimeWindowInvalid::class.java)
}
@Test
fun `should sign identical transaction multiple times (notarisation is idempotent)`() {
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
val firstAttempt = NotaryFlow.Client(stx)
val secondAttempt = NotaryFlow.Client(stx)
val f1 = aliceNode.services.startFlow(firstAttempt).resultFuture
val f2 = aliceNode.services.startFlow(secondAttempt).resultFuture
mockNet.runNetwork()
// Note that the notary will only return identical signatures when using deterministic signature
// schemes (e.g. EdDSA) and when deterministic metadata is attached (no timestamps or nonces).
// We only really care that both signatures are over the same transaction and by the same notary.
val sig1 = f1.getOrThrow().single()
assertEquals(sig1.by, notary.owningKey)
assertTrue(sig1.isValid(stx.id))
val sig2 = f2.getOrThrow().single()
assertEquals(sig2.by, notary.owningKey)
assertTrue(sig2.isValid(stx.id))
}
@Test
fun `should report conflict when inputs are reused across transactions`() {
val firstState = issueState(aliceNode.services, alice)
val secondState = issueState(aliceNode.services, alice)
fun spendState(state: StateAndRef<*>): SignedTransaction {
val stx = run {
val tx = TransactionBuilder(notary)
.addInputState(state)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
aliceNode.services.startFlow(NotaryFlow.Client(stx))
mockNet.runNetwork()
return stx
}
val firstSpendTx = spendState(firstState)
val secondSpendTx = spendState(secondState)
val doubleSpendTx = run {
val tx = TransactionBuilder(notary)
.addInputState(issueState(aliceNode.services, alice))
.addInputState(firstState)
.addInputState(secondState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
val doubleSpend = NotaryFlow.Client(doubleSpendTx) // Double spend the inputState in a second transaction.
val future = aliceNode.services.startFlow(doubleSpend)
mockNet.runNetwork()
val ex = assertFailsWith(NotaryException::class) { future.resultFuture.getOrThrow() }
val notaryError = ex.error as NotaryError.Conflict
assertEquals(notaryError.txId, doubleSpendTx.id)
with(notaryError) {
assertEquals(consumedStates.size, 2)
assertEquals(consumedStates[firstState.ref]!!.hashOfTransactionId, firstSpendTx.id.sha256())
assertEquals(consumedStates[secondState.ref]!!.hashOfTransactionId, secondSpendTx.id.sha256())
}
}
@Test
fun `should reject when notarisation request not signed by the requesting party`() {
runNotarisationAndInterceptClientPayload { originalPayload ->
val transaction = originalPayload.signedTransaction
val randomKeyPair = Crypto.generateKeyPair()
val bytesToSign = NotarisationRequest(transaction.inputs, transaction.id).serialize().bytes
val modifiedSignature = NotarisationRequestSignature(randomKeyPair.sign(bytesToSign), aliceNode.services.myInfo.platformVersion)
originalPayload.copy(requestSignature = modifiedSignature)
}
}
@Test
fun `should reject when incorrect notarisation request signed - inputs don't match`() {
runNotarisationAndInterceptClientPayload { originalPayload ->
val transaction = originalPayload.signedTransaction
val wrongInputs = listOf(StateRef(SecureHash.randomSHA256(), 0))
val request = NotarisationRequest(wrongInputs, transaction.id)
val modifiedSignature = request.generateSignature(aliceNode.services)
originalPayload.copy(requestSignature = modifiedSignature)
}
}
@Test
fun `should reject when incorrect notarisation request signed - transaction id doesn't match`() {
runNotarisationAndInterceptClientPayload { originalPayload ->
val transaction = originalPayload.signedTransaction
val wrongTransactionId = SecureHash.randomSHA256()
val request = NotarisationRequest(transaction.inputs, wrongTransactionId)
val modifiedSignature = request.generateSignature(aliceNode.services)
originalPayload.copy(requestSignature = modifiedSignature)
}
}
@Test
fun `should reject a transaction with too many inputs`() {
NotaryServiceTests.notariseWithTooManyInputs(aliceNode, alice, notary, mockNet)
}
private fun runNotarisationAndInterceptClientPayload(payloadModifier: (NotarisationPayload) -> NotarisationPayload) {
aliceNode.setMessagingServiceSpy(object : MessagingServiceSpy(aliceNode.network) {
override fun send(message: Message, target: MessageRecipients, retryId: Long?, sequenceKey: Any) {
val messageData = message.data.deserialize<Any>() as? InitialSessionMessage
val payload = messageData?.firstPayload!!.deserialize()
if (payload is NotarisationPayload) {
val alteredPayload = payloadModifier(payload)
val alteredMessageData = messageData.copy(firstPayload = alteredPayload.serialize())
val alteredMessage = InMemoryMessage(message.topic, OpaqueBytes(alteredMessageData.serialize().bytes), message.uniqueMessageId)
messagingService.send(alteredMessage, target, retryId)
} else {
messagingService.send(message, target, retryId)
}
}
})
val stx = run {
val inputState = issueState(aliceNode.services, alice)
val tx = TransactionBuilder(notary)
.addInputState(inputState)
.addCommand(dummyCommand(alice.owningKey))
aliceNode.services.signInitialTransaction(tx)
}
val future = runNotaryClient(stx)
val ex = assertFailsWith(NotaryException::class) { future.getOrThrow() }
assertThat(ex.error).isInstanceOf(NotaryError.RequestSignatureInvalid::class.java)
}
private fun runNotaryClient(stx: SignedTransaction): CordaFuture<List<TransactionSignature>> {
val flow = NotaryFlow.Client(stx) val flow = NotaryFlow.Client(stx)
val future = aliceNode.startFlow(flow) val future = aliceNode.services.startFlow(flow).resultFuture
mockNet.runNetwork() mockNet.runNetwork()
return future return future
} }