Remove links to Kryo from serialization "clients" (#1079)

This commit is contained in:
Rick Parker
2017-07-21 14:23:05 +01:00
committed by GitHub
parent e1551fc74e
commit fe9db6f1f7
92 changed files with 1454 additions and 722 deletions

View File

@ -9,13 +9,13 @@ import net.corda.core.messaging.startFlow
import net.corda.core.node.services.ServiceInfo
import net.corda.core.node.services.ServiceType
import net.corda.testing.ALICE
import net.corda.testing.driver.driver
import net.corda.node.internal.NodeStartup
import net.corda.node.services.startFlowPermission
import net.corda.nodeapi.User
import net.corda.testing.driver.ListenProcessDeathException
import net.corda.testing.driver.NetworkMapStartStrategy
import net.corda.testing.ProjectStructure.projectRootDir
import net.corda.testing.driver.driver
import org.assertj.core.api.Assertions.assertThat
import org.assertj.core.api.Assertions.assertThatThrownBy
import org.junit.Test

View File

@ -10,7 +10,10 @@ import com.google.common.collect.testing.features.MapFeature
import com.google.common.collect.testing.features.SetFeature
import com.google.common.collect.testing.testers.*
import junit.framework.TestSuite
import net.corda.testing.TestDependencyInjectionBase
import net.corda.testing.initialiseTestSerialization
import net.corda.testing.node.makeTestDataSourceProperties
import net.corda.testing.resetTestSerialization
import org.assertj.core.api.Assertions.assertThat
import org.jetbrains.exposed.sql.Transaction
import org.jetbrains.exposed.sql.transactions.TransactionManager
@ -42,6 +45,7 @@ class JDBCHashMapTestSuite {
@JvmStatic
@BeforeClass
fun before() {
initialiseTestSerialization()
database = configureDatabase(makeTestDataSourceProperties())
setUpDatabaseTx()
loadOnInitFalseMap = JDBCHashMap<String, String>("test_map_false", loadOnInit = false)
@ -57,6 +61,7 @@ class JDBCHashMapTestSuite {
fun after() {
closeDatabaseTx()
database.close()
resetTestSerialization()
}
@JvmStatic
@ -198,7 +203,7 @@ class JDBCHashMapTestSuite {
*
* If the Map reloads, then so will the Set as it just delegates.
*/
class MapCanBeReloaded {
class MapCanBeReloaded : TestDependencyInjectionBase() {
private val ops = listOf(Triple(AddOrRemove.ADD, "A", "1"),
Triple(AddOrRemove.ADD, "B", "2"),
Triple(AddOrRemove.ADD, "C", "3"),
@ -235,7 +240,6 @@ class JDBCHashMapTestSuite {
database.close()
}
@Test
fun `fill map and check content after reconstruction`() {
database.transaction {

View File

@ -151,7 +151,7 @@ abstract class MQSecurityTest : NodeBasedTest() {
}
fun loginToRPC(target: NetworkHostAndPort, rpcUser: User, sslConfiguration: SSLConfiguration? = null): CordaRPCOps {
return CordaRPCClient(target, sslConfiguration).start(rpcUser.username, rpcUser.password).proxy
return CordaRPCClient(target, sslConfiguration, initialiseSerialization = false).start(rpcUser.username, rpcUser.password).proxy
}
fun loginToRPCAndGetClientQueue(): String {

View File

@ -4,12 +4,15 @@ import com.codahale.metrics.JmxReporter
import com.google.common.util.concurrent.Futures
import com.google.common.util.concurrent.ListenableFuture
import com.google.common.util.concurrent.SettableFuture
import net.corda.core.*
import net.corda.core.flatMap
import net.corda.core.messaging.RPCOps
import net.corda.core.node.ServiceHub
import net.corda.core.node.services.ServiceInfo
import net.corda.core.serialization.SerializationDefaults
import net.corda.core.thenMatch
import net.corda.core.utilities.*
import net.corda.node.VersionInfo
import net.corda.node.serialization.KryoServerSerializationScheme
import net.corda.node.serialization.NodeClock
import net.corda.node.services.RPCUserService
import net.corda.node.services.RPCUserServiceImpl
@ -29,6 +32,7 @@ import net.corda.nodeapi.ArtemisTcpTransport
import net.corda.nodeapi.ConnectionDirection
import net.corda.nodeapi.internal.ShutdownHook
import net.corda.nodeapi.internal.addShutdownHook
import net.corda.nodeapi.serialization.*
import org.apache.activemq.artemis.api.core.ActiveMQNotConnectedException
import org.apache.activemq.artemis.api.core.RoutingType
import org.apache.activemq.artemis.api.core.client.ActiveMQClient
@ -54,7 +58,8 @@ import kotlin.system.exitProcess
open class Node(override val configuration: FullNodeConfiguration,
advertisedServices: Set<ServiceInfo>,
val versionInfo: VersionInfo,
clock: Clock = NodeClock()) : AbstractNode(configuration, advertisedServices, clock) {
clock: Clock = NodeClock(),
val initialiseSerialization: Boolean = true) : AbstractNode(configuration, advertisedServices, clock) {
companion object {
private val logger = loggerFor<Node>()
var renderBasicInfoToConsole = true
@ -290,6 +295,9 @@ open class Node(override val configuration: FullNodeConfiguration,
val startupComplete: ListenableFuture<Unit> = SettableFuture.create()
override fun start(): Node {
if (initialiseSerialization) {
initialiseSerialization()
}
super.start()
networkMapRegistrationFuture.thenMatch({
@ -321,6 +329,16 @@ open class Node(override val configuration: FullNodeConfiguration,
return this
}
private fun initialiseSerialization() {
SerializationDefaults.SERIALIZATION_FACTORY = SerializationFactoryImpl().apply {
registerScheme(KryoServerSerializationScheme())
}
SerializationDefaults.P2P_CONTEXT = KRYO_P2P_CONTEXT
SerializationDefaults.RPC_SERVER_CONTEXT = KRYO_RPC_SERVER_CONTEXT
SerializationDefaults.STORAGE_CONTEXT = KRYO_STORAGE_CONTEXT
SerializationDefaults.CHECKPOINT_CONTEXT = KRYO_CHECKPOINT_CONTEXT
}
/** Starts a blocking event loop for message dispatch. */
fun run() {
(network as NodeMessagingClient).run(messageBroker!!.serverControl)

View File

@ -0,0 +1,26 @@
package net.corda.node.serialization
import com.esotericsoftware.kryo.pool.KryoPool
import net.corda.core.serialization.DefaultKryoCustomizer
import net.corda.core.serialization.SerializationContext
import net.corda.core.utilities.ByteSequence
import net.corda.node.services.messaging.RpcServerObservableSerializer
import net.corda.nodeapi.RPCKryo
import net.corda.nodeapi.serialization.AbstractKryoSerializationScheme
import net.corda.nodeapi.serialization.KryoHeaderV0_1
class KryoServerSerializationScheme : AbstractKryoSerializationScheme() {
override fun canDeserializeVersion(byteSequence: ByteSequence, target: SerializationContext.UseCase): Boolean {
return byteSequence.equals(KryoHeaderV0_1) && target != SerializationContext.UseCase.RPCClient
}
override fun rpcClientKryoPool(context: SerializationContext): KryoPool {
throw UnsupportedOperationException()
}
override fun rpcServerKryoPool(context: SerializationContext): KryoPool {
return KryoPool.Builder {
DefaultKryoCustomizer.customize(RPCKryo(RpcServerObservableSerializer, context.whitelist)).apply { classLoader = context.deserializationClassLoader }
}.build()
}
}

View File

@ -154,7 +154,8 @@ fun <M : Any> MessagingService.onNext(topic: String, sessionId: Long): Listenabl
val messageFuture = SettableFuture.create<M>()
runOnNextMessage(topic, sessionId) { message ->
messageFuture.catch {
message.data.deserialize<M>()
@Suppress("UNCHECKED_CAST")
message.data.deserialize<Any>() as M
}
}
return messageFuture

View File

@ -1,18 +1,21 @@
package net.corda.node.services.messaging
import com.google.common.util.concurrent.ListenableFuture
import net.corda.core.*
import net.corda.core.ThreadBox
import net.corda.core.andForget
import net.corda.core.crypto.random63BitValue
import net.corda.core.getOrThrow
import net.corda.core.messaging.CordaRPCOps
import net.corda.core.messaging.MessageRecipients
import net.corda.core.messaging.RPCOps
import net.corda.core.messaging.SingleMessageRecipient
import net.corda.core.node.services.PartyInfo
import net.corda.core.node.services.TransactionVerifierService
import net.corda.core.utilities.opaque
import net.corda.core.thenMatch
import net.corda.core.transactions.LedgerTransaction
import net.corda.core.utilities.NetworkHostAndPort
import net.corda.core.utilities.loggerFor
import net.corda.core.utilities.sequence
import net.corda.core.utilities.trace
import net.corda.node.VersionInfo
import net.corda.node.services.RPCUserService
@ -346,7 +349,7 @@ class NodeMessagingClient(override val config: NodeConfiguration,
private val message: ClientMessage) : ReceivedMessage {
override val data: ByteArray by lazy { ByteArray(message.bodySize).apply { message.bodyBuffer.readBytes(this) } }
override val debugTimestamp: Instant get() = Instant.ofEpochMilli(message.timestamp)
override fun toString() = "${topicSession.topic}#${data.opaque()}"
override fun toString() = "${topicSession.topic}#${data.sequence()}"
}
private fun deliver(msg: ReceivedMessage): Boolean {

View File

@ -4,7 +4,6 @@ import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.Serializer
import com.esotericsoftware.kryo.io.Input
import com.esotericsoftware.kryo.io.Output
import com.esotericsoftware.kryo.pool.KryoPool
import com.google.common.cache.Cache
import com.google.common.cache.CacheBuilder
import com.google.common.cache.RemovalListener
@ -17,7 +16,8 @@ import net.corda.core.internal.LazyStickyPool
import net.corda.core.internal.LifeCycle
import net.corda.core.messaging.RPCOps
import net.corda.core.utilities.seconds
import net.corda.core.serialization.KryoPoolWithContext
import net.corda.core.serialization.SerializationContext
import net.corda.core.serialization.SerializationDefaults.RPC_SERVER_CONTEXT
import net.corda.core.utilities.*
import net.corda.node.services.RPCUserService
import net.corda.nodeapi.*
@ -81,7 +81,6 @@ class RPCServer(
) {
private companion object {
val log = loggerFor<RPCServer>()
val kryoPool = KryoPool.Builder { RPCKryo(RpcServerObservableSerializer) }.build()
}
private enum class State {
UNSTARTED,
@ -258,7 +257,7 @@ class RPCServer(
private fun clientArtemisMessageHandler(artemisMessage: ClientMessage) {
lifeCycle.requireState(State.STARTED)
val clientToServer = RPCApi.ClientToServer.fromClientMessage(kryoPool, artemisMessage)
val clientToServer = RPCApi.ClientToServer.fromClientMessage(RPC_SERVER_CONTEXT, artemisMessage)
log.debug { "-> RPC -> $clientToServer" }
when (clientToServer) {
is RPCApi.ClientToServer.RpcRequest -> {
@ -302,8 +301,7 @@ class RPCServer(
clientAddress,
serverControl!!,
sessionAndProducerPool,
observationSendExecutor!!,
kryoPool
observationSendExecutor!!
)
val buffered = bufferIfQueueNotBound(clientAddress, reply, observableContext)
@ -385,19 +383,19 @@ class ObservableContext(
val clientAddress: SimpleString,
val serverControl: ActiveMQServerControl,
val sessionAndProducerPool: LazyStickyPool<ArtemisProducer>,
val observationSendExecutor: ExecutorService,
kryoPool: KryoPool
val observationSendExecutor: ExecutorService
) {
private companion object {
val log = loggerFor<ObservableContext>()
}
private val kryoPoolWithObservableContext = RpcServerObservableSerializer.createPoolWithContext(kryoPool, this)
private val serializationContextWithObservableContext = RpcServerObservableSerializer.createContext(this)
fun sendMessage(serverToClient: RPCApi.ServerToClient) {
try {
sessionAndProducerPool.run(rpcRequestId) {
val artemisMessage = it.session.createMessage(false)
serverToClient.writeToClientMessage(kryoPoolWithObservableContext, artemisMessage)
serverToClient.writeToClientMessage(serializationContextWithObservableContext, artemisMessage)
it.producer.send(clientAddress, artemisMessage)
log.debug("<- RPC <- $serverToClient")
}
@ -408,12 +406,12 @@ class ObservableContext(
}
}
private object RpcServerObservableSerializer : Serializer<Observable<Any>>() {
object RpcServerObservableSerializer : Serializer<Observable<Any>>() {
private object RpcObservableContextKey
private val log = loggerFor<RpcServerObservableSerializer>()
fun createPoolWithContext(kryoPool: KryoPool, observableContext: ObservableContext): KryoPool {
return KryoPoolWithContext(kryoPool, RpcObservableContextKey, observableContext)
fun createContext(observableContext: ObservableContext): SerializationContext {
return RPC_SERVER_CONTEXT.withProperty(RpcServerObservableSerializer.RpcObservableContextKey, observableContext)
}
override fun read(kryo: Kryo?, input: Input?, type: Class<Observable<Any>>?): Observable<Any> {

View File

@ -1,10 +1,10 @@
package net.corda.node.services.persistence
import net.corda.core.crypto.SecureHash
import net.corda.core.serialization.SerializationDefaults.CHECKPOINT_CONTEXT
import net.corda.core.serialization.SerializedBytes
import net.corda.core.serialization.deserialize
import net.corda.core.serialization.serialize
import net.corda.core.serialization.storageKryo
import net.corda.node.services.api.Checkpoint
import net.corda.node.services.api.CheckpointStorage
import net.corda.node.utilities.*
@ -39,7 +39,7 @@ class DBCheckpointStorage : CheckpointStorage {
private val checkpointStorage = synchronizedMap(CheckpointMap())
override fun addCheckpoint(checkpoint: Checkpoint) {
checkpointStorage.put(checkpoint.id, checkpoint.serialize(storageKryo(), true))
checkpointStorage.put(checkpoint.id, checkpoint.serialize(context = CHECKPOINT_CONTEXT))
}
override fun removeCheckpoint(checkpoint: Checkpoint) {
@ -49,7 +49,7 @@ class DBCheckpointStorage : CheckpointStorage {
override fun forEach(block: (Checkpoint) -> Boolean) {
synchronized(checkpointStorage) {
for (checkpoint in checkpointStorage.values) {
if (!block(checkpoint.deserialize())) {
if (!block(checkpoint.deserialize(context = CHECKPOINT_CONTEXT))) {
break
}
}

View File

@ -2,20 +2,12 @@ package net.corda.node.services.statemachine
import co.paralleluniverse.fibers.Fiber
import co.paralleluniverse.fibers.FiberExecutorScheduler
import co.paralleluniverse.io.serialization.kryo.KryoSerializer
import co.paralleluniverse.strands.Strand
import com.codahale.metrics.Gauge
import com.esotericsoftware.kryo.ClassResolver
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.KryoException
import com.esotericsoftware.kryo.Serializer
import com.esotericsoftware.kryo.io.Input
import com.esotericsoftware.kryo.io.Output
import com.esotericsoftware.kryo.pool.KryoPool
import com.google.common.collect.HashMultimap
import com.google.common.util.concurrent.ListenableFuture
import com.google.common.util.concurrent.MoreExecutors
import io.requery.util.CloseableIterator
import net.corda.core.ThreadBox
import net.corda.core.bufferUntilSubscribed
import net.corda.core.crypto.SecureHash
@ -25,9 +17,10 @@ import net.corda.core.flows.FlowInitiator
import net.corda.core.flows.FlowLogic
import net.corda.core.flows.StateMachineRunId
import net.corda.core.identity.Party
import net.corda.core.internal.declaredField
import net.corda.core.messaging.DataFeed
import net.corda.core.serialization.*
import net.corda.core.serialization.SerializationDefaults.CHECKPOINT_CONTEXT
import net.corda.core.serialization.SerializationDefaults.SERIALIZATION_FACTORY
import net.corda.core.then
import net.corda.core.utilities.Try
import net.corda.core.utilities.debug
@ -85,34 +78,6 @@ class StateMachineManager(val serviceHub: ServiceHubInternal,
inner class FiberScheduler : FiberExecutorScheduler("Same thread scheduler", executor)
private val quasarKryoPool = KryoPool.Builder {
val serializer = Fiber.getFiberSerializer(false) as KryoSerializer
val classResolver = makeNoWhitelistClassResolver().apply { setKryo(serializer.kryo) }
serializer.kryo.apply {
// TODO The ClassResolver can only be set in the Kryo constructor and Quasar doesn't provide us with a way of doing that
declaredField<ClassResolver>(Kryo::class, "classResolver").value = classResolver
DefaultKryoCustomizer.customize(this)
addDefaultSerializer(AutoCloseable::class.java, AutoCloseableSerialisationDetector)
}
}.build()
// TODO Move this into the blacklist and upgrade the blacklist to allow custom messages
private object AutoCloseableSerialisationDetector : Serializer<AutoCloseable>() {
override fun write(kryo: Kryo, output: Output, closeable: AutoCloseable) {
val message = if (closeable is CloseableIterator<*>) {
"A live Iterator pointing to the database has been detected during flow checkpointing. This may be due " +
"to a Vault query - move it into a private method."
} else {
"${closeable.javaClass.name}, which is a closeable resource, has been detected during flow checkpointing. " +
"Restoring such resources across node restarts is not supported. Make sure code accessing it is " +
"confined to a private method or the reference is nulled out."
}
throw UnsupportedOperationException(message)
}
override fun read(kryo: Kryo, input: Input, type: Class<AutoCloseable>) = throw IllegalStateException("Should not reach here!")
}
companion object {
private val logger = loggerFor<StateMachineManager>()
internal val sessionTopic = TopicSession("platform.session")
@ -173,7 +138,7 @@ class StateMachineManager(val serviceHub: ServiceHubInternal,
internal val tokenizableServices = ArrayList<Any>()
// Context for tokenized services in checkpoints
private val serializationContext by lazy {
SerializeAsTokenContext(tokenizableServices, quasarKryoPool, serviceHub)
SerializeAsTokenContext(tokenizableServices, SERIALIZATION_FACTORY, CHECKPOINT_CONTEXT, serviceHub)
}
/** Returns a list of all state machines executing the given flow logic at the top level (subflows do not count) */
@ -410,22 +375,12 @@ class StateMachineManager(val serviceHub: ServiceHubInternal,
}
private fun serializeFiber(fiber: FlowStateMachineImpl<*>): SerializedBytes<FlowStateMachineImpl<*>> {
return quasarKryoPool.run { kryo ->
// add the map of tokens -> tokenizedServices to the kyro context
kryo.withSerializationContext(serializationContext) {
fiber.serialize(kryo)
}
}
return fiber.serialize(context = CHECKPOINT_CONTEXT.withTokenContext(serializationContext))
}
private fun deserializeFiber(checkpoint: Checkpoint, logger: Logger): FlowStateMachineImpl<*>? {
return try {
quasarKryoPool.run { kryo ->
// put the map of token -> tokenized into the kryo context
kryo.withSerializationContext(serializationContext) {
checkpoint.serializedFiber.deserialize(kryo)
}.apply { fromCheckpoint = true }
}
checkpoint.serializedFiber.deserialize<FlowStateMachineImpl<*>>(context = CHECKPOINT_CONTEXT.withTokenContext(serializationContext)).apply { fromCheckpoint = true }
} catch (t: Throwable) {
logger.error("Encountered unrestorable checkpoint!", t)
null

View File

@ -13,9 +13,9 @@ import net.corda.core.node.services.VaultQueryException
import net.corda.core.node.services.VaultQueryService
import net.corda.core.node.services.vault.*
import net.corda.core.node.services.vault.QueryCriteria.VaultCustomQueryCriteria
import net.corda.core.serialization.SerializationDefaults.STORAGE_CONTEXT
import net.corda.core.serialization.SingletonSerializeAsToken
import net.corda.core.serialization.deserialize
import net.corda.core.serialization.storageKryo
import net.corda.core.utilities.debug
import net.corda.core.utilities.loggerFor
import net.corda.node.services.database.HibernateConfiguration
@ -96,7 +96,7 @@ class HibernateVaultQueryImpl(hibernateConfig: HibernateConfiguration,
return@forEachIndexed
val vaultState = result[0] as VaultSchemaV1.VaultStates
val stateRef = StateRef(SecureHash.parse(vaultState.stateRef!!.txId!!), vaultState.stateRef!!.index!!)
val state = vaultState.contractState.deserialize<TransactionState<T>>(storageKryo())
val state = vaultState.contractState.deserialize<TransactionState<T>>(context = STORAGE_CONTEXT)
statesMeta.add(Vault.StateMetadata(stateRef, vaultState.contractStateClassName, vaultState.recordedTime, vaultState.consumedTime, vaultState.stateStatus, vaultState.notaryName, vaultState.notaryKey, vaultState.lockId, vaultState.lockUpdateTime))
statesAndRefs.add(StateAndRef(state, stateRef))
}

View File

@ -26,10 +26,10 @@ import net.corda.core.node.services.StatesNotAvailableException
import net.corda.core.node.services.Vault
import net.corda.core.node.services.VaultService
import net.corda.core.node.services.unconsumedStates
import net.corda.core.serialization.SerializationDefaults.STORAGE_CONTEXT
import net.corda.core.serialization.SingletonSerializeAsToken
import net.corda.core.serialization.deserialize
import net.corda.core.serialization.serialize
import net.corda.core.serialization.storageKryo
import net.corda.core.tee
import net.corda.core.transactions.TransactionBuilder
import net.corda.core.transactions.WireTransaction
@ -95,7 +95,7 @@ class NodeVaultService(private val services: ServiceHub, dataSourceProperties: P
index = it.key.index
stateStatus = Vault.StateStatus.UNCONSUMED
contractStateClassName = it.value.state.data.javaClass.name
contractState = it.value.state.serialize(storageKryo()).bytes
contractState = it.value.state.serialize(context = STORAGE_CONTEXT).bytes
notaryName = it.value.state.notary.name.toString()
notaryKey = it.value.state.notary.owningKey.toBase58String()
recordedTime = services.clock.instant()
@ -198,7 +198,7 @@ class NodeVaultService(private val services: ServiceHub, dataSourceProperties: P
Sequence { iterator }
.map { it ->
val stateRef = StateRef(SecureHash.parse(it.txId), it.index)
val state = it.contractState.deserialize<TransactionState<T>>(storageKryo())
val state = it.contractState.deserialize<TransactionState<T>>(context = STORAGE_CONTEXT)
Vault.StateMetadata(stateRef, it.contractStateClassName, it.recordedTime, it.consumedTime, it.stateStatus, it.notaryName, it.notaryKey, it.lockId, it.lockUpdateTime)
StateAndRef(state, stateRef)
}
@ -217,7 +217,7 @@ class NodeVaultService(private val services: ServiceHub, dataSourceProperties: P
.and(VaultSchema.VaultStates::index eq it.index)
result.get()?.each {
val stateRef = StateRef(SecureHash.parse(it.txId), it.index)
val state = it.contractState.deserialize<TransactionState<*>>(storageKryo())
val state = it.contractState.deserialize<TransactionState<*>>(context = STORAGE_CONTEXT)
results += StateAndRef(state, stateRef)
}
}
@ -380,7 +380,7 @@ class NodeVaultService(private val services: ServiceHub, dataSourceProperties: P
val txHash = SecureHash.parse(rs.getString(1))
val index = rs.getInt(2)
val stateRef = StateRef(txHash, index)
val state = rs.getBytes(3).deserialize<TransactionState<T>>(storageKryo())
val state = rs.getBytes(3).deserialize<TransactionState<T>>(context = STORAGE_CONTEXT)
val pennies = rs.getLong(4)
totalPennies = rs.getLong(5)
val rowLockId = rs.getString(6)
@ -435,7 +435,7 @@ class NodeVaultService(private val services: ServiceHub, dataSourceProperties: P
query.get()
.map { it ->
val stateRef = StateRef(SecureHash.parse(it.txId), it.index)
val state = it.contractState.deserialize<TransactionState<T>>(storageKryo())
val state = it.contractState.deserialize<TransactionState<T>>(context = STORAGE_CONTEXT)
StateAndRef(state, stateRef)
}.toList()
}
@ -480,7 +480,7 @@ class NodeVaultService(private val services: ServiceHub, dataSourceProperties: P
result.get().forEach {
val txHash = SecureHash.parse(it.txId)
val index = it.index
val state = it.contractState.deserialize<TransactionState<ContractState>>(storageKryo())
val state = it.contractState.deserialize<TransactionState<ContractState>>(context = STORAGE_CONTEXT)
consumedStates.add(StateAndRef(state, StateRef(txHash, index)))
}
}

View File

@ -1,9 +1,9 @@
package net.corda.node.utilities
import net.corda.core.serialization.SerializationDefaults.STORAGE_CONTEXT
import net.corda.core.serialization.SerializedBytes
import net.corda.core.serialization.deserialize
import net.corda.core.serialization.serialize
import net.corda.core.serialization.storageKryo
import net.corda.core.utilities.loggerFor
import net.corda.core.utilities.trace
import org.jetbrains.exposed.sql.*
@ -65,17 +65,18 @@ fun bytesToBlob(value: SerializedBytes<*>, finalizables: MutableList<() -> Unit>
return blob
}
fun serializeToBlob(value: Any, finalizables: MutableList<() -> Unit>): Blob = bytesToBlob(value.serialize(storageKryo(), true), finalizables)
fun serializeToBlob(value: Any, finalizables: MutableList<() -> Unit>): Blob = bytesToBlob(value.serialize(context = STORAGE_CONTEXT), finalizables)
fun <T : Any> bytesFromBlob(blob: Blob): SerializedBytes<T> {
try {
return SerializedBytes(blob.getBytes(0, blob.length().toInt()), true)
return SerializedBytes(blob.getBytes(0, blob.length().toInt()))
} finally {
blob.free()
}
}
fun <T : Any> deserializeFromBlob(blob: Blob): T = bytesFromBlob<T>(blob).deserialize()
@Suppress("UNCHECKED_CAST")
fun <T : Any> deserializeFromBlob(blob: Blob): T = bytesFromBlob<Any>(blob).deserialize(context = STORAGE_CONTEXT) as T
/**
* A convenient JDBC table backed hash set with iteration order based on insertion order.

View File

@ -37,7 +37,6 @@ object ServiceIdentityGenerator {
keyPairs.zip(dirs) { keyPair, dir ->
Files.createDirectories(dir)
Files.write(dir.resolve(compositeKeyFile), notaryKey.encoded)
// Use storageKryo as our whitelist is not available in the gradle build environment:
Files.write(dir.resolve(privateKeyFile), keyPair.private.encoded)
Files.write(dir.resolve(publicKeyFile), keyPair.public.encoded)
}

View File

@ -1,47 +1,60 @@
package net.corda.node.services.vault;
import com.google.common.collect.*;
import kotlin.*;
import net.corda.contracts.*;
import net.corda.contracts.asset.*;
import com.google.common.collect.ImmutableSet;
import net.corda.contracts.DealState;
import net.corda.contracts.asset.Cash;
import net.corda.core.contracts.*;
import net.corda.core.crypto.*;
import net.corda.core.identity.*;
import net.corda.core.messaging.*;
import net.corda.core.node.services.*;
import net.corda.core.crypto.EncodingUtils;
import net.corda.core.crypto.SecureHash;
import net.corda.core.identity.AbstractParty;
import net.corda.core.messaging.DataFeed;
import net.corda.core.node.services.Vault;
import net.corda.core.node.services.VaultQueryException;
import net.corda.core.node.services.VaultQueryService;
import net.corda.core.node.services.VaultService;
import net.corda.core.node.services.vault.*;
import net.corda.core.node.services.vault.QueryCriteria.*;
import net.corda.testing.contracts.DummyLinearContract;
import net.corda.core.schemas.*;
import net.corda.core.transactions.*;
import net.corda.core.utilities.*;
import net.corda.core.node.services.vault.QueryCriteria.LinearStateQueryCriteria;
import net.corda.core.node.services.vault.QueryCriteria.VaultCustomQueryCriteria;
import net.corda.core.node.services.vault.QueryCriteria.VaultQueryCriteria;
import net.corda.core.schemas.MappedSchema;
import net.corda.core.transactions.SignedTransaction;
import net.corda.core.transactions.WireTransaction;
import net.corda.core.utilities.OpaqueBytes;
import net.corda.node.services.database.HibernateConfiguration;
import net.corda.node.services.schema.NodeSchemaService;
import net.corda.node.utilities.CordaPersistence;
import net.corda.node.services.database.*;
import net.corda.node.services.schema.*;
import net.corda.schemas.*;
import net.corda.testing.*;
import net.corda.testing.contracts.*;
import net.corda.testing.node.*;
import net.corda.schemas.CashSchemaV1;
import net.corda.testing.TestConstants;
import net.corda.testing.TestDependencyInjectionBase;
import net.corda.testing.contracts.DummyLinearContract;
import net.corda.testing.contracts.VaultFiller;
import net.corda.testing.node.MockServices;
import net.corda.testing.schemas.DummyLinearStateSchemaV1;
import org.jetbrains.annotations.*;
import org.junit.*;
import org.jetbrains.annotations.NotNull;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import rx.Observable;
import java.io.*;
import java.lang.reflect.*;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.*;
import java.util.stream.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static net.corda.contracts.asset.CashKt.*;
import static net.corda.core.contracts.ContractsDSL.*;
import static net.corda.core.node.services.vault.QueryCriteriaUtils.*;
import static net.corda.contracts.asset.CashKt.getDUMMY_CASH_ISSUER;
import static net.corda.contracts.asset.CashKt.getDUMMY_CASH_ISSUER_KEY;
import static net.corda.core.contracts.ContractsDSL.USD;
import static net.corda.core.node.services.vault.QueryCriteriaUtils.DEFAULT_PAGE_NUM;
import static net.corda.core.node.services.vault.QueryCriteriaUtils.MAX_PAGE_SIZE;
import static net.corda.core.utilities.ByteArrays.toHexString;
import static net.corda.node.utilities.CordaPersistenceKt.configureDatabase;
import static net.corda.testing.CoreTestUtils.*;
import static net.corda.testing.node.MockServicesKt.*;
import static net.corda.core.utilities.ByteArrays.toHexString;
import static org.assertj.core.api.Assertions.*;
import static net.corda.testing.node.MockServicesKt.makeTestDataSourceProperties;
import static org.assertj.core.api.Assertions.assertThat;
public class VaultQueryJavaTests {
public class VaultQueryJavaTests extends TestDependencyInjectionBase {
private MockServices services;
private VaultService vaultSvc;

View File

@ -8,6 +8,8 @@ import net.corda.node.services.messaging.createMessage
import net.corda.node.services.network.NetworkMapService
import net.corda.testing.node.MockNetwork
import org.junit.After
import net.corda.testing.resetTestSerialization
import org.junit.Before
import org.junit.Test
import java.util.*
import kotlin.test.assertEquals
@ -15,11 +17,20 @@ import kotlin.test.assertFails
import kotlin.test.assertTrue
class InMemoryMessagingTests {
val mockNet = MockNetwork()
lateinit var mockNet: MockNetwork
@Before
fun setUp() {
mockNet = MockNetwork()
}
@After
fun cleanUp() {
mockNet.stopNodes()
fun tearDown() {
if (mockNet.nodes.isNotEmpty()) {
mockNet.stopNodes()
} else {
resetTestSerialization()
}
}
@Test

View File

@ -76,7 +76,6 @@ class TwoPartyTradeFlowTests {
@Before
fun before() {
mockNet = MockNetwork(false)
LogHelper.setLevel("platform.trade", "core.contract.TransactionGroup", "recordingmap")
}
@ -93,7 +92,7 @@ class TwoPartyTradeFlowTests {
// allow interruption half way through.
mockNet = MockNetwork(false, true)
ledger {
ledger(initialiseSerialization = false) {
val basketOfNodes = mockNet.createSomeNodes(3)
val notaryNode = basketOfNodes.notaryNode
val aliceNode = basketOfNodes.partyNodes[0]
@ -140,7 +139,7 @@ class TwoPartyTradeFlowTests {
fun `trade cash for commercial paper fails using soft locking`() {
mockNet = MockNetwork(false, true)
ledger {
ledger(initialiseSerialization = false) {
val notaryNode = mockNet.createNotaryNode(null, DUMMY_NOTARY.name)
val aliceNode = mockNet.createPartyNode(notaryNode.network.myAddress, ALICE.name)
val bobNode = mockNet.createPartyNode(notaryNode.network.myAddress, BOB.name)
@ -191,7 +190,8 @@ class TwoPartyTradeFlowTests {
@Test
fun `shutdown and restore`() {
ledger {
mockNet = MockNetwork(false)
ledger(initialiseSerialization = false) {
val notaryNode = mockNet.createNotaryNode(null, DUMMY_NOTARY.name)
val aliceNode = mockNet.createPartyNode(notaryNode.network.myAddress, ALICE.name)
var bobNode = mockNet.createPartyNode(notaryNode.network.myAddress, BOB.name)
@ -313,13 +313,15 @@ class TwoPartyTradeFlowTests {
@Test
fun `check dependencies of sale asset are resolved`() {
mockNet = MockNetwork(false)
val notaryNode = mockNet.createNotaryNode(null, DUMMY_NOTARY.name)
val aliceNode = makeNodeWithTracking(notaryNode.network.myAddress, ALICE.name)
val bobNode = makeNodeWithTracking(notaryNode.network.myAddress, BOB.name)
val bankNode = makeNodeWithTracking(notaryNode.network.myAddress, BOC.name)
val issuer = bankNode.info.legalIdentity.ref(1, 2, 3)
ledger(aliceNode.services) {
ledger(aliceNode.services, initialiseSerialization = false) {
// Insert a prospectus type attachment into the commercial paper transaction.
val stream = ByteArrayOutputStream()
@ -412,13 +414,15 @@ class TwoPartyTradeFlowTests {
@Test
fun `track works`() {
mockNet = MockNetwork(false)
val notaryNode = mockNet.createNotaryNode(null, DUMMY_NOTARY.name)
val aliceNode = makeNodeWithTracking(notaryNode.network.myAddress, ALICE.name)
val bobNode = makeNodeWithTracking(notaryNode.network.myAddress, BOB.name)
val bankNode = makeNodeWithTracking(notaryNode.network.myAddress, BOC.name)
val issuer = bankNode.info.legalIdentity.ref(1, 2, 3)
ledger(aliceNode.services) {
ledger(aliceNode.services, initialiseSerialization = false) {
// Insert a prospectus type attachment into the commercial paper transaction.
val stream = ByteArrayOutputStream()
@ -487,14 +491,16 @@ class TwoPartyTradeFlowTests {
@Test
fun `dependency with error on buyer side`() {
ledger {
mockNet = MockNetwork(false)
ledger(initialiseSerialization = false) {
runWithError(true, false, "at least one asset input")
}
}
@Test
fun `dependency with error on seller side`() {
ledger {
mockNet = MockNetwork(false)
ledger(initialiseSerialization = false) {
runWithError(false, true, "Issuances must have a time-window")
}
}

View File

@ -34,7 +34,7 @@ class NotaryChangeTests {
lateinit var clientNodeB: MockNetwork.MockNode
@Before
fun setup() {
fun setUp() {
mockNet = MockNetwork()
oldNotaryNode = mockNet.createNode(
legalName = DUMMY_NOTARY.name,

View File

@ -3,39 +3,32 @@ package net.corda.node.services.database
import net.corda.contracts.asset.Cash
import net.corda.contracts.asset.DUMMY_CASH_ISSUER
import net.corda.contracts.asset.DummyFungibleContract
import net.corda.testing.contracts.consumeCash
import net.corda.testing.contracts.fillWithSomeTestCash
import net.corda.testing.contracts.fillWithSomeTestDeals
import net.corda.testing.contracts.fillWithSomeTestLinearStates
import net.corda.core.contracts.*
import net.corda.core.crypto.toBase58String
import net.corda.core.node.services.Vault
import net.corda.core.node.services.VaultService
import net.corda.core.schemas.CommonSchemaV1
import net.corda.core.schemas.PersistentStateRef
import net.corda.testing.schemas.DummyLinearStateSchemaV1
import net.corda.testing.schemas.DummyLinearStateSchemaV2
import net.corda.core.serialization.storageKryo
import net.corda.core.serialization.deserialize
import net.corda.core.transactions.SignedTransaction
import net.corda.testing.ALICE
import net.corda.testing.BOB
import net.corda.testing.BOB_KEY
import net.corda.testing.DUMMY_NOTARY
import net.corda.node.services.schema.HibernateObserver
import net.corda.node.services.schema.NodeSchemaService
import net.corda.node.services.vault.NodeVaultService
import net.corda.core.schemas.CommonSchemaV1
import net.corda.core.serialization.deserialize
import net.corda.node.services.vault.VaultSchemaV1
import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.schemas.CashSchemaV1
import net.corda.schemas.SampleCashSchemaV2
import net.corda.schemas.SampleCashSchemaV3
import net.corda.testing.BOB_PUBKEY
import net.corda.testing.BOC
import net.corda.testing.BOC_KEY
import net.corda.testing.*
import net.corda.testing.contracts.consumeCash
import net.corda.testing.contracts.fillWithSomeTestCash
import net.corda.testing.contracts.fillWithSomeTestDeals
import net.corda.testing.contracts.fillWithSomeTestLinearStates
import net.corda.testing.node.MockServices
import net.corda.testing.node.makeTestDataSourceProperties
import net.corda.testing.schemas.DummyLinearStateSchemaV1
import net.corda.testing.schemas.DummyLinearStateSchemaV2
import org.assertj.core.api.Assertions
import org.assertj.core.api.Assertions.assertThat
import org.hibernate.SessionFactory
@ -48,7 +41,7 @@ import javax.persistence.EntityManager
import javax.persistence.Tuple
import javax.persistence.criteria.CriteriaBuilder
class HibernateConfigurationTest {
class HibernateConfigurationTest : TestDependencyInjectionBase() {
lateinit var services: MockServices
lateinit var database: CordaPersistence
@ -655,7 +648,7 @@ class HibernateConfigurationTest {
val queryResults = entityManager.createQuery(criteriaQuery).resultList
queryResults.forEach {
val contractState = it.contractState.deserialize<TransactionState<ContractState>>(storageKryo())
val contractState = it.contractState.deserialize<TransactionState<ContractState>>()
val cashState = contractState.data as Cash.State
println("${it.stateRef} with owner: ${cashState.owner.owningKey.toBase58String()}") }
@ -739,7 +732,7 @@ class HibernateConfigurationTest {
// execute query
val queryResults = entityManager.createQuery(criteriaQuery).resultList
queryResults.forEach {
val contractState = it.contractState.deserialize<TransactionState<ContractState>>(storageKryo())
val contractState = it.contractState.deserialize<TransactionState<ContractState>>()
val cashState = contractState.data as Cash.State
println("${it.stateRef} with owner ${cashState.owner.owningKey.toBase58String()} and participants ${cashState.participants.map { it.owningKey.toBase58String() }}")
}

View File

@ -5,7 +5,6 @@ import io.requery.kotlin.eq
import io.requery.sql.KotlinEntityDataStore
import net.corda.core.contracts.StateRef
import net.corda.core.contracts.TransactionType
import net.corda.testing.contracts.DummyContract
import net.corda.core.crypto.DigitalSignature
import net.corda.core.crypto.SecureHash
import net.corda.core.crypto.testing.NullPublicKey
@ -13,11 +12,8 @@ import net.corda.core.crypto.toBase58String
import net.corda.core.identity.AnonymousParty
import net.corda.core.node.services.Vault
import net.corda.core.serialization.serialize
import net.corda.core.serialization.storageKryo
import net.corda.core.transactions.SignedTransaction
import net.corda.core.transactions.WireTransaction
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.DUMMY_PUBKEY_1
import net.corda.node.services.persistence.DBTransactionStorage
import net.corda.node.services.vault.schemas.requery.Models
import net.corda.node.services.vault.schemas.requery.VaultCashBalancesEntity
@ -25,6 +21,10 @@ import net.corda.node.services.vault.schemas.requery.VaultSchema
import net.corda.node.services.vault.schemas.requery.VaultStatesEntity
import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.DUMMY_PUBKEY_1
import net.corda.testing.TestDependencyInjectionBase
import net.corda.testing.contracts.DummyContract
import net.corda.testing.node.makeTestDataSourceProperties
import org.assertj.core.api.Assertions
import org.junit.After
@ -35,7 +35,7 @@ import org.junit.Test
import java.time.Instant
import java.util.*
class RequeryConfigurationTest {
class RequeryConfigurationTest : TestDependencyInjectionBase() {
lateinit var database: CordaPersistence
lateinit var transactionStorage: DBTransactionStorage
@ -175,7 +175,7 @@ class RequeryConfigurationTest {
index = txnState.index
stateStatus = Vault.StateStatus.UNCONSUMED
contractStateClassName = DummyContract.SingleOwnerState::class.java.name
contractState = DummyContract.SingleOwnerState(owner = AnonymousParty(DUMMY_PUBKEY_1)).serialize(storageKryo()).bytes
contractState = DummyContract.SingleOwnerState(owner = AnonymousParty(DUMMY_PUBKEY_1)).serialize().bytes
notaryName = txn.tx.notary!!.name.toString()
notaryKey = txn.tx.notary!!.owningKey.toBase58String()
recordedTime = Instant.now()

View File

@ -25,6 +25,8 @@ import net.corda.testing.node.MockKeyManagementService
import net.corda.testing.node.TestClock
import net.corda.testing.node.makeTestDataSourceProperties
import net.corda.testing.testNodeConfiguration
import net.corda.testing.initialiseTestSerialization
import net.corda.testing.resetTestSerialization
import org.assertj.core.api.Assertions.assertThat
import org.bouncycastle.asn1.x500.X500Name
import org.junit.After
@ -67,6 +69,7 @@ class NodeSchedulerServiceTest : SingletonSerializeAsToken() {
@Before
fun setup() {
initialiseTestSerialization()
countDown = CountDownLatch(1)
smmHasRemovedAllFlows = CountDownLatch(1)
calls = 0
@ -114,6 +117,7 @@ class NodeSchedulerServiceTest : SingletonSerializeAsToken() {
smmExecutor.shutdown()
smmExecutor.awaitTermination(60, TimeUnit.SECONDS)
database.close()
resetTestSerialization()
}
class TestState(val flowLogicRef: FlowLogicRef, val instant: Instant) : LinearState, SchedulableState {

View File

@ -37,7 +37,7 @@ import kotlin.test.assertEquals
import kotlin.test.assertNull
//TODO This needs to be merged into P2PMessagingTest as that creates a more realistic environment
class ArtemisMessagingTests {
class ArtemisMessagingTests : TestDependencyInjectionBase() {
@Rule @JvmField val temporaryFolder = TemporaryFolder()
val serverPort = freePort()

View File

@ -80,16 +80,18 @@ class InMemoryIdentityServiceTests {
*/
@Test
fun `assert unknown anonymous key is unrecognised`() {
val rootKey = Crypto.generateKeyPair(X509Utilities.DEFAULT_TLS_SIGNATURE_SCHEME)
val rootCert = X509Utilities.createSelfSignedCACertificate(ALICE.name, rootKey)
val txKey = Crypto.generateKeyPair(X509Utilities.DEFAULT_TLS_SIGNATURE_SCHEME)
val service = InMemoryIdentityService(trustRoot = DUMMY_CA.certificate)
// TODO: Generate certificate with an EdDSA key rather than ECDSA
val identity = Party(CertificateAndKeyPair(rootCert, rootKey))
val txIdentity = AnonymousParty(txKey.public)
withTestSerialization {
val rootKey = Crypto.generateKeyPair(X509Utilities.DEFAULT_TLS_SIGNATURE_SCHEME)
val rootCert = X509Utilities.createSelfSignedCACertificate(ALICE.name, rootKey)
val txKey = Crypto.generateKeyPair(X509Utilities.DEFAULT_TLS_SIGNATURE_SCHEME)
val service = InMemoryIdentityService(trustRoot = DUMMY_CA.certificate)
// TODO: Generate certificate with an EdDSA key rather than ECDSA
val identity = Party(CertificateAndKeyPair(rootCert, rootKey))
val txIdentity = AnonymousParty(txKey.public)
assertFailsWith<IdentityService.UnknownAnonymousPartyException> {
service.assertOwnership(identity, txIdentity)
assertFailsWith<IdentityService.UnknownAnonymousPartyException> {
service.assertOwnership(identity, txIdentity)
}
}
}
@ -122,38 +124,40 @@ class InMemoryIdentityServiceTests {
*/
@Test
fun `assert ownership`() {
val trustRoot = DUMMY_CA
val (alice, aliceTxIdentity) = createParty(ALICE.name, trustRoot)
withTestSerialization {
val trustRoot = DUMMY_CA
val (alice, aliceTxIdentity) = createParty(ALICE.name, trustRoot)
val certFactory = CertificateFactory.getInstance("X509")
val bobRootKey = Crypto.generateKeyPair()
val bobRoot = getTestPartyAndCertificate(BOB.name, bobRootKey.public)
val bobRootCert = bobRoot.certificate
val bobTxKey = Crypto.generateKeyPair()
val bobTxCert = X509Utilities.createCertificate(CertificateType.IDENTITY, bobRootCert, bobRootKey, BOB.name, bobTxKey.public)
val bobCertPath = certFactory.generateCertPath(listOf(bobTxCert.cert, bobRootCert.cert))
val bob = PartyAndCertificate(BOB.name, bobRootKey.public, bobRootCert, bobCertPath)
val certFactory = CertificateFactory.getInstance("X509")
val bobRootKey = Crypto.generateKeyPair()
val bobRoot = getTestPartyAndCertificate(BOB.name, bobRootKey.public)
val bobRootCert = bobRoot.certificate
val bobTxKey = Crypto.generateKeyPair()
val bobTxCert = X509Utilities.createCertificate(CertificateType.IDENTITY, bobRootCert, bobRootKey, BOB.name, bobTxKey.public)
val bobCertPath = certFactory.generateCertPath(listOf(bobTxCert.cert, bobRootCert.cert))
val bob = PartyAndCertificate(BOB.name, bobRootKey.public, bobRootCert, bobCertPath)
// Now we have identities, construct the service and let it know about both
val service = InMemoryIdentityService(setOf(alice, bob), emptyMap(), trustRoot.certificate.cert)
service.verifyAndRegisterAnonymousIdentity(aliceTxIdentity, alice.party)
// Now we have identities, construct the service and let it know about both
val service = InMemoryIdentityService(setOf(alice, bob), emptyMap(), trustRoot.certificate.cert)
service.verifyAndRegisterAnonymousIdentity(aliceTxIdentity, alice.party)
val anonymousBob = AnonymousPartyAndPath(AnonymousParty(bobTxKey.public),bobCertPath)
service.verifyAndRegisterAnonymousIdentity(anonymousBob, bob.party)
val anonymousBob = AnonymousPartyAndPath(AnonymousParty(bobTxKey.public),bobCertPath)
service.verifyAndRegisterAnonymousIdentity(anonymousBob, bob.party)
// Verify that paths are verified
service.assertOwnership(alice.party, aliceTxIdentity.party)
service.assertOwnership(bob.party, anonymousBob.party)
assertFailsWith<IllegalArgumentException> {
service.assertOwnership(alice.party, anonymousBob.party)
}
assertFailsWith<IllegalArgumentException> {
service.assertOwnership(bob.party, aliceTxIdentity.party)
}
// Verify that paths are verified
service.assertOwnership(alice.party, aliceTxIdentity.party)
service.assertOwnership(bob.party, anonymousBob.party)
assertFailsWith<IllegalArgumentException> {
service.assertOwnership(alice.party, anonymousBob.party)
}
assertFailsWith<IllegalArgumentException> {
service.assertOwnership(bob.party, aliceTxIdentity.party)
}
assertFailsWith<IllegalArgumentException> {
val owningKey = Crypto.decodePublicKey(trustRoot.certificate.subjectPublicKeyInfo.encoded)
service.assertOwnership(Party(trustRoot.certificate.subject, owningKey), aliceTxIdentity.party)
assertFailsWith<IllegalArgumentException> {
val owningKey = Crypto.decodePublicKey(trustRoot.certificate.subjectPublicKeyInfo.encoded)
service.assertOwnership(Party(trustRoot.certificate.subject, owningKey), aliceTxIdentity.party)
}
}
}

View File

@ -7,12 +7,18 @@ import net.corda.testing.ALICE
import net.corda.testing.BOB
import net.corda.testing.node.MockNetwork
import org.junit.After
import org.junit.Before
import org.junit.Test
import java.math.BigInteger
import kotlin.test.assertEquals
class InMemoryNetworkMapCacheTest {
private val mockNet = MockNetwork()
lateinit var mockNet: MockNetwork
@Before
fun setUp() {
mockNet = MockNetwork()
}
@After
fun teardown() {

View File

@ -2,12 +2,13 @@ package net.corda.node.services.persistence
import com.google.common.primitives.Ints
import net.corda.core.serialization.SerializedBytes
import net.corda.testing.LogHelper
import net.corda.node.services.api.Checkpoint
import net.corda.node.services.api.CheckpointStorage
import net.corda.node.services.transactions.PersistentUniquenessProvider
import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.testing.LogHelper
import net.corda.testing.TestDependencyInjectionBase
import net.corda.testing.node.makeTestDataSourceProperties
import org.assertj.core.api.Assertions.assertThat
import org.assertj.core.api.Assertions.assertThatExceptionOfType
@ -24,7 +25,7 @@ internal fun CheckpointStorage.checkpoints(): List<Checkpoint> {
return checkpoints
}
class DBCheckpointStorageTests {
class DBCheckpointStorageTests : TestDependencyInjectionBase() {
lateinit var checkpointStorage: DBCheckpointStorage
lateinit var database: CordaPersistence

View File

@ -8,11 +8,12 @@ import net.corda.core.crypto.testing.NullPublicKey
import net.corda.core.toFuture
import net.corda.core.transactions.SignedTransaction
import net.corda.core.transactions.WireTransaction
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.LogHelper
import net.corda.node.services.transactions.PersistentUniquenessProvider
import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.LogHelper
import net.corda.testing.TestDependencyInjectionBase
import net.corda.testing.node.makeTestDataSourceProperties
import org.assertj.core.api.Assertions.assertThat
import org.junit.After
@ -21,7 +22,7 @@ import org.junit.Test
import java.util.concurrent.TimeUnit
import kotlin.test.assertEquals
class DBTransactionStorageTests {
class DBTransactionStorageTests : TestDependencyInjectionBase() {
lateinit var database: CordaPersistence
lateinit var transactionStorage: DBTransactionStorage

View File

@ -8,10 +8,11 @@ import io.atomix.copycat.server.storage.Storage
import io.atomix.copycat.server.storage.StorageLevel
import net.corda.core.getOrThrow
import net.corda.core.utilities.NetworkHostAndPort
import net.corda.testing.LogHelper
import net.corda.node.services.network.NetworkMapService
import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.testing.LogHelper
import net.corda.testing.TestDependencyInjectionBase
import net.corda.testing.freeLocalHostAndPort
import net.corda.testing.node.makeTestDataSourceProperties
import org.jetbrains.exposed.sql.Transaction
@ -22,7 +23,7 @@ import java.util.concurrent.CompletableFuture
import kotlin.test.assertEquals
import kotlin.test.assertTrue
class DistributedImmutableMapTests {
class DistributedImmutableMapTests : TestDependencyInjectionBase() {
data class Member(val client: CopycatClient, val server: CopycatServer)
lateinit var cluster: List<Member>

View File

@ -6,6 +6,7 @@ import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.testing.LogHelper
import net.corda.testing.MEGA_CORP
import net.corda.testing.TestDependencyInjectionBase
import net.corda.testing.generateStateRef
import net.corda.testing.node.makeTestDataSourceProperties
import org.junit.After
@ -14,7 +15,7 @@ import org.junit.Test
import kotlin.test.assertEquals
import kotlin.test.assertFailsWith
class PersistentUniquenessProviderTests {
class PersistentUniquenessProviderTests : TestDependencyInjectionBase() {
val identity = MEGA_CORP
val txID = SecureHash.randomSHA256()

View File

@ -34,7 +34,7 @@ import kotlin.test.assertFalse
import kotlin.test.assertNull
import kotlin.test.assertTrue
class NodeVaultServiceTest {
class NodeVaultServiceTest : TestDependencyInjectionBase() {
lateinit var services: MockServices
val vaultSvc: VaultService get() = services.vaultService
lateinit var database: CordaPersistence

View File

@ -15,8 +15,8 @@ import net.corda.core.node.services.vault.*
import net.corda.core.node.services.vault.QueryCriteria.*
import net.corda.core.utilities.seconds
import net.corda.core.transactions.SignedTransaction
import net.corda.core.utilities.OpaqueBytes
import net.corda.core.utilities.NonEmptySet
import net.corda.core.utilities.OpaqueBytes
import net.corda.core.utilities.toHexString
import net.corda.node.services.database.HibernateConfiguration
import net.corda.node.services.schema.NodeSchemaService
@ -46,7 +46,7 @@ import java.time.ZoneOffset
import java.time.temporal.ChronoUnit
import java.util.*
class VaultQueryTests {
class VaultQueryTests : TestDependencyInjectionBase() {
lateinit var services: MockServices
val vaultSvc: VaultService get() = services.vaultService

View File

@ -1,13 +1,8 @@
package net.corda.node.services.vault
import net.corda.testing.contracts.DummyDealContract
import net.corda.contracts.asset.Cash
import net.corda.contracts.asset.DUMMY_CASH_ISSUER
import net.corda.testing.contracts.fillWithSomeTestCash
import net.corda.testing.contracts.fillWithSomeTestDeals
import net.corda.testing.contracts.fillWithSomeTestLinearStates
import net.corda.core.contracts.*
import net.corda.testing.contracts.DummyLinearContract
import net.corda.core.identity.AnonymousParty
import net.corda.core.node.services.VaultService
import net.corda.core.node.services.consumedStates
@ -15,12 +10,8 @@ import net.corda.core.node.services.unconsumedStates
import net.corda.core.transactions.SignedTransaction
import net.corda.node.utilities.CordaPersistence
import net.corda.node.utilities.configureDatabase
import net.corda.testing.BOB
import net.corda.testing.DUMMY_NOTARY
import net.corda.testing.DUMMY_NOTARY_KEY
import net.corda.testing.LogHelper
import net.corda.testing.MEGA_CORP
import net.corda.testing.MEGA_CORP_KEY
import net.corda.testing.*
import net.corda.testing.contracts.*
import net.corda.testing.node.MockServices
import net.corda.testing.node.makeTestDataSourceProperties
import org.assertj.core.api.Assertions.assertThat
@ -36,7 +27,7 @@ import kotlin.test.assertNull
// TODO: Move this to the cash contract tests once mock services are further split up.
class VaultWithCashTest {
class VaultWithCashTest : TestDependencyInjectionBase() {
lateinit var services: MockServices
val vault: VaultService get() = services.vaultService
lateinit var database: CordaPersistence