ENT-10284 Performance optimise deserialisation (#7425)

This commit is contained in:
Rick Parker 2023-07-20 09:51:35 +01:00 committed by GitHub
parent bcf4c11420
commit 48213b5f8c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 288 additions and 96 deletions

View File

@ -84,6 +84,7 @@ def patchCore = tasks.register('patchCore', Zip) {
exclude 'net/corda/core/internal/utilities/PrivateInterner*.class' exclude 'net/corda/core/internal/utilities/PrivateInterner*.class'
exclude 'net/corda/core/crypto/internal/PublicKeyCache*.class' exclude 'net/corda/core/crypto/internal/PublicKeyCache*.class'
exclude 'net/corda/core/internal/ContractStateClassCache*.class' exclude 'net/corda/core/internal/ContractStateClassCache*.class'
exclude 'net/corda/core/internal/LazyPool*.class'
} }
reproducibleFileOrder = true reproducibleFileOrder = true

View File

@ -0,0 +1,41 @@
package net.corda.core.internal
import net.corda.core.KeepForDJVM
/**
* A lazy pool of resources [A], modified for DJVM.
*
* @param clear If specified this function will be run on each borrowed instance before handing it over.
* @param shouldReturnToPool If specified this function will be run on each release to determine whether the instance
* should be returned to the pool for reuse. This may be useful for pooled resources that dynamically grow during
* usage, and we may not want to retain them forever.
* @param bound If specified the pool will be bounded. Once all instances are borrowed subsequent borrows will block until an
* instance is released.
* @param newInstance The function to call to lazily newInstance a pooled resource.
*/
@Suppress("unused")
@KeepForDJVM
class LazyPool<A>(
private val clear: ((A) -> Unit)? = null,
private val shouldReturnToPool: ((A) -> Boolean)? = null,
private val bound: Int? = null,
private val newInstance: () -> A
) {
fun borrow(): A {
return newInstance()
}
@Suppress("unused_parameter")
fun release(instance: A) {
}
/**
* Closes the pool. Note that all borrowed instances must have been released before calling this function, otherwise
* the returned iterable will be inaccurate.
*/
fun close(): Iterable<A> {
return emptyList()
}
fun <R> reentrantRun(withInstance: (A) -> R): R = withInstance(borrow())
}

View File

@ -1,6 +1,5 @@
package net.corda.core.internal package net.corda.core.internal
import net.corda.core.DeleteForDJVM
import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.ConcurrentLinkedQueue
import java.util.concurrent.Semaphore import java.util.concurrent.Semaphore
@ -15,7 +14,6 @@ import java.util.concurrent.Semaphore
* instance is released. * instance is released.
* @param newInstance The function to call to lazily newInstance a pooled resource. * @param newInstance The function to call to lazily newInstance a pooled resource.
*/ */
@DeleteForDJVM
class LazyPool<A>( class LazyPool<A>(
private val clear: ((A) -> Unit)? = null, private val clear: ((A) -> Unit)? = null,
private val shouldReturnToPool: ((A) -> Boolean)? = null, private val shouldReturnToPool: ((A) -> Boolean)? = null,
@ -76,4 +74,18 @@ class LazyPool<A>(
release(instance) release(instance)
} }
} }
private val currentBorrowed = ThreadLocal<A>()
fun <R> reentrantRun(withInstance: (A) -> R): R {
return currentBorrowed.get()?.let {
withInstance(it)
} ?: run {
currentBorrowed.set(it)
try {
withInstance(it)
} finally {
currentBorrowed.set(null)
}
}
}
} }

View File

@ -57,17 +57,50 @@ class CompatibilityTest {
assertTrue(inByteArray.contentEquals(serializedBytes.bytes)) assertTrue(inByteArray.contentEquals(serializedBytes.bytes))
} }
@Test(timeout = 300_000)
fun performanceTest() {
val inputStream = javaClass.classLoader.getResourceAsStream("compatibilityData/v3/node_transaction.dat")
assertNotNull(inputStream)
val inByteArray: ByteArray = inputStream.readBytes()
val deserializationInput = DeserializationInput(serializerFactory)
val bytes = SerializedBytes<SignedTransaction>(inByteArray)
val transaction = deserializationInput.deserialize(bytes, SignedTransaction::class.java, SerializationDefaults.STORAGE_CONTEXT)
assertNotNull(transaction)
val counts = 1000
val loops = 200
for (loop in 0 until loops) {
val start = System.nanoTime()
for (count in 0 until counts) {
val stx = deserializationInput.deserialize(bytes, SignedTransaction::class.java, SerializationDefaults.STORAGE_CONTEXT)
for (input in stx.inputs) {
assertNotNull(input)
}
for (output in stx.tx.outputs) {
assertNotNull(output)
}
for (command in stx.tx.commands) {
assertNotNull(command)
}
}
val end = System.nanoTime()
println("Time per transaction deserialize on loop $loop = ${(end - start) / counts} nanoseconds")
}
}
private fun assertSchemasMatch(original: Schema, reserialized: Schema) { private fun assertSchemasMatch(original: Schema, reserialized: Schema) {
if (original.toString() == reserialized.toString()) return if (original.toString() == reserialized.toString()) return
original.types.forEach { originalType -> original.types.forEach { originalType ->
val reserializedType = reserialized.types.firstOrNull { it.name == originalType.name } ?: val reserializedType = reserialized.types.firstOrNull { it.name == originalType.name }
fail("""Schema mismatch between original and re-serialized data. Could not find reserialized schema matching: ?: fail("""Schema mismatch between original and re-serialized data. Could not find reserialized schema matching:
$originalType $originalType
""") """)
if (originalType.toString() != reserializedType.toString()) if (originalType.toString() != reserializedType.toString())
fail("""Schema mismatch between original and re-serialized data. Expected: fail("""Schema mismatch between original and re-serialized data. Expected:
$originalType $originalType

View File

@ -7,7 +7,13 @@ import com.nhaarman.mockito_kotlin.whenever
import net.corda.client.rpc.RPCException import net.corda.client.rpc.RPCException
import net.corda.core.CordaException import net.corda.core.CordaException
import net.corda.core.CordaRuntimeException import net.corda.core.CordaRuntimeException
import net.corda.core.contracts.* import net.corda.core.contracts.Amount
import net.corda.core.contracts.Contract
import net.corda.core.contracts.ContractAttachment
import net.corda.core.contracts.ContractState
import net.corda.core.contracts.PrivacySalt
import net.corda.core.contracts.StateRef
import net.corda.core.contracts.TransactionState
import net.corda.core.crypto.Crypto import net.corda.core.crypto.Crypto
import net.corda.core.crypto.SecureHash import net.corda.core.crypto.SecureHash
import net.corda.core.crypto.secureRandomBytes import net.corda.core.crypto.secureRandomBytes
@ -15,24 +21,43 @@ import net.corda.core.flows.FlowException
import net.corda.core.identity.AbstractParty import net.corda.core.identity.AbstractParty
import net.corda.core.identity.CordaX500Name import net.corda.core.identity.CordaX500Name
import net.corda.core.internal.AbstractAttachment import net.corda.core.internal.AbstractAttachment
import net.corda.core.serialization.* import net.corda.core.serialization.ConstructorForDeserialization
import net.corda.core.serialization.CordaSerializable
import net.corda.core.serialization.EncodingWhitelist
import net.corda.core.serialization.MissingAttachmentsException
import net.corda.core.serialization.SerializationContext
import net.corda.core.serialization.SerializationFactory
import net.corda.core.transactions.LedgerTransaction import net.corda.core.transactions.LedgerTransaction
import net.corda.core.utilities.OpaqueBytes import net.corda.core.utilities.OpaqueBytes
import net.corda.nodeapi.internal.serialization.amqp.AMQPServerSerializationScheme import net.corda.coretesting.internal.rigorousMock
import net.corda.nodeapi.internal.crypto.ContentSignerBuilder import net.corda.nodeapi.internal.crypto.ContentSignerBuilder
import net.corda.nodeapi.internal.serialization.amqp.AMQPServerSerializationScheme
import net.corda.serialization.internal.* import net.corda.serialization.internal.*
import net.corda.serialization.internal.amqp.testutils.* import net.corda.serialization.internal.amqp.testutils.deserialize
import net.corda.serialization.internal.amqp.testutils.serialize
import net.corda.serialization.internal.amqp.testutils.testDefaultFactory
import net.corda.serialization.internal.amqp.testutils.testDefaultFactoryNoEvolution
import net.corda.serialization.internal.amqp.testutils.testSerializationContext
import net.corda.serialization.internal.carpenter.ClassCarpenterImpl import net.corda.serialization.internal.carpenter.ClassCarpenterImpl
import net.corda.testing.contracts.DummyContract import net.corda.testing.contracts.DummyContract
import net.corda.testing.core.BOB_NAME import net.corda.testing.core.BOB_NAME
import net.corda.testing.core.SerializationEnvironmentRule import net.corda.testing.core.SerializationEnvironmentRule
import net.corda.testing.core.TestIdentity import net.corda.testing.core.TestIdentity
import net.corda.coretesting.internal.rigorousMock
import org.apache.activemq.artemis.api.core.SimpleString import org.apache.activemq.artemis.api.core.SimpleString
import org.apache.qpid.proton.amqp.* import org.apache.qpid.proton.amqp.Decimal128
import org.apache.qpid.proton.amqp.Decimal32
import org.apache.qpid.proton.amqp.Decimal64
import org.apache.qpid.proton.amqp.Symbol
import org.apache.qpid.proton.amqp.UnsignedByte
import org.apache.qpid.proton.amqp.UnsignedInteger
import org.apache.qpid.proton.amqp.UnsignedLong
import org.apache.qpid.proton.amqp.UnsignedShort
import org.apache.qpid.proton.codec.DecoderImpl import org.apache.qpid.proton.codec.DecoderImpl
import org.apache.qpid.proton.codec.EncoderImpl import org.apache.qpid.proton.codec.EncoderImpl
import org.assertj.core.api.Assertions.* import org.assertj.core.api.Assertions.assertThat
import org.assertj.core.api.Assertions.assertThatExceptionOfType
import org.assertj.core.api.Assertions.assertThatThrownBy
import org.assertj.core.api.Assertions.catchThrowable
import org.bouncycastle.asn1.x500.X500Name import org.bouncycastle.asn1.x500.X500Name
import org.bouncycastle.cert.X509v2CRLBuilder import org.bouncycastle.cert.X509v2CRLBuilder
import org.bouncycastle.cert.jcajce.JcaX509CRLConverter import org.bouncycastle.cert.jcajce.JcaX509CRLConverter
@ -49,9 +74,36 @@ import java.io.NotSerializableException
import java.math.BigDecimal import java.math.BigDecimal
import java.math.BigInteger import java.math.BigInteger
import java.security.cert.X509CRL import java.security.cert.X509CRL
import java.time.* import java.time.DayOfWeek
import java.time.Duration
import java.time.Instant
import java.time.LocalDate
import java.time.LocalDateTime
import java.time.LocalTime
import java.time.Month
import java.time.MonthDay
import java.time.OffsetDateTime
import java.time.OffsetTime
import java.time.Period
import java.time.Year
import java.time.YearMonth
import java.time.ZonedDateTime
import java.time.temporal.ChronoUnit import java.time.temporal.ChronoUnit
import java.util.* import java.util.ArrayList
import java.util.Arrays
import java.util.BitSet
import java.util.Currency
import java.util.Date
import java.util.EnumMap
import java.util.EnumSet
import java.util.HashMap
import java.util.NavigableMap
import java.util.Objects
import java.util.Random
import java.util.SortedSet
import java.util.TreeMap
import java.util.TreeSet
import java.util.UUID
import kotlin.reflect.full.superclasses import kotlin.reflect.full.superclasses
import kotlin.test.assertEquals import kotlin.test.assertEquals
import kotlin.test.assertNotNull import kotlin.test.assertNotNull
@ -222,16 +274,16 @@ class SerializationOutputTests(private val compression: CordaSerializationEncodi
val bytes = ser.serialize(obj, compression) val bytes = ser.serialize(obj, compression)
val decoder = DecoderImpl().apply { val decoder = DecoderImpl().apply {
this.register(Envelope.DESCRIPTOR, Envelope) register(Envelope.DESCRIPTOR, Envelope.FastPathConstructor(this))
this.register(Schema.DESCRIPTOR, Schema) register(Schema.DESCRIPTOR, Schema)
this.register(Descriptor.DESCRIPTOR, Descriptor) register(Descriptor.DESCRIPTOR, Descriptor)
this.register(Field.DESCRIPTOR, Field) register(Field.DESCRIPTOR, Field)
this.register(CompositeType.DESCRIPTOR, CompositeType) register(CompositeType.DESCRIPTOR, CompositeType)
this.register(Choice.DESCRIPTOR, Choice) register(Choice.DESCRIPTOR, Choice)
this.register(RestrictedType.DESCRIPTOR, RestrictedType) register(RestrictedType.DESCRIPTOR, RestrictedType)
this.register(ReferencedObject.DESCRIPTOR, ReferencedObject) register(ReferencedObject.DESCRIPTOR, ReferencedObject)
this.register(TransformsSchema.DESCRIPTOR, TransformsSchema) register(TransformsSchema.DESCRIPTOR, TransformsSchema)
this.register(TransformTypes.DESCRIPTOR, TransformTypes) register(TransformTypes.DESCRIPTOR, TransformTypes)
} }
EncoderImpl(decoder) EncoderImpl(decoder)
DeserializationInput.withDataBytes(bytes, encodingWhitelist) { DeserializationInput.withDataBytes(bytes, encodingWhitelist) {

View File

@ -1,6 +1,7 @@
package net.corda.serialization.internal.amqp package net.corda.serialization.internal.amqp
import net.corda.core.KeepForDJVM import net.corda.core.KeepForDJVM
import net.corda.core.internal.LazyPool
import net.corda.core.internal.VisibleForTesting import net.corda.core.internal.VisibleForTesting
import net.corda.core.serialization.AMQP_ENVELOPE_CACHE_PROPERTY import net.corda.core.serialization.AMQP_ENVELOPE_CACHE_PROPERTY
import net.corda.core.serialization.EncodingWhitelist import net.corda.core.serialization.EncodingWhitelist
@ -18,7 +19,8 @@ import net.corda.serialization.internal.model.TypeIdentifier
import org.apache.qpid.proton.amqp.Binary import org.apache.qpid.proton.amqp.Binary
import org.apache.qpid.proton.amqp.DescribedType import org.apache.qpid.proton.amqp.DescribedType
import org.apache.qpid.proton.amqp.UnsignedInteger import org.apache.qpid.proton.amqp.UnsignedInteger
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.DecoderImpl
import org.apache.qpid.proton.codec.EncoderImpl
import java.io.InputStream import java.io.InputStream
import java.io.NotSerializableException import java.io.NotSerializableException
import java.lang.reflect.ParameterizedType import java.lang.reflect.ParameterizedType
@ -72,17 +74,32 @@ class DeserializationInput constructor(
} }
} }
private val decoderPool = LazyPool<DecoderImpl> {
val decoder = DecoderImpl().apply {
register(Envelope.DESCRIPTOR, Envelope.FastPathConstructor(this))
register(Schema.DESCRIPTOR, Schema)
register(Descriptor.DESCRIPTOR, Descriptor)
register(Field.DESCRIPTOR, Field)
register(CompositeType.DESCRIPTOR, CompositeType)
register(Choice.DESCRIPTOR, Choice)
register(RestrictedType.DESCRIPTOR, RestrictedType)
register(ReferencedObject.DESCRIPTOR, ReferencedObject)
register(TransformsSchema.DESCRIPTOR, TransformsSchema)
register(TransformTypes.DESCRIPTOR, TransformTypes)
}
EncoderImpl(decoder)
decoder
}
@Throws(AMQPNoTypeNotSerializableException::class) @Throws(AMQPNoTypeNotSerializableException::class)
fun getEnvelope(byteSequence: ByteSequence, encodingWhitelist: EncodingWhitelist = NullEncodingWhitelist): Envelope { fun getEnvelope(byteSequence: ByteSequence, encodingWhitelist: EncodingWhitelist = NullEncodingWhitelist, lazy: Boolean = false): Envelope {
return withDataBytes(byteSequence, encodingWhitelist) { dataBytes -> return withDataBytes(byteSequence, encodingWhitelist) { dataBytes ->
val data = Data.Factory.create() decoderPool.reentrantRun {
val expectedSize = dataBytes.remaining() it.byteBuffer = dataBytes
if (data.decode(dataBytes) != expectedSize.toLong()) { (it.readObject() as Envelope).apply {
throw AMQPNoTypeNotSerializableException( if (!lazy) this.resolvedSchema
"Unexpected size of data", }
"Blob is corrupted!.")
} }
Envelope.get(data)
} }
} }
} }
@ -124,22 +141,29 @@ class DeserializationInput constructor(
fun <T : Any> deserialize(bytes: ByteSequence, clazz: Class<T>, context: SerializationContext): T = fun <T : Any> deserialize(bytes: ByteSequence, clazz: Class<T>, context: SerializationContext): T =
des { des {
/** /**
* The cache uses object identity rather than [ByteSequence.equals] and * So that the [DecoderImpl] is held whilst we get the [Envelope] and [doReadObject],
* [ByteSequence.hashCode]. This is for speed: each [ByteSequence] object * since we are using lazy when getting the [Envelope] which means [Schema] and
* can potentially be large, and we are optimizing for the case when we * [TransformsSchema] are only parsed out of the [bytes] if demanded by [doReadObject].
* know we will be deserializing the exact same objects multiple times.
* This also means that the cache MUST be short-lived, as otherwise it
* becomes a memory leak.
*/ */
@Suppress("unchecked_cast") decoderPool.reentrantRun {
val envelope = (context.properties[AMQP_ENVELOPE_CACHE_PROPERTY] as? MutableMap<IdentityKey, Envelope>) /**
?.computeIfAbsent(IdentityKey(bytes)) { key -> * The cache uses object identity rather than [ByteSequence.equals] and
getEnvelope(key.bytes, context.encodingWhitelist) * [ByteSequence.hashCode]. This is for speed: each [ByteSequence] object
} ?: getEnvelope(bytes, context.encodingWhitelist) * can potentially be large, and we are optimizing for the case when we
* know we will be deserializing the exact same objects multiple times.
* This also means that the cache MUST be short-lived, as otherwise it
* becomes a memory leak.
*/
@Suppress("unchecked_cast")
val envelope = (context.properties[AMQP_ENVELOPE_CACHE_PROPERTY] as? MutableMap<IdentityKey, Envelope>)
?.computeIfAbsent(IdentityKey(bytes)) { key ->
getEnvelope(key.bytes, context.encodingWhitelist, true)
} ?: getEnvelope(bytes, context.encodingWhitelist, true)
logger.trace { "deserialize blob scheme=\"${envelope.schema}\"" } logger.trace { "deserialize blob scheme=\"${envelope.schema}\"" }
doReadObject(envelope, clazz, context) doReadObject(envelope, clazz, context)
}
} }
@Throws(NotSerializableException::class) @Throws(NotSerializableException::class)
@ -155,10 +179,10 @@ class DeserializationInput constructor(
private fun <T: Any> doReadObject(envelope: Envelope, clazz: Class<T>, context: SerializationContext): T { private fun <T: Any> doReadObject(envelope: Envelope, clazz: Class<T>, context: SerializationContext): T {
return clazz.cast(readObjectOrNull( return clazz.cast(readObjectOrNull(
obj = redescribe(envelope.obj, clazz), obj = redescribe(envelope.obj, clazz),
schema = SerializationSchemas(envelope.schema, envelope.transformsSchema), schema = SerializationSchemas(envelope::resolvedSchema),
type = clazz, type = clazz,
context = context context = context
)) ))
} }

View File

@ -1,70 +1,85 @@
package net.corda.serialization.internal.amqp package net.corda.serialization.internal.amqp
import net.corda.core.KeepForDJVM import net.corda.core.KeepForDJVM
import org.apache.qpid.proton.ProtonException
import org.apache.qpid.proton.amqp.DescribedType import org.apache.qpid.proton.amqp.DescribedType
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import org.apache.qpid.proton.codec.DescribedTypeConstructor import org.apache.qpid.proton.codec.DecoderImpl
import org.apache.qpid.proton.codec.EncodingCodes
import org.apache.qpid.proton.codec.FastPathDescribedTypeConstructor
import java.nio.Buffer
import java.nio.ByteBuffer
/** /**
* This class wraps all serialized data, so that the schema can be carried along with it. We will provide various * This class wraps all serialized data, so that the schema can be carried along with it. We will provide various
* internal utilities to decompose and recompose with/without schema etc so that e.g. we can store objects with a * internal utilities to decompose and recompose with/without schema etc so that e.g. we can store objects with a
* (relationally) normalised out schema to avoid excessive duplication. * (relationally) normalised out schema to avoid excessive duplication.
*/ */
// TODO: make the schema parsing lazy since mostly schemas will have been seen before and we only need it if we
// TODO: don't recognise a type descriptor.
@KeepForDJVM @KeepForDJVM
data class Envelope(val obj: Any?, val schema: Schema, val transformsSchema: TransformsSchema) : DescribedType { class Envelope(val obj: Any?, resolveSchema: () -> Pair<Schema, TransformsSchema>) : DescribedType {
companion object : DescribedTypeConstructor<Envelope> {
val resolvedSchema: Pair<Schema, TransformsSchema> by lazy(resolveSchema)
val schema: Schema get() = resolvedSchema.first
val transformsSchema: TransformsSchema get() = resolvedSchema.second
companion object {
val DESCRIPTOR = AMQPDescriptorRegistry.ENVELOPE.amqpDescriptor val DESCRIPTOR = AMQPDescriptorRegistry.ENVELOPE.amqpDescriptor
val DESCRIPTOR_OBJECT = Descriptor(null, DESCRIPTOR) val DESCRIPTOR_OBJECT = Descriptor(null, DESCRIPTOR)
// described list should either be two or three elements long // described list should either be two or three elements long
private const val ENVELOPE_WITHOUT_TRANSFORMS = 2 private const val ENVELOPE_WITHOUT_TRANSFORMS = 2
private const val ENVELOPE_WITH_TRANSFORMS = 3 private const val ENVELOPE_WITH_TRANSFORMS = 3
}
private const val BLOB_IDX = 0 class FastPathConstructor(private val decoder: DecoderImpl) : FastPathDescribedTypeConstructor<Envelope> {
private const val SCHEMA_IDX = 1
private const val TRANSFORMS_SCHEMA_IDX = 2
fun get(data: Data): Envelope { private val _buffer: ByteBuffer get() = decoder.byteBuffer
val describedType = data.`object` as DescribedType
if (describedType.descriptor != DESCRIPTOR) { @Suppress("ComplexMethod", "MagicNumber")
throw AMQPNoTypeNotSerializableException( private fun readEncodingAndReturnSize(buffer: ByteBuffer, inBytes: Boolean = true): Int {
"Unexpected descriptor ${describedType.descriptor}, should be $DESCRIPTOR.") val encodingCode: Byte = buffer.get()
return when (encodingCode) {
EncodingCodes.LIST8 -> {
(buffer.get().toInt() and 0xff).let { if (inBytes) it else (buffer.get().toInt() and 0xff) }
}
EncodingCodes.LIST32 -> {
buffer.int.let { if (inBytes) it else buffer.int }
}
else -> throw ProtonException("Expected List type but found encoding: $encodingCode")
} }
val list = describedType.described as List<*>
// We need to cope with objects serialised without the transforms header element in the
// envelope
val transformSchema: Any? = when (list.size) {
ENVELOPE_WITHOUT_TRANSFORMS -> null
ENVELOPE_WITH_TRANSFORMS -> list[TRANSFORMS_SCHEMA_IDX]
else -> throw AMQPNoTypeNotSerializableException(
"Malformed list, bad length of ${list.size} (should be 2 or 3)")
}
return newInstance(listOf(list[BLOB_IDX], Schema.get(list[SCHEMA_IDX]!!),
TransformsSchema.newInstance(transformSchema)))
} }
// This separation of functions is needed as this will be the entry point for the default override fun readValue(): Envelope? {
// AMQP decoder if one is used (see the unit tests). val buffer = _buffer
override fun newInstance(described: Any?): Envelope { val size = readEncodingAndReturnSize(buffer, false)
val list = described as? List<*> ?: throw IllegalStateException("Was expecting a list") if (size != ENVELOPE_WITHOUT_TRANSFORMS && size != ENVELOPE_WITH_TRANSFORMS) {
throw AMQPNoTypeNotSerializableException("Malformed list, bad length of $size (should be 2 or 3)")
// We need to cope with objects serialised without the transforms header element in the
// envelope
val transformSchema = when (list.size) {
ENVELOPE_WITHOUT_TRANSFORMS -> TransformsSchema.newInstance(null)
ENVELOPE_WITH_TRANSFORMS -> list[TRANSFORMS_SCHEMA_IDX] as TransformsSchema
else -> throw AMQPNoTypeNotSerializableException(
"Malformed list, bad length of ${list.size} (should be 2 or 3)")
} }
val data = Data.Factory.create()
return Envelope(list[BLOB_IDX], list[SCHEMA_IDX] as Schema, transformSchema) data.decode(buffer)
val obj = data.`object`
val lambda: () -> Pair<Schema, TransformsSchema> = {
data.decode(buffer)
val schema = data.`object`
val transformsSchema = if (size > 2) {
data.decode(buffer)
data.`object`
} else null
Schema.get(schema) to TransformsSchema.newInstance(transformsSchema)
}
return Envelope(obj, lambda)
} }
override fun getTypeClass(): Class<*> = Envelope::class.java override fun skipValue() {
val buffer = _buffer
val size = readEncodingAndReturnSize(buffer)
(buffer as Buffer).position(buffer.position() + size)
}
override fun encodesJavaPrimitive(): Boolean = false
override fun getTypeClass(): Class<Envelope> = Envelope::class.java
} }
override fun getDescriptor(): Any = DESCRIPTOR override fun getDescriptor(): Any = DESCRIPTOR

View File

@ -5,9 +5,13 @@ import net.corda.core.internal.uncheckedCast
import net.corda.serialization.internal.CordaSerializationMagic import net.corda.serialization.internal.CordaSerializationMagic
import net.corda.serialization.internal.amqp.AMQPTypeIdentifiers.isPrimitive import net.corda.serialization.internal.amqp.AMQPTypeIdentifiers.isPrimitive
import net.corda.serialization.internal.model.TypeIdentifier import net.corda.serialization.internal.model.TypeIdentifier
import net.corda.serialization.internal.model.TypeIdentifier.TopType
import net.corda.serialization.internal.model.TypeIdentifier.Companion.forGenericType import net.corda.serialization.internal.model.TypeIdentifier.Companion.forGenericType
import org.apache.qpid.proton.amqp.* import net.corda.serialization.internal.model.TypeIdentifier.TopType
import org.apache.qpid.proton.amqp.Binary
import org.apache.qpid.proton.amqp.DescribedType
import org.apache.qpid.proton.amqp.Symbol
import org.apache.qpid.proton.amqp.UnsignedInteger
import org.apache.qpid.proton.amqp.UnsignedLong
import org.apache.qpid.proton.codec.DescribedTypeConstructor import org.apache.qpid.proton.codec.DescribedTypeConstructor
import java.io.NotSerializableException import java.io.NotSerializableException
import java.lang.reflect.Type import java.lang.reflect.Type

View File

@ -5,7 +5,17 @@ import java.io.NotSerializableException
import javax.annotation.concurrent.ThreadSafe import javax.annotation.concurrent.ThreadSafe
@KeepForDJVM @KeepForDJVM
data class SerializationSchemas(val schema: Schema, val transforms: TransformsSchema) class SerializationSchemas(resolveSchema: () -> Pair<Schema, TransformsSchema>) {
constructor(schema: Schema, transforms: TransformsSchema) : this({ schema to transforms })
private val resolvedSchema: Pair<Schema, TransformsSchema> by lazy(resolveSchema)
val schema: Schema get() = resolvedSchema.first
val transforms: TransformsSchema get() = resolvedSchema.second
operator fun component1(): Schema = schema
operator fun component2(): TransformsSchema = transforms
}
/** /**
* Factory of serializers designed to be shared across threads and invocations. * Factory of serializers designed to be shared across threads and invocations.

View File

@ -183,7 +183,7 @@ public class JavaSerializationOutputTests {
DecoderImpl decoder = new DecoderImpl(); DecoderImpl decoder = new DecoderImpl();
decoder.register(Envelope.Companion.getDESCRIPTOR(), Envelope.Companion); decoder.register(Envelope.Companion.getDESCRIPTOR(), new Envelope.FastPathConstructor(decoder));
decoder.register(Schema.Companion.getDESCRIPTOR(), Schema.Companion); decoder.register(Schema.Companion.getDESCRIPTOR(), Schema.Companion);
decoder.register(Descriptor.Companion.getDESCRIPTOR(), Descriptor.Companion); decoder.register(Descriptor.Companion.getDESCRIPTOR(), Descriptor.Companion);
decoder.register(Field.Companion.getDESCRIPTOR(), Field.Companion); decoder.register(Field.Companion.getDESCRIPTOR(), Field.Companion);