Added checks against the use of the special uploader tokens (#3724)

"app", "rpc", "p2p" and "unknown" have security implications (see isUploaderTrusted method) and thus they are not allowed to be used in the uploader field when importing attachments via the public API.
This commit is contained in:
Shams Asari 2018-07-31 17:01:39 +01:00 committed by GitHub
parent 994fe0dbdc
commit dfafdbcb9f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 148 additions and 67 deletions

View File

@ -3,7 +3,7 @@ package net.corda.deterministic.common
import net.corda.core.contracts.Attachment import net.corda.core.contracts.Attachment
import net.corda.core.contracts.ContractAttachment import net.corda.core.contracts.ContractAttachment
import net.corda.core.contracts.ContractClassName import net.corda.core.contracts.ContractClassName
import net.corda.core.internal.TEST_UPLOADER import net.corda.core.internal.DEPLOYED_CORDAPP_UPLOADER
import net.corda.core.serialization.CordaSerializable import net.corda.core.serialization.CordaSerializable
import net.corda.core.serialization.SerializedBytes import net.corda.core.serialization.SerializedBytes
import net.corda.core.serialization.deserialize import net.corda.core.serialization.deserialize
@ -18,8 +18,9 @@ class TransactionVerificationRequest(val wtxToVerify: SerializedBytes<WireTransa
fun toLedgerTransaction(): LedgerTransaction { fun toLedgerTransaction(): LedgerTransaction {
val deps = dependencies.map { it.deserialize() }.associateBy(WireTransaction::id) val deps = dependencies.map { it.deserialize() }.associateBy(WireTransaction::id)
val attachments = attachments.map { it.deserialize<Attachment>() } val attachments = attachments.map { it.deserialize<Attachment>() }
val attachmentMap = attachments.mapNotNull { it as? MockContractAttachment } val attachmentMap = attachments
.associateBy(Attachment::id) { ContractAttachment(it, it.contract, uploader=TEST_UPLOADER) } .mapNotNull { it as? MockContractAttachment }
.associateBy(Attachment::id) { ContractAttachment(it, it.contract, uploader = DEPLOYED_CORDAPP_UPLOADER) }
val contractAttachmentMap = emptyMap<ContractClassName, ContractAttachment>() val contractAttachmentMap = emptyMap<ContractClassName, ContractAttachment>()
@Suppress("DEPRECATION") @Suppress("DEPRECATION")
return wtxToVerify.deserialize().toLedgerTransaction( return wtxToVerify.deserialize().toLedgerTransaction(

View File

@ -16,15 +16,14 @@ import java.security.CodeSigner
import java.security.cert.X509Certificate import java.security.cert.X509Certificate
import java.util.jar.JarInputStream import java.util.jar.JarInputStream
// Possible attachment uploaders
const val DEPLOYED_CORDAPP_UPLOADER = "app" const val DEPLOYED_CORDAPP_UPLOADER = "app"
const val RPC_UPLOADER = "rpc" const val RPC_UPLOADER = "rpc"
const val TEST_UPLOADER = "test"
const val P2P_UPLOADER = "p2p" const val P2P_UPLOADER = "p2p"
const val UNKNOWN_UPLOADER = "unknown" const val UNKNOWN_UPLOADER = "unknown"
fun isUploaderTrusted(uploader: String?) = private val TRUSTED_UPLOADERS = listOf(DEPLOYED_CORDAPP_UPLOADER, RPC_UPLOADER)
uploader?.let { it in listOf(DEPLOYED_CORDAPP_UPLOADER, RPC_UPLOADER, TEST_UPLOADER) } ?: false
fun isUploaderTrusted(uploader: String?): Boolean = uploader in TRUSTED_UPLOADERS
@KeepForDJVM @KeepForDJVM
abstract class AbstractAttachment(dataLoader: () -> ByteArray) : Attachment { abstract class AbstractAttachment(dataLoader: () -> ByteArray) : Attachment {

View File

@ -8,6 +8,9 @@ Unreleased
---------- ----------
* Added ``registerResponderFlow`` method to ``StartedMockNode``, to support isolated testing of responder flow behaviour. * Added ``registerResponderFlow`` method to ``StartedMockNode``, to support isolated testing of responder flow behaviour.
* "app", "rpc", "p2p" and "unknown" are no longer allowed as uploader values when importing attachments. These are used
internally in security sensitive code.
* Introduced ``TestCorDapp`` and utilities to support asymmetric setups for nodes through ``DriverDSL``, ``MockNetwork`` and ``MockServices``. * Introduced ``TestCorDapp`` and utilities to support asymmetric setups for nodes through ``DriverDSL``, ``MockNetwork`` and ``MockServices``.
* Change type of the `checkpoint_value` column. Please check the upgrade-notes on how to update your database. * Change type of the `checkpoint_value` column. Please check the upgrade-notes on how to update your database.

View File

@ -894,7 +894,7 @@ abstract class AbstractNode<S>(val configuration: NodeConfiguration,
override val transactionVerifierService: TransactionVerifierService get() = this@AbstractNode.transactionVerifierService override val transactionVerifierService: TransactionVerifierService get() = this@AbstractNode.transactionVerifierService
override val contractUpgradeService: ContractUpgradeService get() = this@AbstractNode.contractUpgradeService override val contractUpgradeService: ContractUpgradeService get() = this@AbstractNode.contractUpgradeService
override val auditService: AuditService get() = this@AbstractNode.auditService override val auditService: AuditService get() = this@AbstractNode.auditService
override val attachments: AttachmentStorage get() = this@AbstractNode.attachments override val attachments: AttachmentStorageInternal get() = this@AbstractNode.attachments
override val networkService: MessagingService get() = network override val networkService: MessagingService get() = network
override val clock: Clock get() = platformClock override val clock: Clock get() = platformClock
override val configuration: NodeConfiguration get() = this@AbstractNode.configuration override val configuration: NodeConfiguration get() = this@AbstractNode.configuration

View File

@ -18,26 +18,12 @@ import net.corda.core.internal.FlowStateMachine
import net.corda.core.internal.RPC_UPLOADER import net.corda.core.internal.RPC_UPLOADER
import net.corda.core.internal.STRUCTURAL_STEP_PREFIX import net.corda.core.internal.STRUCTURAL_STEP_PREFIX
import net.corda.core.internal.sign import net.corda.core.internal.sign
import net.corda.core.messaging.CordaRPCOps import net.corda.core.messaging.*
import net.corda.core.messaging.DataFeed
import net.corda.core.messaging.FlowHandle
import net.corda.core.messaging.FlowHandleImpl
import net.corda.core.messaging.FlowProgressHandle
import net.corda.core.messaging.FlowProgressHandleImpl
import net.corda.core.messaging.ParametersUpdateInfo
import net.corda.core.messaging.RPCReturnsObservables
import net.corda.core.messaging.StateMachineInfo
import net.corda.core.messaging.StateMachineTransactionMapping
import net.corda.core.messaging.StateMachineUpdate
import net.corda.core.node.NodeInfo import net.corda.core.node.NodeInfo
import net.corda.core.node.services.AttachmentId import net.corda.core.node.services.AttachmentId
import net.corda.core.node.services.NetworkMapCache import net.corda.core.node.services.NetworkMapCache
import net.corda.core.node.services.Vault import net.corda.core.node.services.Vault
import net.corda.core.node.services.vault.AttachmentQueryCriteria import net.corda.core.node.services.vault.*
import net.corda.core.node.services.vault.AttachmentSort
import net.corda.core.node.services.vault.PageSpecification
import net.corda.core.node.services.vault.QueryCriteria
import net.corda.core.node.services.vault.Sort
import net.corda.core.serialization.serialize import net.corda.core.serialization.serialize
import net.corda.core.transactions.SignedTransaction import net.corda.core.transactions.SignedTransaction
import net.corda.core.utilities.getOrThrow import net.corda.core.utilities.getOrThrow
@ -74,11 +60,10 @@ internal class CordaRPCOpsImpl(
} }
override fun acceptNewNetworkParameters(parametersHash: SecureHash) { override fun acceptNewNetworkParameters(parametersHash: SecureHash) {
services.networkMapUpdater.acceptNewNetworkParameters( // TODO When multiple identities design will be better specified this should be signature from node operator.
parametersHash, services.networkMapUpdater.acceptNewNetworkParameters(parametersHash) { hash ->
// TODO When multiple identities design will be better specified this should be signature from node operator. hash.serialize().sign { services.keyManagementService.sign(it.bytes, services.myInfo.legalIdentities[0].owningKey) }
{ hash -> hash.serialize().sign { services.keyManagementService.sign(it.bytes, services.myInfo.legalIdentities[0].owningKey) } } }
)
} }
override fun networkMapFeed(): DataFeed<List<NodeInfo>, NetworkMapCache.MapChange> { override fun networkMapFeed(): DataFeed<List<NodeInfo>, NetworkMapCache.MapChange> {
@ -191,7 +176,7 @@ internal class CordaRPCOpsImpl(
} }
override fun uploadAttachment(jar: InputStream): SecureHash { override fun uploadAttachment(jar: InputStream): SecureHash {
return services.attachments.importAttachment(jar, RPC_UPLOADER, null) return services.attachments.privilegedImportAttachment(jar, RPC_UPLOADER, null)
} }
override fun uploadAttachmentWithMetadata(jar: InputStream, uploader: String, filename: String): SecureHash { override fun uploadAttachmentWithMetadata(jar: InputStream, uploader: String, filename: String): SecureHash {

View File

@ -14,6 +14,7 @@ import net.corda.core.node.services.AttachmentStorage
import net.corda.core.serialization.SingletonSerializeAsToken import net.corda.core.serialization.SingletonSerializeAsToken
import net.corda.core.utilities.contextLogger import net.corda.core.utilities.contextLogger
import net.corda.node.cordapp.CordappLoader import net.corda.node.cordapp.CordappLoader
import net.corda.node.services.persistence.AttachmentStorageInternal
import java.net.URL import java.net.URL
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
@ -89,7 +90,13 @@ open class CordappProviderImpl(private val cordappLoader: CordappLoader,
cordapps.filter { !it.contractClassNames.isEmpty() }.map { cordapps.filter { !it.contractClassNames.isEmpty() }.map {
it.jarPath.openStream().use { stream -> it.jarPath.openStream().use { stream ->
try { try {
attachmentStorage.importAttachment(stream, DEPLOYED_CORDAPP_UPLOADER, null) // We can't make attachmentStorage a AttachmentStorageInternal as that ends up requiring
// MockAttachmentStorage to implement it.
if (attachmentStorage is AttachmentStorageInternal) {
attachmentStorage.privilegedImportAttachment(stream, DEPLOYED_CORDAPP_UPLOADER, null)
} else {
attachmentStorage.importAttachment(stream, DEPLOYED_CORDAPP_UPLOADER, null)
}
} catch (faee: java.nio.file.FileAlreadyExistsException) { } catch (faee: java.nio.file.FileAlreadyExistsException) {
AttachmentId.parse(faee.message!!) AttachmentId.parse(faee.message!!)
} }

View File

@ -21,6 +21,7 @@ import net.corda.node.internal.cordapp.CordappProviderInternal
import net.corda.node.services.config.NodeConfiguration import net.corda.node.services.config.NodeConfiguration
import net.corda.node.services.messaging.MessagingService import net.corda.node.services.messaging.MessagingService
import net.corda.node.services.network.NetworkMapUpdater import net.corda.node.services.network.NetworkMapUpdater
import net.corda.node.services.persistence.AttachmentStorageInternal
import net.corda.node.services.statemachine.ExternalEvent import net.corda.node.services.statemachine.ExternalEvent
import net.corda.node.services.statemachine.FlowStateMachineImpl import net.corda.node.services.statemachine.FlowStateMachineImpl
import net.corda.nodeapi.internal.persistence.CordaPersistence import net.corda.nodeapi.internal.persistence.CordaPersistence
@ -105,6 +106,7 @@ interface ServiceHubInternal : ServiceHub {
} }
} }
override val attachments: AttachmentStorageInternal
override val vaultService: VaultServiceInternal override val vaultService: VaultServiceInternal
/** /**
* A map of hash->tx where tx has been signature/contract validated and the states are known to be correct. * A map of hash->tx where tx has been signature/contract validated and the states are known to be correct.

View File

@ -0,0 +1,13 @@
package net.corda.node.services.persistence
import net.corda.core.node.services.AttachmentId
import net.corda.core.node.services.AttachmentStorage
import java.io.InputStream
interface AttachmentStorageInternal : AttachmentStorage {
/**
* This is the same as [importAttachment] expect there are no checks done on the uploader field. This API is internal
* and is only for the node.
*/
fun privilegedImportAttachment(jar: InputStream, uploader: String, filename: String?): AttachmentId
}

View File

@ -12,12 +12,8 @@ import net.corda.core.contracts.ContractAttachment
import net.corda.core.contracts.ContractClassName import net.corda.core.contracts.ContractClassName
import net.corda.core.crypto.SecureHash import net.corda.core.crypto.SecureHash
import net.corda.core.crypto.sha256 import net.corda.core.crypto.sha256
import net.corda.core.internal.AbstractAttachment import net.corda.core.internal.*
import net.corda.core.internal.UNKNOWN_UPLOADER
import net.corda.core.internal.VisibleForTesting
import net.corda.core.internal.readFully
import net.corda.core.node.services.AttachmentId import net.corda.core.node.services.AttachmentId
import net.corda.core.node.services.AttachmentStorage
import net.corda.core.node.services.vault.AttachmentQueryCriteria import net.corda.core.node.services.vault.AttachmentQueryCriteria
import net.corda.core.node.services.vault.AttachmentSort import net.corda.core.node.services.vault.AttachmentSort
import net.corda.core.serialization.* import net.corda.core.serialization.*
@ -50,10 +46,12 @@ class NodeAttachmentService(
private val database: CordaPersistence, private val database: CordaPersistence,
attachmentContentCacheSize: Long = NodeConfiguration.defaultAttachmentContentCacheSize, attachmentContentCacheSize: Long = NodeConfiguration.defaultAttachmentContentCacheSize,
attachmentCacheBound: Long = NodeConfiguration.defaultAttachmentCacheBound attachmentCacheBound: Long = NodeConfiguration.defaultAttachmentCacheBound
) : AttachmentStorage, SingletonSerializeAsToken() { ) : AttachmentStorageInternal, SingletonSerializeAsToken() {
companion object { companion object {
private val log = contextLogger() private val log = contextLogger()
private val PRIVILEGED_UPLOADERS = listOf(DEPLOYED_CORDAPP_UPLOADER, RPC_UPLOADER, P2P_UPLOADER, UNKNOWN_UPLOADER)
// Just iterate over the entries with verification enabled: should be good enough to catch mistakes. // Just iterate over the entries with verification enabled: should be good enough to catch mistakes.
// Note that JarInputStream won't throw any kind of error at all if the file stream is in fact not // Note that JarInputStream won't throw any kind of error at all if the file stream is in fact not
// a ZIP! It'll just pretend it's an empty archive, which is kind of stupid but that's how it works. // a ZIP! It'll just pretend it's an empty archive, which is kind of stupid but that's how it works.
@ -228,10 +226,9 @@ class NodeAttachmentService(
} }
} }
private val attachmentCache = NonInvalidatingCache<SecureHash, Optional<Attachment>>( private val attachmentCache = NonInvalidatingCache<SecureHash, Optional<Attachment>>(attachmentCacheBound) { key ->
attachmentCacheBound, Optional.ofNullable(createAttachment(key))
{ key -> Optional.ofNullable(createAttachment(key)) } }
)
private fun createAttachment(key: SecureHash): Attachment? { private fun createAttachment(key: SecureHash): Attachment? {
val content = attachmentContentCache.get(key)!! val content = attachmentContentCache.get(key)!!
@ -258,6 +255,18 @@ class NodeAttachmentService(
} }
override fun importAttachment(jar: InputStream, uploader: String, filename: String?): AttachmentId { override fun importAttachment(jar: InputStream, uploader: String, filename: String?): AttachmentId {
require(uploader !in PRIVILEGED_UPLOADERS) { "$uploader is a reserved uploader token" }
if (uploader.startsWith("$P2P_UPLOADER:")) {
// FetchAttachmentsFlow is in core and thus doesn't have access to AttachmentStorageInternal to call
// privilegedImportAttachment
require(Thread.currentThread().stackTrace.any { it.className == FetchAttachmentsFlow::class.java.name }) {
"$P2P_UPLOADER is a reserved uploader token prefix"
}
}
return import(jar, uploader, filename)
}
override fun privilegedImportAttachment(jar: InputStream, uploader: String, filename: String?): AttachmentId {
return import(jar, uploader, filename) return import(jar, uploader, filename)
} }
@ -282,7 +291,13 @@ class NodeAttachmentService(
if (!hasAttachment(id)) { if (!hasAttachment(id)) {
checkIsAValidJAR(bytes.inputStream()) checkIsAValidJAR(bytes.inputStream())
val session = currentDBSession() val session = currentDBSession()
val attachment = NodeAttachmentService.DBAttachment(attId = id.toString(), content = bytes, uploader = uploader, filename = filename, contractClassNames = contractClassNames) val attachment = NodeAttachmentService.DBAttachment(
attId = id.toString(),
content = bytes,
uploader = uploader,
filename = filename,
contractClassNames = contractClassNames
)
session.save(attachment) session.save(attachment)
attachmentCount.inc() attachmentCount.inc()
log.info("Stored new attachment $id") log.info("Stored new attachment $id")
@ -295,10 +310,12 @@ class NodeAttachmentService(
} }
@Suppress("OverridingDeprecatedMember") @Suppress("OverridingDeprecatedMember")
override fun importOrGetAttachment(jar: InputStream): AttachmentId = try { override fun importOrGetAttachment(jar: InputStream): AttachmentId {
import(jar, UNKNOWN_UPLOADER, null) return try {
} catch (faee: java.nio.file.FileAlreadyExistsException) { import(jar, UNKNOWN_UPLOADER, null)
AttachmentId.parse(faee.message!!) } catch (faee: java.nio.file.FileAlreadyExistsException) {
AttachmentId.parse(faee.message!!)
}
} }
override fun queryAttachments(criteria: AttachmentQueryCriteria, sorting: AttachmentSort?): List<AttachmentId> { override fun queryAttachments(criteria: AttachmentQueryCriteria, sorting: AttachmentSort?): List<AttachmentId> {

View File

@ -1,25 +1,33 @@
package net.corda.node.services.persistence package net.corda.node.services.persistence
import co.paralleluniverse.fibers.Suspendable
import com.codahale.metrics.MetricRegistry import com.codahale.metrics.MetricRegistry
import com.google.common.jimfs.Configuration import com.google.common.jimfs.Configuration
import com.google.common.jimfs.Jimfs import com.google.common.jimfs.Jimfs
import net.corda.core.crypto.SecureHash import net.corda.core.crypto.SecureHash
import net.corda.core.crypto.sha256 import net.corda.core.crypto.sha256
import net.corda.core.flows.FlowLogic
import net.corda.core.internal.* import net.corda.core.internal.*
import net.corda.core.node.services.vault.AttachmentQueryCriteria import net.corda.core.node.services.vault.AttachmentQueryCriteria
import net.corda.core.node.services.vault.AttachmentSort import net.corda.core.node.services.vault.AttachmentSort
import net.corda.core.node.services.vault.Builder import net.corda.core.node.services.vault.Builder
import net.corda.core.node.services.vault.Sort import net.corda.core.node.services.vault.Sort
import net.corda.core.utilities.getOrThrow
import net.corda.node.internal.configureDatabase import net.corda.node.internal.configureDatabase
import net.corda.node.services.transactions.PersistentUniquenessProvider import net.corda.node.services.transactions.PersistentUniquenessProvider
import net.corda.nodeapi.internal.persistence.CordaPersistence import net.corda.nodeapi.internal.persistence.CordaPersistence
import net.corda.nodeapi.internal.persistence.DatabaseConfig import net.corda.nodeapi.internal.persistence.DatabaseConfig
import net.corda.testing.internal.LogHelper import net.corda.testing.internal.LogHelper
import net.corda.testing.node.MockServices.Companion.makeTestDataSourceProperties import net.corda.testing.node.MockServices.Companion.makeTestDataSourceProperties
import net.corda.testing.node.internal.InternalMockNetwork
import net.corda.testing.node.internal.startFlow
import org.assertj.core.api.Assertions.assertThatIllegalArgumentException
import org.junit.After import org.junit.After
import org.junit.Before import org.junit.Before
import org.junit.Ignore import org.junit.Ignore
import org.junit.Test import org.junit.Test
import java.io.ByteArrayOutputStream
import java.io.OutputStream
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import java.nio.file.FileAlreadyExistsException import java.nio.file.FileAlreadyExistsException
import java.nio.file.FileSystem import java.nio.file.FileSystem
@ -30,7 +38,7 @@ import kotlin.test.assertEquals
import kotlin.test.assertFailsWith import kotlin.test.assertFailsWith
import kotlin.test.assertNull import kotlin.test.assertNull
class NodeAttachmentStorageTest { class NodeAttachmentServiceTest {
// Use an in memory file system for testing attachment storage. // Use an in memory file system for testing attachment storage.
private lateinit var fs: FileSystem private lateinit var fs: FileSystem
private lateinit var database: CordaPersistence private lateinit var database: CordaPersistence
@ -185,7 +193,7 @@ class NodeAttachmentStorageTest {
@Ignore("We need to be able to restart nodes - make importing attachments idempotent?") @Ignore("We need to be able to restart nodes - make importing attachments idempotent?")
@Test @Test
fun `duplicates not allowed`() { fun `duplicates not allowed`() {
val (testJar, _) = makeTestJar() val (testJar) = makeTestJar()
testJar.read { testJar.read {
storage.importAttachment(it, "test", null) storage.importAttachment(it, "test", null)
} }
@ -198,7 +206,7 @@ class NodeAttachmentStorageTest {
@Test @Test
fun `corrupt entry throws exception`() { fun `corrupt entry throws exception`() {
val (testJar, _) = makeTestJar() val (testJar) = makeTestJar()
val id = database.transaction { val id = database.transaction {
val id = testJar.read { storage.importAttachment(it, "test", null) } val id = testJar.read { storage.importAttachment(it, "test", null) }
@ -233,23 +241,68 @@ class NodeAttachmentStorageTest {
} }
} }
@Test
fun `using reserved uploader tokens`() {
val (testJar) = makeTestJar()
fun assertImportFails(uploader: String) {
testJar.read {
assertThatIllegalArgumentException().isThrownBy {
storage.importAttachment(it, uploader, null)
}.withMessageContaining(uploader)
}
}
database.transaction {
assertImportFails(DEPLOYED_CORDAPP_UPLOADER)
assertImportFails(P2P_UPLOADER)
assertImportFails(RPC_UPLOADER)
assertImportFails(UNKNOWN_UPLOADER)
}
// Import an attachment similar to how net.corda.core.internal.FetchAttachmentsFlow does it.
InternalMockNetwork(threadPerNode = true).use { mockNet ->
val node = mockNet.createNode()
val result = node.services.startFlow(FetchAttachmentsFlow()).resultFuture
assertThatIllegalArgumentException().isThrownBy {
result.getOrThrow()
}.withMessageContaining(P2P_UPLOADER)
}
}
// Not the real FetchAttachmentsFlow!
private class FetchAttachmentsFlow : FlowLogic<Unit>() {
@Suspendable
override fun call() {
val baos = ByteArrayOutputStream()
makeTestJar(baos)
serviceHub.attachments.importAttachment(baos.toByteArray().inputStream(), "$P2P_UPLOADER:${ourIdentity.name}", null)
}
}
private var counter = 0 private var counter = 0
private fun makeTestJar(extraEntries: List<Pair<String, String>> = emptyList()): Pair<Path, SecureHash> { private fun makeTestJar(extraEntries: List<Pair<String, String>> = emptyList()): Pair<Path, SecureHash> {
counter++ counter++
val file = fs.getPath("$counter.jar") val file = fs.getPath("$counter.jar")
file.write { makeTestJar(file.outputStream(), extraEntries)
val jar = JarOutputStream(it)
jar.putNextEntry(JarEntry("test1.txt"))
jar.write("This is some useful content".toByteArray())
jar.closeEntry()
jar.putNextEntry(JarEntry("test2.txt"))
jar.write("Some more useful content".toByteArray())
extraEntries.forEach {
jar.putNextEntry(JarEntry(it.first))
jar.write(it.second.toByteArray())
}
jar.closeEntry()
}
return Pair(file, file.readAll().sha256()) return Pair(file, file.readAll().sha256())
} }
private companion object {
private fun makeTestJar(output: OutputStream, extraEntries: List<Pair<String, String>> = emptyList()) {
output.use {
val jar = JarOutputStream(it)
jar.putNextEntry(JarEntry("test1.txt"))
jar.write("This is some useful content".toByteArray())
jar.closeEntry()
jar.putNextEntry(JarEntry("test2.txt"))
jar.write("Some more useful content".toByteArray())
extraEntries.forEach {
jar.putNextEntry(JarEntry(it.first))
jar.write(it.second.toByteArray())
}
jar.closeEntry()
}
}
}
} }

View File

@ -147,7 +147,7 @@ open class InternalMockNetwork(defaultParameters: MockNetworkParameters = MockNe
val testDirectory: Path = Paths.get("build", getTimestampAsDirectoryName()), val testDirectory: Path = Paths.get("build", getTimestampAsDirectoryName()),
val networkParameters: NetworkParameters = testNetworkParameters(), val networkParameters: NetworkParameters = testNetworkParameters(),
val defaultFactory: (MockNodeArgs, CordappLoader?) -> MockNode = { args, cordappLoader -> cordappLoader?.let { MockNode(args, it) } ?: MockNode(args) }, val defaultFactory: (MockNodeArgs, CordappLoader?) -> MockNode = { args, cordappLoader -> cordappLoader?.let { MockNode(args, it) } ?: MockNode(args) },
val cordappsForAllNodes: Set<TestCorDapp> = emptySet()) { val cordappsForAllNodes: Set<TestCorDapp> = emptySet()) : AutoCloseable {
init { init {
// Apache SSHD for whatever reason registers a SFTP FileSystemProvider - which gets loaded by JimFS. // Apache SSHD for whatever reason registers a SFTP FileSystemProvider - which gets loaded by JimFS.
// This SFTP support loads BouncyCastle, which we want to avoid. // This SFTP support loads BouncyCastle, which we want to avoid.
@ -551,6 +551,8 @@ open class InternalMockNetwork(defaultParameters: MockNetworkParameters = MockNe
fun waitQuiescent() { fun waitQuiescent() {
busyLatch.await() busyLatch.await()
} }
override fun close() = stopNodes()
} }
abstract class MessagingServiceSpy { abstract class MessagingServiceSpy {

View File

@ -288,7 +288,7 @@ data class TestLedgerDSLInterpreter private constructor(
copy().dsl() copy().dsl()
override fun attachment(attachment: InputStream): SecureHash { override fun attachment(attachment: InputStream): SecureHash {
return services.attachments.importAttachment(attachment, UNKNOWN_UPLOADER, null) return services.attachments.importAttachment(attachment, "TestDSL", null)
} }
override fun verifies(): EnforceVerifyOrFail { override fun verifies(): EnforceVerifyOrFail {

View File

@ -3,12 +3,11 @@ package net.corda.testing.internal
import net.corda.core.contracts.ContractClassName import net.corda.core.contracts.ContractClassName
import net.corda.core.cordapp.Cordapp import net.corda.core.cordapp.Cordapp
import net.corda.core.crypto.SecureHash import net.corda.core.crypto.SecureHash
import net.corda.core.internal.TEST_UPLOADER import net.corda.core.internal.DEPLOYED_CORDAPP_UPLOADER
import net.corda.core.internal.cordapp.CordappImpl import net.corda.core.internal.cordapp.CordappImpl
import net.corda.core.node.services.AttachmentId import net.corda.core.node.services.AttachmentId
import net.corda.core.node.services.AttachmentStorage import net.corda.core.node.services.AttachmentStorage
import net.corda.node.cordapp.CordappLoader import net.corda.node.cordapp.CordappLoader
import net.corda.node.internal.cordapp.JarScanningCordappLoader
import net.corda.node.internal.cordapp.CordappProviderImpl import net.corda.node.internal.cordapp.CordappProviderImpl
import net.corda.testing.services.MockAttachmentStorage import net.corda.testing.services.MockAttachmentStorage
import java.nio.file.Paths import java.nio.file.Paths
@ -50,7 +49,7 @@ class MockCordappProvider(
return if (!existingAttachment.isEmpty()) { return if (!existingAttachment.isEmpty()) {
existingAttachment.keys.first() existingAttachment.keys.first()
} else { } else {
attachments.importContractAttachment(contractClassNames, TEST_UPLOADER, data.inputStream()) attachments.importContractAttachment(contractClassNames, DEPLOYED_CORDAPP_UPLOADER, data.inputStream())
} }
} }
} }