AMQP serialization part 3: some custom serializers, integration with Kryo (disabled) (#859)

This commit is contained in:
Rick Parker 2017-06-29 14:03:41 +01:00 committed by GitHub
parent b791530b28
commit 5a45459b9d
21 changed files with 854 additions and 238 deletions

View File

@ -29,7 +29,11 @@ fun makeAllButBlacklistedClassResolver(): ClassResolver {
return CordaClassResolver(AllButBlacklisted) return CordaClassResolver(AllButBlacklisted)
} }
class CordaClassResolver(val whitelist: ClassWhitelist) : DefaultClassResolver() { /**
* @param amqpEnabled Setting this to true turns on experimental AMQP serialization for any class annotated with
* [CordaSerializable].
*/
class CordaClassResolver(val whitelist: ClassWhitelist, val amqpEnabled: Boolean = false) : DefaultClassResolver() {
/** Returns the registration for the specified class, or null if the class is not registered. */ /** Returns the registration for the specified class, or null if the class is not registered. */
override fun getRegistration(type: Class<*>): Registration? { override fun getRegistration(type: Class<*>): Registration? {
return super.getRegistration(type) ?: checkClass(type) return super.getRegistration(type) ?: checkClass(type)
@ -59,7 +63,7 @@ class CordaClassResolver(val whitelist: ClassWhitelist) : DefaultClassResolver()
return checkClass(type.superclass) return checkClass(type.superclass)
} }
// It's safe to have the Class already, since Kryo loads it with initialisation off. // It's safe to have the Class already, since Kryo loads it with initialisation off.
// If we use a whitelist with blacklisting capabilities, whitelist.hasListed(type) may throw a NotSerializableException if input class is blacklisted. // If we use a whitelist with blacklisting capabilities, whitelist.hasListed(type) may throw an IllegalStateException if input class is blacklisted.
// Thus, blacklisting precedes annotation checking. // Thus, blacklisting precedes annotation checking.
if (!whitelist.hasListed(type) && !checkForAnnotation(type)) { if (!whitelist.hasListed(type) && !checkForAnnotation(type)) {
throw KryoException("Class ${Util.className(type)} is not annotated or on the whitelist, so cannot be used in serialization") throw KryoException("Class ${Util.className(type)} is not annotated or on the whitelist, so cannot be used in serialization")
@ -68,6 +72,11 @@ class CordaClassResolver(val whitelist: ClassWhitelist) : DefaultClassResolver()
} }
override fun registerImplicit(type: Class<*>): Registration { override fun registerImplicit(type: Class<*>): Registration {
val hasAnnotation = checkForAnnotation(type)
// If something is not annotated, or AMQP is disabled, we stay serializing with Kryo. This will typically be the
// case for flow checkpoints (ignoring all cases where AMQP is disabled) since our top level messaging data structures
// are annotated and once we enter AMQP serialisation we stay with it for the entire object subgraph.
if (!hasAnnotation || !amqpEnabled) {
// We have to set reference to true, since the flag influences how String fields are treated and we want it to be consistent. // We have to set reference to true, since the flag influences how String fields are treated and we want it to be consistent.
val references = kryo.references val references = kryo.references
try { try {
@ -76,6 +85,10 @@ class CordaClassResolver(val whitelist: ClassWhitelist) : DefaultClassResolver()
} finally { } finally {
kryo.references = references kryo.references = references
} }
} else {
// Build AMQP serializer
return register(Registration(type, KryoAMQPSerializer, NAME.toInt()))
}
} }
// We don't allow the annotation for classes in attachments for now. The class will be on the main classpath if we have the CorDapp installed. // We don't allow the annotation for classes in attachments for now. The class will be on the main classpath if we have the CorDapp installed.
@ -85,13 +98,13 @@ class CordaClassResolver(val whitelist: ClassWhitelist) : DefaultClassResolver()
return (type.classLoader !is AttachmentsClassLoader) return (type.classLoader !is AttachmentsClassLoader)
&& !KryoSerializable::class.java.isAssignableFrom(type) && !KryoSerializable::class.java.isAssignableFrom(type)
&& !type.isAnnotationPresent(DefaultSerializer::class.java) && !type.isAnnotationPresent(DefaultSerializer::class.java)
&& (type.isAnnotationPresent(CordaSerializable::class.java) || hasAnnotationOnInterface(type)) && (type.isAnnotationPresent(CordaSerializable::class.java) || hasInheritedAnnotation(type))
} }
// Recursively check interfaces for our annotation. // Recursively check interfaces for our annotation.
private fun hasAnnotationOnInterface(type: Class<*>): Boolean { private fun hasInheritedAnnotation(type: Class<*>): Boolean {
return type.interfaces.any { it.isAnnotationPresent(CordaSerializable::class.java) || hasAnnotationOnInterface(it) } return type.interfaces.any { it.isAnnotationPresent(CordaSerializable::class.java) || hasInheritedAnnotation(it) }
|| (type.superclass != null && hasAnnotationOnInterface(type.superclass)) || (type.superclass != null && hasInheritedAnnotation(type.superclass))
} }
// Need to clear out class names from attachments. // Need to clear out class names from attachments.

View File

@ -0,0 +1,51 @@
package net.corda.core.serialization
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.Serializer
import com.esotericsoftware.kryo.io.Input
import com.esotericsoftware.kryo.io.Output
import net.corda.core.serialization.amqp.DeserializationInput
import net.corda.core.serialization.amqp.SerializationOutput
import net.corda.core.serialization.amqp.SerializerFactory
/**
* This [Kryo] custom [Serializer] switches the object graph of anything annotated with `@CordaSerializable`
* to using the AMQP serialization wire format, and simply writes that out as bytes to the wire.
*
* There is no need to write out the length, since this can be peeked out of the first few bytes of the stream.
*/
object KryoAMQPSerializer : Serializer<Any>() {
internal fun registerCustomSerializers(factory: SerializerFactory) {
factory.apply {
register(net.corda.core.serialization.amqp.custom.PublicKeySerializer)
register(net.corda.core.serialization.amqp.custom.ThrowableSerializer(this))
register(net.corda.core.serialization.amqp.custom.X500NameSerializer)
register(net.corda.core.serialization.amqp.custom.BigDecimalSerializer)
register(net.corda.core.serialization.amqp.custom.CurrencySerializer)
register(net.corda.core.serialization.amqp.custom.InstantSerializer(this))
}
}
// TODO: need to sort out the whitelist... we currently do not apply the whitelist attached to the [Kryo]
// instance to the factory. We need to do this before turning on AMQP serialization.
private val serializerFactory = SerializerFactory().apply {
registerCustomSerializers(this)
}
override fun write(kryo: Kryo, output: Output, obj: Any) {
val amqpOutput = SerializationOutput(serializerFactory)
val bytes = amqpOutput.serialize(obj).bytes
// No need to write out the size since it's encoded within the AMQP.
output.write(bytes)
}
override fun read(kryo: Kryo, input: Input, type: Class<Any>): Any {
val amqpInput = DeserializationInput(serializerFactory)
// Use our helper functions to peek the size of the serialized object out of the AMQP byte stream.
val peekedBytes = input.readBytes(DeserializationInput.BYTES_NEEDED_TO_PEEK)
val size = DeserializationInput.peekSize(peekedBytes)
val allBytes = peekedBytes.copyOf(size)
input.readBytes(allBytes, peekedBytes.size, size - peekedBytes.size)
return amqpInput.deserialize(SerializedBytes<Any>(allBytes), type)
}
}

View File

@ -1,14 +1,16 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import com.google.common.primitives.Primitives import org.apache.qpid.proton.amqp.Binary
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.lang.reflect.Type import java.lang.reflect.Type
/** /**
* Serializer / deserializer for native AMQP types (Int, Float, String etc). * Serializer / deserializer for native AMQP types (Int, Float, String etc).
*
* [ByteArray] is automatically marshalled to/from the Proton-J wrapper, [Binary].
*/ */
class AMQPPrimitiveSerializer(clazz: Class<*>) : AMQPSerializer<Any> { class AMQPPrimitiveSerializer(clazz: Class<*>) : AMQPSerializer<Any> {
override val typeDescriptor: String = SerializerFactory.primitiveTypeName(Primitives.wrap(clazz))!! override val typeDescriptor: String = SerializerFactory.primitiveTypeName(clazz)!!
override val type: Type = clazz override val type: Type = clazz
// NOOP since this is a primitive type. // NOOP since this is a primitive type.
@ -16,8 +18,12 @@ class AMQPPrimitiveSerializer(clazz: Class<*>) : AMQPSerializer<Any> {
} }
override fun writeObject(obj: Any, data: Data, type: Type, output: SerializationOutput) { override fun writeObject(obj: Any, data: Data, type: Type, output: SerializationOutput) {
if (obj is ByteArray) {
data.putObject(Binary(obj))
} else {
data.putObject(obj) data.putObject(obj)
} }
}
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): Any = obj
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): Any = (obj as? Binary)?.array ?: obj
} }

View File

@ -2,8 +2,6 @@ package net.corda.core.serialization.amqp
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.io.NotSerializableException import java.io.NotSerializableException
import java.lang.reflect.GenericArrayType
import java.lang.reflect.ParameterizedType
import java.lang.reflect.Type import java.lang.reflect.Type
/** /**
@ -12,14 +10,10 @@ import java.lang.reflect.Type
class ArraySerializer(override val type: Type, factory: SerializerFactory) : AMQPSerializer<Any> { class ArraySerializer(override val type: Type, factory: SerializerFactory) : AMQPSerializer<Any> {
override val typeDescriptor = "$DESCRIPTOR_DOMAIN:${fingerprintForType(type, factory)}" override val typeDescriptor = "$DESCRIPTOR_DOMAIN:${fingerprintForType(type, factory)}"
internal val elementType: Type = makeElementType() internal val elementType: Type = type.componentType()
private val typeNotation: TypeNotation = RestrictedType(type.typeName, null, emptyList(), "list", Descriptor(typeDescriptor, null), emptyList()) private val typeNotation: TypeNotation = RestrictedType(type.typeName, null, emptyList(), "list", Descriptor(typeDescriptor, null), emptyList())
private fun makeElementType(): Type {
return (type as? Class<*>)?.componentType ?: (type as GenericArrayType).genericComponentType
}
override fun writeClassInfo(output: SerializationOutput) { override fun writeClassInfo(output: SerializationOutput) {
if (output.writeTypeNotations(typeNotation)) { if (output.writeTypeNotations(typeNotation)) {
output.requireSerializer(elementType) output.requireSerializer(elementType)
@ -44,13 +38,7 @@ class ArraySerializer(override val type: Type, factory: SerializerFactory) : AMQ
} }
private fun <T> List<T>.toArrayOfType(type: Type): Any { private fun <T> List<T>.toArrayOfType(type: Type): Any {
val elementType: Class<*> = if (type is Class<*>) { val elementType = type.asClass() ?: throw NotSerializableException("Unexpected array element type $type")
type
} else if (type is ParameterizedType) {
type.rawType as Class<*>
} else {
throw NotSerializableException("Unexpected array element type $type")
}
val list = this val list = this
return java.lang.reflect.Array.newInstance(elementType, this.size).apply { return java.lang.reflect.Array.newInstance(elementType, this.size).apply {
val array = this val array = this

View File

@ -32,7 +32,7 @@ class CollectionSerializer(val declaredType: ParameterizedType, factory: Seriali
private val concreteBuilder: (List<*>) -> Collection<*> = findConcreteType(declaredType.rawType as Class<*>) private val concreteBuilder: (List<*>) -> Collection<*> = findConcreteType(declaredType.rawType as Class<*>)
private val typeNotation: TypeNotation = RestrictedType(declaredType.toString(), null, emptyList(), "list", Descriptor(typeDescriptor, null), emptyList()) private val typeNotation: TypeNotation = RestrictedType(SerializerFactory.nameForType(declaredType), null, emptyList(), "list", Descriptor(typeDescriptor, null), emptyList())
override fun writeClassInfo(output: SerializationOutput) { override fun writeClassInfo(output: SerializationOutput) {
if (output.writeTypeNotations(typeNotation)) { if (output.writeTypeNotations(typeNotation)) {

View File

@ -1,5 +1,6 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import net.corda.core.serialization.amqp.SerializerFactory.Companion.nameForType
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.lang.reflect.Type import java.lang.reflect.Type
@ -10,11 +11,16 @@ import java.lang.reflect.Type
abstract class CustomSerializer<T> : AMQPSerializer<T> { abstract class CustomSerializer<T> : AMQPSerializer<T> {
/** /**
* This is a collection of custom serializers that this custom serializer depends on. e.g. for proxy objects * This is a collection of custom serializers that this custom serializer depends on. e.g. for proxy objects
* that refer to arrays of types etc. * that refer to other custom types etc.
*/ */
abstract val additionalSerializers: Iterable<CustomSerializer<out Any>> abstract val additionalSerializers: Iterable<CustomSerializer<out Any>>
/**
* This method should return true if the custom serializer can serialize an instance of the class passed as the
* parameter.
*/
abstract fun isSerializerFor(clazz: Class<*>): Boolean abstract fun isSerializerFor(clazz: Class<*>): Boolean
protected abstract val descriptor: Descriptor protected abstract val descriptor: Descriptor
/** /**
* This exists purely for documentation and cross-platform purposes. It is not used by our serialization / deserialization * This exists purely for documentation and cross-platform purposes. It is not used by our serialization / deserialization
@ -32,12 +38,42 @@ abstract class CustomSerializer<T> : AMQPSerializer<T> {
abstract fun writeDescribedObject(obj: T, data: Data, type: Type, output: SerializationOutput) abstract fun writeDescribedObject(obj: T, data: Data, type: Type, output: SerializationOutput)
/** /**
* Additional base features for a custom serializer that is a particular class. * This custom serializer represents a sort of symbolic link from a subclass to a super class, where the super
* class custom serializer is responsible for the "on the wire" format but we want to create a reference to the
* subclass in the schema, so that we can distinguish between subclasses.
*/
// TODO: should this be a custom serializer at all, or should it just be a plain AMQPSerializer?
class SubClass<T>(protected val clazz: Class<*>, protected val superClassSerializer: CustomSerializer<T>) : CustomSerializer<T>() {
override val additionalSerializers: Iterable<CustomSerializer<out Any>> = emptyList()
// TODO: should this be empty or contain the schema of the super?
override val schemaForDocumentation = Schema(emptyList())
override fun isSerializerFor(clazz: Class<*>): Boolean = clazz == this.clazz
override val type: Type get() = clazz
override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${fingerprintForDescriptors(superClassSerializer.typeDescriptor, nameForType(clazz))}"
private val typeNotation: TypeNotation = RestrictedType(SerializerFactory.nameForType(clazz), null, emptyList(), SerializerFactory.nameForType(superClassSerializer.type), Descriptor(typeDescriptor, null), emptyList())
override fun writeClassInfo(output: SerializationOutput) {
output.writeTypeNotations(typeNotation)
}
override val descriptor: Descriptor = Descriptor(typeDescriptor)
override fun writeDescribedObject(obj: T, data: Data, type: Type, output: SerializationOutput) {
superClassSerializer.writeDescribedObject(obj, data, type, output)
}
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): T {
return superClassSerializer.readObject(obj, schema, input)
}
}
/**
* Additional base features for a custom serializer for a particular class, that excludes subclasses.
*/ */
abstract class Is<T>(protected val clazz: Class<T>) : CustomSerializer<T>() { abstract class Is<T>(protected val clazz: Class<T>) : CustomSerializer<T>() {
override fun isSerializerFor(clazz: Class<*>): Boolean = clazz == this.clazz override fun isSerializerFor(clazz: Class<*>): Boolean = clazz == this.clazz
override val type: Type get() = clazz override val type: Type get() = clazz
override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${clazz.name}" override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${nameForType(clazz)}"
override fun writeClassInfo(output: SerializationOutput) {} override fun writeClassInfo(output: SerializationOutput) {}
override val descriptor: Descriptor = Descriptor(typeDescriptor) override val descriptor: Descriptor = Descriptor(typeDescriptor)
} }
@ -48,13 +84,13 @@ abstract class CustomSerializer<T> : AMQPSerializer<T> {
abstract class Implements<T>(protected val clazz: Class<T>) : CustomSerializer<T>() { abstract class Implements<T>(protected val clazz: Class<T>) : CustomSerializer<T>() {
override fun isSerializerFor(clazz: Class<*>): Boolean = this.clazz.isAssignableFrom(clazz) override fun isSerializerFor(clazz: Class<*>): Boolean = this.clazz.isAssignableFrom(clazz)
override val type: Type get() = clazz override val type: Type get() = clazz
override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${clazz.name}" override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${nameForType(clazz)}"
override fun writeClassInfo(output: SerializationOutput) {} override fun writeClassInfo(output: SerializationOutput) {}
override val descriptor: Descriptor = Descriptor(typeDescriptor) override val descriptor: Descriptor = Descriptor(typeDescriptor)
} }
/** /**
* Addition base features over and above [Implements] or [Is] custom serializer for when the serialize form should be * Additional base features over and above [Implements] or [Is] custom serializer for when the serialized form should be
* the serialized form of a proxy class, and the object can be re-created from that proxy on deserialization. * the serialized form of a proxy class, and the object can be re-created from that proxy on deserialization.
* *
* The proxy class must use only types which are either native AMQP or other types for which there are pre-registered * The proxy class must use only types which are either native AMQP or other types for which there are pre-registered
@ -66,14 +102,14 @@ abstract class CustomSerializer<T> : AMQPSerializer<T> {
val withInheritance: Boolean = true) : CustomSerializer<T>() { val withInheritance: Boolean = true) : CustomSerializer<T>() {
override fun isSerializerFor(clazz: Class<*>): Boolean = if (withInheritance) this.clazz.isAssignableFrom(clazz) else this.clazz == clazz override fun isSerializerFor(clazz: Class<*>): Boolean = if (withInheritance) this.clazz.isAssignableFrom(clazz) else this.clazz == clazz
override val type: Type get() = clazz override val type: Type get() = clazz
override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${clazz.name}" override val typeDescriptor: String = "$DESCRIPTOR_DOMAIN:${nameForType(clazz)}"
override fun writeClassInfo(output: SerializationOutput) {} override fun writeClassInfo(output: SerializationOutput) {}
override val descriptor: Descriptor = Descriptor(typeDescriptor) override val descriptor: Descriptor = Descriptor(typeDescriptor)
private val proxySerializer: ObjectSerializer by lazy { ObjectSerializer(proxyClass, factory) } private val proxySerializer: ObjectSerializer by lazy { ObjectSerializer(proxyClass, factory) }
override val schemaForDocumentation: Schema by lazy { override val schemaForDocumentation: Schema by lazy {
val typeNotations = mutableSetOf<TypeNotation>(CompositeType(type.typeName, null, emptyList(), descriptor, (proxySerializer.typeNotation as CompositeType).fields)) val typeNotations = mutableSetOf<TypeNotation>(CompositeType(nameForType(type), null, emptyList(), descriptor, (proxySerializer.typeNotation as CompositeType).fields))
for (additional in additionalSerializers) { for (additional in additionalSerializers) {
typeNotations.addAll(additional.schemaForDocumentation.types) typeNotations.addAll(additional.schemaForDocumentation.types)
} }
@ -102,4 +138,38 @@ abstract class CustomSerializer<T> : AMQPSerializer<T> {
return fromProxy(proxy) return fromProxy(proxy)
} }
} }
/**
* A custom serializer where the on-wire representation is a string. For example, a [Currency] might be represented
* as a 3 character currency code, and converted to and from that string. By default, it is assumed that the
* [toString] method will generate the string representation and that there is a constructor that takes such a
* string as an argument to reconstruct.
*
* @param clazz The type to be marshalled
* @param withInheritance Whether subclasses of the class can also be marshalled.
* @param make A lambda for constructing an instance, that defaults to calling a constructor that expects a string.
* @param unmake A lambda that extracts the string value for an instance, that defaults to the [toString] method.
*/
abstract class ToString<T>(clazz: Class<T>, withInheritance: Boolean = false,
private val maker: (String) -> T = clazz.getConstructor(String::class.java).let { `constructor` -> { string -> `constructor`.newInstance(string) } },
private val unmaker: (T) -> String = { obj -> obj.toString() }) : Proxy<T, String>(clazz, String::class.java, /* Unused */ SerializerFactory(), withInheritance) {
override val additionalSerializers: Iterable<CustomSerializer<out Any>> = emptyList()
override val schemaForDocumentation = Schema(listOf(RestrictedType(nameForType(type), "", listOf(nameForType(type)), SerializerFactory.primitiveTypeName(String::class.java)!!, descriptor, emptyList())))
override fun toProxy(obj: T): String = unmaker(obj)
override fun fromProxy(proxy: String): T = maker(proxy)
override fun writeDescribedObject(obj: T, data: Data, type: Type, output: SerializationOutput) {
val proxy = toProxy(obj)
data.putObject(proxy)
}
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): T {
val proxy = input.readObject(obj, schema, String::class.java) as String
return fromProxy(proxy)
}
}
} }

View File

@ -2,7 +2,9 @@ package net.corda.core.serialization.amqp
import com.google.common.base.Throwables import com.google.common.base.Throwables
import net.corda.core.serialization.SerializedBytes import net.corda.core.serialization.SerializedBytes
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.UnsignedByte
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.io.NotSerializableException import java.io.NotSerializableException
import java.lang.reflect.Type import java.lang.reflect.Type
@ -19,6 +21,41 @@ class DeserializationInput(internal val serializerFactory: SerializerFactory = S
// TODO: we're not supporting object refs yet // TODO: we're not supporting object refs yet
private val objectHistory: MutableList<Any> = ArrayList() private val objectHistory: MutableList<Any> = ArrayList()
internal companion object {
val BYTES_NEEDED_TO_PEEK: Int = 23
private fun subArraysEqual(a: ByteArray, aOffset: Int, length: Int, b: ByteArray, bOffset: Int): Boolean {
if (aOffset + length > a.size || bOffset + length > b.size) throw IndexOutOfBoundsException()
var bytesRemaining = length
var aPos = aOffset
var bPos = bOffset
while (bytesRemaining-- > 0) {
if (a[aPos++] != b[bPos++]) return false
}
return true
}
fun peekSize(bytes: ByteArray): Int {
// There's an 8 byte header, and then a 0 byte plus descriptor followed by constructor
val eighth = bytes[8].toInt()
check(eighth == 0x0) { "Expected to find a descriptor in the AMQP stream" }
// We should always have an Envelope, so the descriptor should be a 64-bit long (0x80)
val ninth = UnsignedByte.valueOf(bytes[9]).toInt()
check(ninth == 0x80) { "Expected to find a ulong in the AMQP stream" }
// Skip 8 bytes
val eighteenth = UnsignedByte.valueOf(bytes[18]).toInt()
check(eighteenth == 0xd0 || eighteenth == 0xc0) { "Expected to find a list8 or list32 in the AMQP stream" }
val size = if (eighteenth == 0xc0) {
// Next byte is size
UnsignedByte.valueOf(bytes[19]).toInt() - 3 // Minus three as PEEK_SIZE assumes 4 byte unsigned integer.
} else {
// Next 4 bytes is size
UnsignedByte.valueOf(bytes[19]).toInt().shl(24) + UnsignedByte.valueOf(bytes[20]).toInt().shl(16) + UnsignedByte.valueOf(bytes[21]).toInt().shl(8) + UnsignedByte.valueOf(bytes[22]).toInt()
}
return size + BYTES_NEEDED_TO_PEEK
}
}
@Throws(NotSerializableException::class) @Throws(NotSerializableException::class)
inline fun <reified T : Any> deserialize(bytes: SerializedBytes<T>): T = deserialize(bytes, T::class.java) inline fun <reified T : Any> deserialize(bytes: SerializedBytes<T>): T = deserialize(bytes, T::class.java)
@ -66,25 +103,10 @@ class DeserializationInput(internal val serializerFactory: SerializerFactory = S
if (serializer.type != type && !serializer.type.isSubClassOf(type)) if (serializer.type != type && !serializer.type.isSubClassOf(type))
throw NotSerializableException("Described type with descriptor ${obj.descriptor} was expected to be of type $type") throw NotSerializableException("Described type with descriptor ${obj.descriptor} was expected to be of type $type")
return serializer.readObject(obj.described, schema, this) return serializer.readObject(obj.described, schema, this)
} else if (obj is Binary) {
return obj.array
} else { } else {
return obj return obj
} }
} }
private fun Type.isSubClassOf(type: Type): Boolean {
return type == Object::class.java ||
(this is Class<*> && type is Class<*> && type.isAssignableFrom(this)) ||
(this is DeserializedParameterizedType && type is Class<*> && this.rawType == type && this.isFullyWildcarded)
}
private fun subArraysEqual(a: ByteArray, aOffset: Int, length: Int, b: ByteArray, bOffset: Int): Boolean {
if (aOffset + length > a.size || bOffset + length > b.size) throw IndexOutOfBoundsException()
var bytesRemaining = length
var aPos = aOffset
var bPos = bOffset
while (bytesRemaining-- > 0) {
if (a[aPos++] != b[bPos++]) return false
}
return true
}
} }

View File

@ -1,5 +1,6 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import com.google.common.primitives.Primitives
import java.io.NotSerializableException import java.io.NotSerializableException
import java.lang.reflect.ParameterizedType import java.lang.reflect.ParameterizedType
import java.lang.reflect.Type import java.lang.reflect.Type
@ -119,7 +120,9 @@ class DeserializedParameterizedType(private val rawType: Class<*>, private val p
private fun makeType(typeName: String, cl: ClassLoader): Type { private fun makeType(typeName: String, cl: ClassLoader): Type {
// Not generic // Not generic
return if (typeName == "?") SerializerFactory.AnyType else Class.forName(typeName, false, cl) return if (typeName == "?") SerializerFactory.AnyType else {
Primitives.wrap(SerializerFactory.primitiveType(typeName) ?: Class.forName(typeName, false, cl))
}
} }
private fun makeParameterizedType(rawTypeName: String, args: MutableList<Type>, cl: ClassLoader): Type { private fun makeParameterizedType(rawTypeName: String, args: MutableList<Type>, cl: ClassLoader): Type {

View File

@ -31,7 +31,7 @@ class MapSerializer(val declaredType: ParameterizedType, factory: SerializerFact
private val concreteBuilder: (Map<*, *>) -> Map<*, *> = findConcreteType(declaredType.rawType as Class<*>) private val concreteBuilder: (Map<*, *>) -> Map<*, *> = findConcreteType(declaredType.rawType as Class<*>)
private val typeNotation: TypeNotation = RestrictedType(declaredType.toString(), null, emptyList(), "map", Descriptor(typeDescriptor, null), emptyList()) private val typeNotation: TypeNotation = RestrictedType(SerializerFactory.nameForType(declaredType), null, emptyList(), "map", Descriptor(typeDescriptor, null), emptyList())
override fun writeClassInfo(output: SerializationOutput) { override fun writeClassInfo(output: SerializationOutput) {
if (output.writeTypeNotations(typeNotation)) { if (output.writeTypeNotations(typeNotation)) {

View File

@ -1,5 +1,6 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import net.corda.core.serialization.amqp.SerializerFactory.Companion.nameForType
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.Data
import java.io.NotSerializableException import java.io.NotSerializableException
@ -10,7 +11,7 @@ import kotlin.reflect.jvm.javaConstructor
/** /**
* Responsible for serializing and deserializing a regular object instance via a series of properties (matched with a constructor). * Responsible for serializing and deserializing a regular object instance via a series of properties (matched with a constructor).
*/ */
class ObjectSerializer(val clazz: Class<*>, factory: SerializerFactory) : AMQPSerializer<Any> { class ObjectSerializer(val clazz: Type, factory: SerializerFactory) : AMQPSerializer<Any> {
override val type: Type get() = clazz override val type: Type get() = clazz
private val javaConstructor: Constructor<Any>? private val javaConstructor: Constructor<Any>?
internal val propertySerializers: Collection<PropertySerializer> internal val propertySerializers: Collection<PropertySerializer>
@ -20,7 +21,9 @@ class ObjectSerializer(val clazz: Class<*>, factory: SerializerFactory) : AMQPSe
javaConstructor = kotlinConstructor?.javaConstructor javaConstructor = kotlinConstructor?.javaConstructor
propertySerializers = propertiesForSerialization(kotlinConstructor, clazz, factory) propertySerializers = propertiesForSerialization(kotlinConstructor, clazz, factory)
} }
private val typeName = clazz.name
private val typeName = nameForType(clazz)
override val typeDescriptor = "$DESCRIPTOR_DOMAIN:${fingerprintForType(type, factory)}" override val typeDescriptor = "$DESCRIPTOR_DOMAIN:${fingerprintForType(type, factory)}"
private val interfaces = interfacesForSerialization(clazz) // TODO maybe this proves too much and we need annotations to restrict. private val interfaces = interfacesForSerialization(clazz) // TODO maybe this proves too much and we need annotations to restrict.
@ -65,7 +68,7 @@ class ObjectSerializer(val clazz: Class<*>, factory: SerializerFactory) : AMQPSe
} }
private fun generateProvides(): List<String> { private fun generateProvides(): List<String> {
return interfaces.map { it.typeName } return interfaces.map { nameForType(it) }
} }

View File

@ -1,14 +1,16 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import org.apache.qpid.proton.amqp.Binary
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.lang.reflect.Method import java.lang.reflect.Method
import java.lang.reflect.Type
import kotlin.reflect.full.memberProperties import kotlin.reflect.full.memberProperties
import kotlin.reflect.jvm.javaGetter import kotlin.reflect.jvm.javaGetter
/** /**
* Base class for serialization of a property of an object. * Base class for serialization of a property of an object.
*/ */
sealed class PropertySerializer(val name: String, val readMethod: Method) { sealed class PropertySerializer(val name: String, val readMethod: Method, val resolvedType: Type) {
abstract fun writeClassInfo(output: SerializationOutput) abstract fun writeClassInfo(output: SerializationOutput)
abstract fun writeProperty(obj: Any?, data: Data, output: SerializationOutput) abstract fun writeProperty(obj: Any?, data: Data, output: SerializationOutput)
abstract fun readProperty(obj: Any?, schema: Schema, input: DeserializationInput): Any? abstract fun readProperty(obj: Any?, schema: Schema, input: DeserializationInput): Any?
@ -18,23 +20,20 @@ sealed class PropertySerializer(val name: String, val readMethod: Method) {
val default: String? = generateDefault() val default: String? = generateDefault()
val mandatory: Boolean = generateMandatory() val mandatory: Boolean = generateMandatory()
private val isInterface: Boolean get() = (readMethod.genericReturnType as? Class<*>)?.isInterface ?: false private val isInterface: Boolean get() = resolvedType.asClass()?.isInterface ?: false
private val isJVMPrimitive: Boolean get() = (readMethod.genericReturnType as? Class<*>)?.isPrimitive ?: false private val isJVMPrimitive: Boolean get() = resolvedType.asClass()?.isPrimitive ?: false
private fun generateType(): String { private fun generateType(): String {
return if (isInterface) "*" else { return if (isInterface || resolvedType == Any::class.java) "*" else SerializerFactory.nameForType(resolvedType)
val primitiveName = SerializerFactory.primitiveTypeName(readMethod.genericReturnType)
return primitiveName ?: readMethod.genericReturnType.typeName
}
} }
private fun generateRequires(): List<String> { private fun generateRequires(): List<String> {
return if (isInterface) listOf(readMethod.genericReturnType.typeName) else emptyList() return if (isInterface) listOf(SerializerFactory.nameForType(resolvedType)) else emptyList()
} }
private fun generateDefault(): String? { private fun generateDefault(): String? {
if (isJVMPrimitive) { if (isJVMPrimitive) {
return when (readMethod.genericReturnType) { return when (resolvedType) {
java.lang.Boolean.TYPE -> "false" java.lang.Boolean.TYPE -> "false"
java.lang.Character.TYPE -> "&#0" java.lang.Character.TYPE -> "&#0"
else -> "0" else -> "0"
@ -54,13 +53,12 @@ sealed class PropertySerializer(val name: String, val readMethod: Method) {
} }
companion object { companion object {
fun make(name: String, readMethod: Method, factory: SerializerFactory): PropertySerializer { fun make(name: String, readMethod: Method, resolvedType: Type, factory: SerializerFactory): PropertySerializer {
val type = readMethod.genericReturnType if (SerializerFactory.isPrimitive(resolvedType)) {
if (SerializerFactory.isPrimitive(type)) {
// This is a little inefficient for performance since it does a runtime check of type. We could do build time check with lots of subclasses here. // This is a little inefficient for performance since it does a runtime check of type. We could do build time check with lots of subclasses here.
return AMQPPrimitivePropertySerializer(name, readMethod) return AMQPPrimitivePropertySerializer(name, readMethod, resolvedType)
} else { } else {
return DescribedTypePropertySerializer(name, readMethod) { factory.get(null, type) } return DescribedTypePropertySerializer(name, readMethod, resolvedType) { factory.get(null, resolvedType) }
} }
} }
} }
@ -68,35 +66,43 @@ sealed class PropertySerializer(val name: String, val readMethod: Method) {
/** /**
* A property serializer for a complex type (another object). * A property serializer for a complex type (another object).
*/ */
class DescribedTypePropertySerializer(name: String, readMethod: Method, private val lazyTypeSerializer: () -> AMQPSerializer<Any>) : PropertySerializer(name, readMethod) { class DescribedTypePropertySerializer(name: String, readMethod: Method, resolvedType: Type, private val lazyTypeSerializer: () -> AMQPSerializer<*>) : PropertySerializer(name, readMethod, resolvedType) {
// This is lazy so we don't get an infinite loop when a method returns an instance of the class. // This is lazy so we don't get an infinite loop when a method returns an instance of the class.
private val typeSerializer: AMQPSerializer<Any> by lazy { lazyTypeSerializer() } private val typeSerializer: AMQPSerializer<*> by lazy { lazyTypeSerializer() }
override fun writeClassInfo(output: SerializationOutput) { override fun writeClassInfo(output: SerializationOutput) {
if (resolvedType != Any::class.java) {
typeSerializer.writeClassInfo(output) typeSerializer.writeClassInfo(output)
} }
}
override fun readProperty(obj: Any?, schema: Schema, input: DeserializationInput): Any? { override fun readProperty(obj: Any?, schema: Schema, input: DeserializationInput): Any? {
return input.readObjectOrNull(obj, schema, readMethod.genericReturnType) return input.readObjectOrNull(obj, schema, resolvedType)
} }
override fun writeProperty(obj: Any?, data: Data, output: SerializationOutput) { override fun writeProperty(obj: Any?, data: Data, output: SerializationOutput) {
output.writeObjectOrNull(readMethod.invoke(obj), data, readMethod.genericReturnType) output.writeObjectOrNull(readMethod.invoke(obj), data, resolvedType)
} }
} }
/** /**
* A property serializer for an AMQP primitive type (Int, String, etc). * A property serializer for an AMQP primitive type (Int, String, etc).
*/ */
class AMQPPrimitivePropertySerializer(name: String, readMethod: Method) : PropertySerializer(name, readMethod) { class AMQPPrimitivePropertySerializer(name: String, readMethod: Method, resolvedType: Type) : PropertySerializer(name, readMethod, resolvedType) {
override fun writeClassInfo(output: SerializationOutput) {} override fun writeClassInfo(output: SerializationOutput) {}
override fun readProperty(obj: Any?, schema: Schema, input: DeserializationInput): Any? { override fun readProperty(obj: Any?, schema: Schema, input: DeserializationInput): Any? {
return obj return if (obj is Binary) obj.array else obj
} }
override fun writeProperty(obj: Any?, data: Data, output: SerializationOutput) { override fun writeProperty(obj: Any?, data: Data, output: SerializationOutput) {
data.putObject(readMethod.invoke(obj)) val value = readMethod.invoke(obj)
if (value is ByteArray) {
data.putObject(Binary(value))
} else {
data.putObject(value)
}
} }
} }
} }

View File

@ -2,7 +2,7 @@ package net.corda.core.serialization.amqp
import com.google.common.hash.Hasher import com.google.common.hash.Hasher
import com.google.common.hash.Hashing import com.google.common.hash.Hashing
import net.corda.core.crypto.Base58 import net.corda.core.crypto.toBase64
import net.corda.core.serialization.OpaqueBytes import net.corda.core.serialization.OpaqueBytes
import org.apache.qpid.proton.amqp.DescribedType import org.apache.qpid.proton.amqp.DescribedType
import org.apache.qpid.proton.amqp.UnsignedLong import org.apache.qpid.proton.amqp.UnsignedLong
@ -12,6 +12,8 @@ import java.io.NotSerializableException
import java.lang.reflect.GenericArrayType import java.lang.reflect.GenericArrayType
import java.lang.reflect.ParameterizedType import java.lang.reflect.ParameterizedType
import java.lang.reflect.Type import java.lang.reflect.Type
import java.lang.reflect.TypeVariable
import java.util.*
// TODO: get an assigned number as per AMQP spec // TODO: get an assigned number as per AMQP spec
val DESCRIPTOR_TOP_32BITS: Long = 0xc0da0000 val DESCRIPTOR_TOP_32BITS: Long = 0xc0da0000
@ -310,6 +312,7 @@ private val ALREADY_SEEN_HASH: String = "Already seen = true"
private val NULLABLE_HASH: String = "Nullable = true" private val NULLABLE_HASH: String = "Nullable = true"
private val NOT_NULLABLE_HASH: String = "Nullable = false" private val NOT_NULLABLE_HASH: String = "Nullable = false"
private val ANY_TYPE_HASH: String = "Any type = true" private val ANY_TYPE_HASH: String = "Any type = true"
private val TYPE_VARIABLE_HASH: String = "Type variable = true"
/** /**
* The method generates a fingerprint for a given JVM [Type] that should be unique to the schema representation. * The method generates a fingerprint for a given JVM [Type] that should be unique to the schema representation.
@ -320,44 +323,83 @@ private val ANY_TYPE_HASH: String = "Any type = true"
* different. * different.
*/ */
// TODO: write tests // TODO: write tests
internal fun fingerprintForType(type: Type, factory: SerializerFactory): String = Base58.encode(fingerprintForType(type, HashSet(), Hashing.murmur3_128().newHasher(), factory).hash().asBytes()) internal fun fingerprintForType(type: Type, factory: SerializerFactory): String {
return fingerprintForType(type, null, HashSet(), Hashing.murmur3_128().newHasher(), factory).hash().asBytes().toBase64()
}
private fun fingerprintForType(type: Type, alreadySeen: MutableSet<Type>, hasher: Hasher, factory: SerializerFactory): Hasher { internal fun fingerprintForDescriptors(vararg typeDescriptors: String): String {
val hasher = Hashing.murmur3_128().newHasher()
for (typeDescriptor in typeDescriptors) {
hasher.putUnencodedChars(typeDescriptor)
}
return hasher.hash().asBytes().toBase64()
}
// This method concatentates various elements of the types recursively as unencoded strings into the hasher, effectively
// creating a unique string for a type which we then hash in the calling function above.
private fun fingerprintForType(type: Type, contextType: Type?, alreadySeen: MutableSet<Type>, hasher: Hasher, factory: SerializerFactory): Hasher {
return if (type in alreadySeen) { return if (type in alreadySeen) {
hasher.putUnencodedChars(ALREADY_SEEN_HASH) hasher.putUnencodedChars(ALREADY_SEEN_HASH)
} else { } else {
alreadySeen += type alreadySeen += type
try {
if (type is SerializerFactory.AnyType) { if (type is SerializerFactory.AnyType) {
hasher.putUnencodedChars(ANY_TYPE_HASH) hasher.putUnencodedChars(ANY_TYPE_HASH)
} else if (type is Class<*>) { } else if (type is Class<*>) {
if (type.isArray) { if (type.isArray) {
fingerprintForType(type.componentType, alreadySeen, hasher, factory).putUnencodedChars(ARRAY_HASH) fingerprintForType(type.componentType, contextType, alreadySeen, hasher, factory).putUnencodedChars(ARRAY_HASH)
} else if (SerializerFactory.isPrimitive(type)) { } else if (SerializerFactory.isPrimitive(type)) {
hasher.putUnencodedChars(type.name) hasher.putUnencodedChars(type.name)
} else if (Collection::class.java.isAssignableFrom(type) || Map::class.java.isAssignableFrom(type)) { } else if (isCollectionOrMap(type)) {
hasher.putUnencodedChars(type.name) hasher.putUnencodedChars(type.name)
} else { } else {
// Need to check if a custom serializer is applicable // Need to check if a custom serializer is applicable
val customSerializer = factory.findCustomSerializer(type) val customSerializer = factory.findCustomSerializer(type, type)
if (customSerializer == null) { if (customSerializer == null) {
// Hash the class + properties + interfaces if (type.kotlin.objectInstance != null) {
propertiesForSerialization(constructorForDeserialization(type), type, factory).fold(hasher.putUnencodedChars(type.name)) { orig, param -> // TODO: name collision is too likely for kotlin objects, we need to introduce some reference
fingerprintForType(param.readMethod.genericReturnType, alreadySeen, orig, factory).putUnencodedChars(param.name).putUnencodedChars(if (param.mandatory) NOT_NULLABLE_HASH else NULLABLE_HASH) // to the CorDapp but maybe reference to the JAR in the short term.
hasher.putUnencodedChars(type.name)
} else {
fingerprintForObject(type, contextType, alreadySeen, hasher, factory)
} }
interfacesForSerialization(type).map { fingerprintForType(it, alreadySeen, hasher, factory) }
hasher
} else { } else {
hasher.putUnencodedChars(customSerializer.typeDescriptor) hasher.putUnencodedChars(customSerializer.typeDescriptor)
} }
} }
} else if (type is ParameterizedType) { } else if (type is ParameterizedType) {
// Hash the rawType + params // Hash the rawType + params
type.actualTypeArguments.fold(fingerprintForType(type.rawType, alreadySeen, hasher, factory)) { orig, paramType -> fingerprintForType(paramType, alreadySeen, orig, factory) } val clazz = type.rawType as Class<*>
val startingHash = if (isCollectionOrMap(clazz)) {
hasher.putUnencodedChars(clazz.name)
} else {
fingerprintForObject(type, type, alreadySeen, hasher, factory)
}
// ... and concatentate the type data for each parameter type.
type.actualTypeArguments.fold(startingHash) { orig, paramType -> fingerprintForType(paramType, type, alreadySeen, orig, factory) }
} else if (type is GenericArrayType) { } else if (type is GenericArrayType) {
// Hash the element type + some array hash // Hash the element type + some array hash
fingerprintForType(type.genericComponentType, alreadySeen, hasher, factory).putUnencodedChars(ARRAY_HASH) fingerprintForType(type.genericComponentType, contextType, alreadySeen, hasher, factory).putUnencodedChars(ARRAY_HASH)
} else if (type is TypeVariable<*>) {
// TODO: include bounds
hasher.putUnencodedChars(type.name).putUnencodedChars(TYPE_VARIABLE_HASH)
} else { } else {
throw NotSerializableException("Don't know how to hash $type") throw NotSerializableException("Don't know how to hash")
}
} catch(e: NotSerializableException) {
throw NotSerializableException("${e.message} -> $type")
} }
} }
} }
private fun isCollectionOrMap(type: Class<*>) = Collection::class.java.isAssignableFrom(type) || Map::class.java.isAssignableFrom(type)
private fun fingerprintForObject(type: Type, contextType: Type?, alreadySeen: MutableSet<Type>, hasher: Hasher, factory: SerializerFactory): Hasher {
// Hash the class + properties + interfaces
val name = type.asClass()?.name ?: throw NotSerializableException("Expected only Class or ParameterizedType but found $type")
propertiesForSerialization(constructorForDeserialization(type), contextType ?: type, factory).fold(hasher.putUnencodedChars(name)) { orig, prop ->
fingerprintForType(prop.resolvedType, type, alreadySeen, orig, factory).putUnencodedChars(prop.name).putUnencodedChars(if (prop.mandatory) NOT_NULLABLE_HASH else NULLABLE_HASH)
}
interfacesForSerialization(type).map { fingerprintForType(it, type, alreadySeen, hasher, factory) }
return hasher
}

View File

@ -4,10 +4,8 @@ import com.google.common.reflect.TypeToken
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.beans.Introspector import java.beans.Introspector
import java.io.NotSerializableException import java.io.NotSerializableException
import java.lang.reflect.Method import java.lang.reflect.*
import java.lang.reflect.Modifier import java.util.*
import java.lang.reflect.ParameterizedType
import java.lang.reflect.Type
import kotlin.reflect.KClass import kotlin.reflect.KClass
import kotlin.reflect.KFunction import kotlin.reflect.KFunction
import kotlin.reflect.KParameter import kotlin.reflect.KParameter
@ -29,9 +27,10 @@ annotation class ConstructorForDeserialization
* Otherwise it starts with the primary constructor in kotlin, if there is one, and then will override this with any that is * Otherwise it starts with the primary constructor in kotlin, if there is one, and then will override this with any that is
* annotated with [@CordaConstructor]. It will report an error if more than one constructor is annotated. * annotated with [@CordaConstructor]. It will report an error if more than one constructor is annotated.
*/ */
internal fun <T : Any> constructorForDeserialization(clazz: Class<T>): KFunction<T>? { internal fun constructorForDeserialization(type: Type): KFunction<Any>? {
val clazz: Class<*> = type.asClass()!!
if (isConcrete(clazz)) { if (isConcrete(clazz)) {
var preferredCandidate: KFunction<T>? = clazz.kotlin.primaryConstructor var preferredCandidate: KFunction<Any>? = clazz.kotlin.primaryConstructor
var annotatedCount = 0 var annotatedCount = 0
val kotlinConstructors = clazz.kotlin.constructors val kotlinConstructors = clazz.kotlin.constructors
val hasDefault = kotlinConstructors.any { it.parameters.isEmpty() } val hasDefault = kotlinConstructors.any { it.parameters.isEmpty() }
@ -60,13 +59,14 @@ internal fun <T : Any> constructorForDeserialization(clazz: Class<T>): KFunction
* Note, you will need any Java classes to be compiled with the `-parameters` option to ensure constructor parameters have * Note, you will need any Java classes to be compiled with the `-parameters` option to ensure constructor parameters have
* names accessible via reflection. * names accessible via reflection.
*/ */
internal fun <T : Any> propertiesForSerialization(kotlinConstructor: KFunction<T>?, clazz: Class<*>, factory: SerializerFactory): Collection<PropertySerializer> { internal fun <T : Any> propertiesForSerialization(kotlinConstructor: KFunction<T>?, type: Type, factory: SerializerFactory): Collection<PropertySerializer> {
return if (kotlinConstructor != null) propertiesForSerialization(kotlinConstructor, factory) else propertiesForSerialization(clazz, factory) val clazz = type.asClass()!!
return if (kotlinConstructor != null) propertiesForSerializationFromConstructor(kotlinConstructor, type, factory) else propertiesForSerializationFromAbstract(clazz, type, factory)
} }
private fun isConcrete(clazz: Class<*>): Boolean = !(clazz.isInterface || Modifier.isAbstract(clazz.modifiers)) private fun isConcrete(clazz: Class<*>): Boolean = !(clazz.isInterface || Modifier.isAbstract(clazz.modifiers))
private fun <T : Any> propertiesForSerialization(kotlinConstructor: KFunction<T>, factory: SerializerFactory): Collection<PropertySerializer> { private fun <T : Any> propertiesForSerializationFromConstructor(kotlinConstructor: KFunction<T>, type: Type, factory: SerializerFactory): Collection<PropertySerializer> {
val clazz = (kotlinConstructor.returnType.classifier as KClass<*>).javaObjectType val clazz = (kotlinConstructor.returnType.classifier as KClass<*>).javaObjectType
// Kotlin reflection doesn't work with Java getters the way you might expect, so we drop back to good ol' beans. // Kotlin reflection doesn't work with Java getters the way you might expect, so we drop back to good ol' beans.
val properties = Introspector.getBeanInfo(clazz).propertyDescriptors.filter { it.name != "class" }.groupBy { it.name }.mapValues { it.value[0] } val properties = Introspector.getBeanInfo(clazz).propertyDescriptors.filter { it.name != "class" }.groupBy { it.name }.mapValues { it.value[0] }
@ -78,10 +78,11 @@ private fun <T : Any> propertiesForSerialization(kotlinConstructor: KFunction<T>
// Check that the method has a getter in java. // Check that the method has a getter in java.
val getter = matchingProperty.readMethod ?: throw NotSerializableException("Property has no getter method for $name of $clazz." + val getter = matchingProperty.readMethod ?: throw NotSerializableException("Property has no getter method for $name of $clazz." +
" If using Java and the parameter name looks anonymous, check that you have the -parameters option specified in the Java compiler.") " If using Java and the parameter name looks anonymous, check that you have the -parameters option specified in the Java compiler.")
val returnType = resolveTypeVariables(getter.genericReturnType, type)
if (constructorParamTakesReturnTypeOfGetter(getter, param)) { if (constructorParamTakesReturnTypeOfGetter(getter, param)) {
rc += PropertySerializer.make(name, getter, factory) rc += PropertySerializer.make(name, getter, returnType, factory)
} else { } else {
throw NotSerializableException("Property type ${getter.genericReturnType} for $name of $clazz differs from constructor parameter type ${param.type.javaType}") throw NotSerializableException("Property type $returnType for $name of $clazz differs from constructor parameter type ${param.type.javaType}")
} }
} }
return rc return rc
@ -89,35 +90,36 @@ private fun <T : Any> propertiesForSerialization(kotlinConstructor: KFunction<T>
private fun constructorParamTakesReturnTypeOfGetter(getter: Method, param: KParameter): Boolean = TypeToken.of(param.type.javaType).isSupertypeOf(getter.genericReturnType) private fun constructorParamTakesReturnTypeOfGetter(getter: Method, param: KParameter): Boolean = TypeToken.of(param.type.javaType).isSupertypeOf(getter.genericReturnType)
private fun propertiesForSerialization(clazz: Class<*>, factory: SerializerFactory): Collection<PropertySerializer> { private fun propertiesForSerializationFromAbstract(clazz: Class<*>, type: Type, factory: SerializerFactory): Collection<PropertySerializer> {
// Kotlin reflection doesn't work with Java getters the way you might expect, so we drop back to good ol' beans. // Kotlin reflection doesn't work with Java getters the way you might expect, so we drop back to good ol' beans.
val properties = Introspector.getBeanInfo(clazz).propertyDescriptors.filter { it.name != "class" }.sortedBy { it.name } val properties = Introspector.getBeanInfo(clazz).propertyDescriptors.filter { it.name != "class" }.sortedBy { it.name }
val rc: MutableList<PropertySerializer> = ArrayList(properties.size) val rc: MutableList<PropertySerializer> = ArrayList(properties.size)
for (property in properties) { for (property in properties) {
// Check that the method has a getter in java. // Check that the method has a getter in java.
val getter = property.readMethod ?: throw NotSerializableException("Property has no getter method for ${property.name} of $clazz.") val getter = property.readMethod ?: throw NotSerializableException("Property has no getter method for ${property.name} of $clazz.")
rc += PropertySerializer.make(property.name, getter, factory) val returnType = resolveTypeVariables(getter.genericReturnType, type)
rc += PropertySerializer.make(property.name, getter, returnType, factory)
} }
return rc return rc
} }
internal fun interfacesForSerialization(clazz: Class<*>): List<Type> { internal fun interfacesForSerialization(type: Type): List<Type> {
val interfaces = LinkedHashSet<Type>() val interfaces = LinkedHashSet<Type>()
exploreType(clazz, interfaces) exploreType(type, interfaces)
return interfaces.toList() return interfaces.toList()
} }
private fun exploreType(type: Type?, interfaces: MutableSet<Type>) { private fun exploreType(type: Type?, interfaces: MutableSet<Type>) {
val clazz = (type as? Class<*>) ?: (type as? ParameterizedType)?.rawType as? Class<*> val clazz = type?.asClass()
if (clazz != null) { if (clazz != null) {
if (clazz.isInterface) interfaces += clazz if (clazz.isInterface) interfaces += type!!
for (newInterface in clazz.genericInterfaces) { for (newInterface in clazz.genericInterfaces) {
if (newInterface !in interfaces) { if (newInterface !in interfaces) {
interfaces += newInterface exploreType(resolveTypeVariables(newInterface, type), interfaces)
exploreType(newInterface, interfaces)
} }
} }
exploreType(clazz.genericSuperclass, interfaces) val superClass = clazz.genericSuperclass ?: return
exploreType(resolveTypeVariables(superClass, type), interfaces)
} }
} }
@ -144,3 +146,57 @@ fun Data.withList(block: Data.() -> Unit) {
block() block()
exit() // exit list exit() // exit list
} }
private fun resolveTypeVariables(actualType: Type, contextType: Type?): Type {
val resolvedType = if (contextType != null) TypeToken.of(contextType).resolveType(actualType).type else actualType
// TODO: surely we check it is concrete at this point with no TypeVariables
return if (resolvedType is TypeVariable<*>) {
val bounds = resolvedType.bounds
return if (bounds.isEmpty()) SerializerFactory.AnyType else if (bounds.size == 1) resolveTypeVariables(bounds[0], contextType) else throw NotSerializableException("Got bounded type $actualType but only support single bound.")
} else {
resolvedType
}
}
internal fun Type.asClass(): Class<*>? {
return if (this is Class<*>) {
this
} else if (this is ParameterizedType) {
this.rawType.asClass()
} else if (this is GenericArrayType) {
this.genericComponentType.asClass()?.arrayClass()
} else null
}
internal fun Type.asArray(): Type? {
return if (this is Class<*>) {
this.arrayClass()
} else if (this is ParameterizedType) {
DeserializedGenericArrayType(this)
} else null
}
internal fun Class<*>.arrayClass(): Class<*> = java.lang.reflect.Array.newInstance(this, 0).javaClass
internal fun Type.isArray(): Boolean = (this is Class<*> && this.isArray) || (this is GenericArrayType)
internal fun Type.componentType(): Type {
check(this.isArray()) { "$this is not an array type." }
return (this as? Class<*>)?.componentType ?: (this as GenericArrayType).genericComponentType
}
internal fun Class<*>.asParameterizedType(): ParameterizedType {
return DeserializedParameterizedType(this, this.typeParameters)
}
internal fun Type.asParameterizedType(): ParameterizedType {
return when (this) {
is Class<*> -> this.asParameterizedType()
is ParameterizedType -> this
else -> throw NotSerializableException("Don't know how to convert to ParameterizedType")
}
}
internal fun Type.isSubClassOf(type: Type): Boolean {
return TypeToken.of(this).isSubtypeOf(type)
}

View File

@ -1,6 +1,7 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import com.google.common.primitives.Primitives import com.google.common.primitives.Primitives
import com.google.common.reflect.TypeResolver
import net.corda.core.checkNotUnorderedHashMap import net.corda.core.checkNotUnorderedHashMap
import net.corda.core.serialization.AllWhitelist import net.corda.core.serialization.AllWhitelist
import net.corda.core.serialization.ClassWhitelist import net.corda.core.serialization.ClassWhitelist
@ -20,9 +21,9 @@ import javax.annotation.concurrent.ThreadSafe
* Factory of serializers designed to be shared across threads and invocations. * Factory of serializers designed to be shared across threads and invocations.
*/ */
// TODO: enums // TODO: enums
// TODO: object references // TODO: object references - need better fingerprinting?
// TODO: class references? (e.g. cheat with repeated descriptors using a long encoding, like object ref proposal) // TODO: class references? (e.g. cheat with repeated descriptors using a long encoding, like object ref proposal)
// TODO: Inner classes etc // TODO: Inner classes etc. Should we allow? Currently not considered.
// TODO: support for intern-ing of deserialized objects for some core types (e.g. PublicKey) for memory efficiency // TODO: support for intern-ing of deserialized objects for some core types (e.g. PublicKey) for memory efficiency
// TODO: maybe support for caching of serialized form of some core types for performance // TODO: maybe support for caching of serialized form of some core types for performance
// TODO: profile for performance in general // TODO: profile for performance in general
@ -32,7 +33,13 @@ import javax.annotation.concurrent.ThreadSafe
// TODO: apply class loader logic and an "app context" throughout this code. // TODO: apply class loader logic and an "app context" throughout this code.
// TODO: schema evolution solution when the fingerprints do not line up. // TODO: schema evolution solution when the fingerprints do not line up.
// TODO: allow definition of well known types that are left out of the schema. // TODO: allow definition of well known types that are left out of the schema.
// TODO: automatically support byte[] without having to wrap in [Binary]. // TODO: generally map Object to '*' all over the place in the schema and make sure use of '*' amd '?' is consistent and documented in generics.
// TODO: found a document that states textual descriptors are Symbols. Adjust schema class appropriately.
// TODO: document and alert to the fact that classes cannot default superclass/interface properties otherwise they are "erased" due to matching with constructor.
// TODO: type name prefixes for interfaces and abstract classes? Or use label?
// TODO: generic types should define restricted type alias with source of the wildcarded version, I think, if we're to generate classes from schema
// TODO: need to rethink matching of constructor to properties in relation to implementing interfaces and needing those properties etc.
// TODO: need to support super classes as well as interfaces with our current code base... what's involved? If we continue to ban, what is the impact?
@ThreadSafe @ThreadSafe
class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) { class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
private val serializersByType = ConcurrentHashMap<Type, AMQPSerializer<Any>>() private val serializersByType = ConcurrentHashMap<Type, AMQPSerializer<Any>>()
@ -42,42 +49,97 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
/** /**
* Look up, and manufacture if necessary, a serializer for the given type. * Look up, and manufacture if necessary, a serializer for the given type.
* *
* @param actualType Will be null if there isn't an actual object instance available (e.g. for * @param actualClass Will be null if there isn't an actual object instance available (e.g. for
* restricted type processing). * restricted type processing).
*/ */
@Throws(NotSerializableException::class) @Throws(NotSerializableException::class)
fun get(actualType: Class<*>?, declaredType: Type): AMQPSerializer<Any> { fun get(actualClass: Class<*>?, declaredType: Type): AMQPSerializer<Any> {
if (declaredType is ParameterizedType) { val declaredClass = declaredType.asClass()
if (declaredClass != null) {
val actualType: Type = inferTypeVariables(actualClass, declaredClass, declaredType) ?: declaredType
if (Collection::class.java.isAssignableFrom(declaredClass)) {
return serializersByType.computeIfAbsent(declaredType) { return serializersByType.computeIfAbsent(declaredType) {
// We allow only Collection and Map. CollectionSerializer(declaredType as? ParameterizedType ?: DeserializedParameterizedType(declaredClass, arrayOf(AnyType), null), this)
val rawType = declaredType.rawType }
if (rawType is Class<*>) { } else if (Map::class.java.isAssignableFrom(declaredClass)) {
checkParameterisedTypesConcrete(declaredType.actualTypeArguments) return serializersByType.computeIfAbsent(declaredClass) {
if (Collection::class.java.isAssignableFrom(rawType)) { makeMapSerializer(declaredType as? ParameterizedType ?: DeserializedParameterizedType(declaredClass, arrayOf(AnyType, AnyType), null))
CollectionSerializer(declaredType, this) }
} else if (Map::class.java.isAssignableFrom(rawType)) {
makeMapSerializer(declaredType)
} else { } else {
throw NotSerializableException("Declared types of $declaredType are not supported.") return makeClassSerializer(actualClass ?: declaredClass, actualType, declaredType)
} }
} else { } else {
throw NotSerializableException("Declared types of $declaredType are not supported.") throw NotSerializableException("Declared types of $declaredType are not supported.")
} }
} }
/**
* Try and infer concrete types for any generics type variables for the actual class encountered, based on the declared
* type.
*/
// TODO: test GenericArrayType
private fun inferTypeVariables(actualClass: Class<*>?, declaredClass: Class<*>, declaredType: Type): Type? {
if (declaredType is ParameterizedType) {
return inferTypeVariables(actualClass, declaredClass, declaredType)
} else if (declaredType is Class<*>) { } else if (declaredType is Class<*>) {
// Simple classes allowed // Nothing to infer, otherwise we'd have ParameterizedType
if (Collection::class.java.isAssignableFrom(declaredType)) { return actualClass
return serializersByType.computeIfAbsent(declaredType) { CollectionSerializer(DeserializedParameterizedType(declaredType, arrayOf(AnyType), null), this) }
} else if (Map::class.java.isAssignableFrom(declaredType)) {
return serializersByType.computeIfAbsent(declaredType) { makeMapSerializer(DeserializedParameterizedType(declaredType, arrayOf(AnyType, AnyType), null)) }
} else {
return makeClassSerializer(actualType ?: declaredType)
}
} else if (declaredType is GenericArrayType) { } else if (declaredType is GenericArrayType) {
return serializersByType.computeIfAbsent(declaredType) { ArraySerializer(declaredType, this) } val declaredComponent = declaredType.genericComponentType
} else { return inferTypeVariables(actualClass?.componentType, declaredComponent.asClass()!!, declaredComponent)?.asArray()
throw NotSerializableException("Declared types of $declaredType are not supported.") } else return null
} }
/**
* Try and infer concrete types for any generics type variables for the actual class encountered, based on the declared
* type, which must be a [ParameterizedType].
*/
private fun inferTypeVariables(actualClass: Class<*>?, declaredClass: Class<*>, declaredType: ParameterizedType): Type? {
if (actualClass == null || declaredClass == actualClass) {
return null
} else if (declaredClass.isAssignableFrom(actualClass)) {
return if (actualClass.typeParameters.isNotEmpty()) {
// The actual class can never have type variables resolved, due to the JVM's use of type erasure, so let's try and resolve them
// Search for declared type in the inheritance hierarchy and then see if that fills in all the variables
val implementationChain: List<Type>? = findPathToDeclared(actualClass, declaredType, mutableListOf<Type>())
if (implementationChain != null) {
val start = implementationChain.last()
val rest = implementationChain.dropLast(1).drop(1)
val resolver = rest.reversed().fold(TypeResolver().where(start, declaredType)) {
resolved, chainEntry ->
val newResolved = resolved.resolveType(chainEntry)
TypeResolver().where(chainEntry, newResolved)
}
// The end type is a special case as it is a Class, so we need to fake up a ParameterizedType for it to get the TypeResolver to do anything.
val endType = DeserializedParameterizedType(actualClass, actualClass.typeParameters)
val resolvedType = resolver.resolveType(endType)
resolvedType
} else throw NotSerializableException("No inheritance path between actual $actualClass and declared $declaredType.")
} else actualClass
} else throw NotSerializableException("Found object of type $actualClass in a property expecting $declaredType")
}
// Stop when reach declared type or return null if we don't find it.
private fun findPathToDeclared(startingType: Type, declaredType: Type, chain: MutableList<Type>): List<Type>? {
chain.add(startingType)
val startingClass = startingType.asClass()
if (startingClass == declaredType.asClass()) {
// We're done...
return chain
}
// Now explore potential options of superclass and all interfaces
val superClass = startingClass?.genericSuperclass
val superClassChain = if (superClass != null) {
val resolved = TypeResolver().where(startingClass.asParameterizedType(), startingType.asParameterizedType()).resolveType(superClass)
findPathToDeclared(resolved, declaredType, ArrayList(chain))
} else null
if (superClassChain != null) return superClassChain
for (iface in startingClass?.genericInterfaces ?: emptyArray()) {
val resolved = TypeResolver().where(startingClass!!.asParameterizedType(), startingType.asParameterizedType()).resolveType(iface)
return findPathToDeclared(resolved, declaredType, ArrayList(chain)) ?: continue
}
return null
} }
/** /**
@ -93,7 +155,8 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
} }
/** /**
* TODO: Add docs * Register a custom serializer for any type that cannot be serialized or deserialized by the default serializer
* that expects to find getters and a constructor with a parameter for each property.
*/ */
fun register(customSerializer: CustomSerializer<out Any>) { fun register(customSerializer: CustomSerializer<out Any>) {
if (!serializersByDescriptor.containsKey(customSerializer.typeDescriptor)) { if (!serializersByDescriptor.containsKey(customSerializer.typeDescriptor)) {
@ -118,25 +181,10 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
} }
} }
private fun restrictedTypeForName(name: String): Type {
return if (name.endsWith("[]")) {
val elementType = restrictedTypeForName(name.substring(0, name.lastIndex - 1))
if (elementType is ParameterizedType || elementType is GenericArrayType) {
DeserializedGenericArrayType(elementType)
} else if (elementType is Class<*>) {
java.lang.reflect.Array.newInstance(elementType, 0).javaClass
} else {
throw NotSerializableException("Not able to deserialize array type: $name")
}
} else {
DeserializedParameterizedType.make(name)
}
}
private fun processRestrictedType(typeNotation: RestrictedType) { private fun processRestrictedType(typeNotation: RestrictedType) {
serializersByDescriptor.computeIfAbsent(typeNotation.descriptor.name!!) { serializersByDescriptor.computeIfAbsent(typeNotation.descriptor.name!!) {
// TODO: class loader logic, and compare the schema. // TODO: class loader logic, and compare the schema.
val type = restrictedTypeForName(typeNotation.name) val type = typeForName(typeNotation.name)
get(null, type) get(null, type)
} }
} }
@ -144,63 +192,61 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
private fun processCompositeType(typeNotation: CompositeType) { private fun processCompositeType(typeNotation: CompositeType) {
serializersByDescriptor.computeIfAbsent(typeNotation.descriptor.name!!) { serializersByDescriptor.computeIfAbsent(typeNotation.descriptor.name!!) {
// TODO: class loader logic, and compare the schema. // TODO: class loader logic, and compare the schema.
val clazz = Class.forName(typeNotation.name) val type = typeForName(typeNotation.name)
get(clazz, clazz) get(type.asClass() ?: throw NotSerializableException("Unable to build composite type for $type"), type)
} }
} }
private fun checkParameterisedTypesConcrete(actualTypeArguments: Array<out Type>) { private fun makeClassSerializer(clazz: Class<*>, type: Type, declaredType: Type): AMQPSerializer<Any> {
for (type in actualTypeArguments) { return serializersByType.computeIfAbsent(type) {
// Needs to be another parameterised type or a class, or any type.
if (type !is Class<*>) {
if (type is ParameterizedType) {
checkParameterisedTypesConcrete(type.actualTypeArguments)
} else if (type != AnyType) {
throw NotSerializableException("Declared parameterised types containing $type as a parameter are not supported.")
}
}
}
}
private fun makeClassSerializer(clazz: Class<*>): AMQPSerializer<Any> {
return serializersByType.computeIfAbsent(clazz) {
if (isPrimitive(clazz)) { if (isPrimitive(clazz)) {
AMQPPrimitiveSerializer(clazz) AMQPPrimitiveSerializer(clazz)
} else { } else {
findCustomSerializer(clazz) ?: { findCustomSerializer(clazz, declaredType) ?: run {
if (clazz.isArray) { if (type.isArray()) {
whitelisted(clazz.componentType) whitelisted(type.componentType())
ArraySerializer(clazz, this) ArraySerializer(type, this)
} else { } else if (clazz.kotlin.objectInstance != null) {
whitelisted(clazz) whitelisted(clazz)
ObjectSerializer(clazz, this) SingletonSerializer(clazz, clazz.kotlin.objectInstance!!, this)
} else {
whitelisted(type)
ObjectSerializer(type, this)
}
} }
}()
} }
} }
} }
internal fun findCustomSerializer(clazz: Class<*>): AMQPSerializer<Any>? { internal fun findCustomSerializer(clazz: Class<*>, declaredType: Type): AMQPSerializer<Any>? {
// e.g. Imagine if we provided a Map serializer this way, then it won't work if the declared type is AbstractMap, only Map.
// Otherwise it needs to inject additional schema for a RestrictedType source of the super type. Could be done, but do we need it?
for (customSerializer in customSerializers) { for (customSerializer in customSerializers) {
if (customSerializer.isSerializerFor(clazz)) { if (customSerializer.isSerializerFor(clazz)) {
val declaredSuperClass = declaredType.asClass()?.superclass
if (declaredSuperClass == null || !customSerializer.isSerializerFor(declaredSuperClass)) {
return customSerializer return customSerializer
} else {
// Make a subclass serializer for the subclass and return that...
@Suppress("UNCHECKED_CAST")
return CustomSerializer.SubClass<Any>(clazz, customSerializer as CustomSerializer<Any>)
}
} }
} }
return null return null
} }
private fun whitelisted(clazz: Class<*>): Boolean { private fun whitelisted(type: Type) {
if (whitelist.hasListed(clazz) || hasAnnotationInHierarchy(clazz)) { val clazz = type.asClass()!!
return true if (!whitelist.hasListed(clazz) && !hasAnnotationInHierarchy(clazz)) {
} else { throw NotSerializableException("Class $type is not on the whitelist or annotated with @CordaSerializable.")
throw NotSerializableException("Class $clazz is not on the whitelist or annotated with @CordaSerializable.")
} }
} }
// Recursively check the class, interfaces and superclasses for our annotation. // Recursively check the class, interfaces and superclasses for our annotation.
internal fun hasAnnotationInHierarchy(type: Class<*>): Boolean { internal fun hasAnnotationInHierarchy(type: Class<*>): Boolean {
return type.isAnnotationPresent(CordaSerializable::class.java) || return type.isAnnotationPresent(CordaSerializable::class.java) ||
type.interfaces.any { it.isAnnotationPresent(CordaSerializable::class.java) || hasAnnotationInHierarchy(it) } type.interfaces.any { hasAnnotationInHierarchy(it) }
|| (type.superclass != null && hasAnnotationInHierarchy(type.superclass)) || (type.superclass != null && hasAnnotationInHierarchy(type.superclass))
} }
@ -211,9 +257,16 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
} }
companion object { companion object {
fun isPrimitive(type: Type): Boolean = type is Class<*> && Primitives.wrap(type) in primitiveTypeNames fun isPrimitive(type: Type): Boolean = primitiveTypeName(type) != null
fun primitiveTypeName(type: Type): String? = primitiveTypeNames[type as? Class<*>] fun primitiveTypeName(type: Type): String? {
val clazz = type as? Class<*> ?: return null
return primitiveTypeNames[Primitives.unwrap(clazz)]
}
fun primitiveType(type: String): Class<*>? {
return namesOfPrimitiveTypes[type]
}
private val primitiveTypeNames: Map<Class<*>, String> = mapOf( private val primitiveTypeNames: Map<Class<*>, String> = mapOf(
Boolean::class.java to "boolean", Boolean::class.java to "boolean",
@ -221,7 +274,7 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
UnsignedByte::class.java to "ubyte", UnsignedByte::class.java to "ubyte",
Short::class.java to "short", Short::class.java to "short",
UnsignedShort::class.java to "ushort", UnsignedShort::class.java to "ushort",
Integer::class.java to "int", Int::class.java to "int",
UnsignedInteger::class.java to "uint", UnsignedInteger::class.java to "uint",
Long::class.java to "long", Long::class.java to "long",
UnsignedLong::class.java to "ulong", UnsignedLong::class.java to "ulong",
@ -233,9 +286,36 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
Char::class.java to "char", Char::class.java to "char",
Date::class.java to "timestamp", Date::class.java to "timestamp",
UUID::class.java to "uuid", UUID::class.java to "uuid",
Binary::class.java to "binary", ByteArray::class.java to "binary",
String::class.java to "string", String::class.java to "string",
Symbol::class.java to "symbol") Symbol::class.java to "symbol")
private val namesOfPrimitiveTypes: Map<String, Class<*>> = primitiveTypeNames.map { it.value to it.key }.toMap()
fun nameForType(type: Type): String {
if (type is Class<*>) {
return primitiveTypeName(type) ?: if (type.isArray) "${nameForType(type.componentType)}[]" else type.name
} else if (type is ParameterizedType) {
return "${nameForType(type.rawType)}<${type.actualTypeArguments.joinToString { nameForType(it) }}>"
} else if (type is GenericArrayType) {
return "${nameForType(type.genericComponentType)}[]"
} else throw NotSerializableException("Unable to render type $type to a string.")
}
private fun typeForName(name: String): Type {
return if (name.endsWith("[]")) {
val elementType = typeForName(name.substring(0, name.lastIndex - 1))
if (elementType is ParameterizedType || elementType is GenericArrayType) {
DeserializedGenericArrayType(elementType)
} else if (elementType is Class<*>) {
java.lang.reflect.Array.newInstance(elementType, 0).javaClass
} else {
throw NotSerializableException("Not able to deserialize array type: $name")
}
} else {
DeserializedParameterizedType.make(name)
}
}
} }
object AnyType : WildcardType { object AnyType : WildcardType {
@ -246,4 +326,3 @@ class SerializerFactory(val whitelist: ClassWhitelist = AllWhitelist) {
override fun toString(): String = "?" override fun toString(): String = "?"
} }
} }

View File

@ -0,0 +1,32 @@
package net.corda.core.serialization.amqp
import org.apache.qpid.proton.codec.Data
import java.lang.reflect.Type
/**
* A custom serializer that transports nothing on the wire (except a boolean "false", since AMQP does not support
* absolutely nothing, or null as a described type) when we have a singleton within the node that we just
* want converting back to that singleton instance on the receiving JVM.
*/
class SingletonSerializer(override val type: Class<*>, val singleton: Any, factory: SerializerFactory) : AMQPSerializer<Any> {
override val typeDescriptor = "$DESCRIPTOR_DOMAIN:${fingerprintForType(type, factory)}"
private val interfaces = interfacesForSerialization(type)
private fun generateProvides(): List<String> = interfaces.map { it.typeName }
internal val typeNotation: TypeNotation = RestrictedType(type.typeName, "Singleton", generateProvides(), "boolean", Descriptor(typeDescriptor, null), emptyList())
override fun writeClassInfo(output: SerializationOutput) {
output.writeTypeNotations(typeNotation)
}
override fun writeObject(obj: Any, data: Data, type: Type, output: SerializationOutput) {
data.withDescribed(typeNotation.descriptor) {
data.putBoolean(false)
}
}
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): Any {
return singleton
}
}

View File

@ -0,0 +1,11 @@
package net.corda.core.serialization.amqp.custom
import net.corda.core.serialization.amqp.CustomSerializer
import java.math.BigDecimal
/**
* A serializer for [BigDecimal], utilising the string based helper. [BigDecimal] seems to have no import/export
* features that are precision independent other than via a string. The format of the string is discussed in the
* documentation for [BigDecimal.toString].
*/
object BigDecimalSerializer : CustomSerializer.ToString<BigDecimal>(BigDecimal::class.java)

View File

@ -0,0 +1,12 @@
package net.corda.core.serialization.amqp.custom
import net.corda.core.serialization.amqp.CustomSerializer
import java.util.*
/**
* A custom serializer for the [Currency] class, utilizing the currency code string representation.
*/
object CurrencySerializer : CustomSerializer.ToString<Currency>(Currency::class.java,
withInheritance = false,
maker = { Currency.getInstance(it) },
unmaker = { it.currencyCode })

View File

@ -0,0 +1,18 @@
package net.corda.core.serialization.amqp.custom
import net.corda.core.serialization.amqp.CustomSerializer
import net.corda.core.serialization.amqp.SerializerFactory
import java.time.Instant
/**
* A serializer for [Instant] that uses a proxy object to write out the seconds since the epoch and the nanos.
*/
class InstantSerializer(factory: SerializerFactory) : CustomSerializer.Proxy<Instant, InstantSerializer.InstantProxy>(Instant::class.java, InstantProxy::class.java, factory) {
override val additionalSerializers: Iterable<CustomSerializer<out Any>> = emptyList()
override fun toProxy(obj: Instant): InstantProxy = InstantProxy(obj.epochSecond, obj.nano)
override fun fromProxy(proxy: InstantProxy): Instant = Instant.ofEpochSecond(proxy.epochSeconds, proxy.nanos.toLong())
data class InstantProxy(val epochSeconds: Long, val nanos: Int)
}

View File

@ -2,23 +2,25 @@ package net.corda.core.serialization.amqp.custom
import net.corda.core.crypto.Crypto import net.corda.core.crypto.Crypto
import net.corda.core.serialization.amqp.* import net.corda.core.serialization.amqp.*
import org.apache.qpid.proton.amqp.Binary
import org.apache.qpid.proton.codec.Data import org.apache.qpid.proton.codec.Data
import java.lang.reflect.Type import java.lang.reflect.Type
import java.security.PublicKey import java.security.PublicKey
class PublicKeySerializer : CustomSerializer.Implements<PublicKey>(PublicKey::class.java) { /**
* A serializer that writes out a public key in X.509 format.
*/
object PublicKeySerializer : CustomSerializer.Implements<PublicKey>(PublicKey::class.java) {
override val additionalSerializers: Iterable<CustomSerializer<out Any>> = emptyList() override val additionalSerializers: Iterable<CustomSerializer<out Any>> = emptyList()
override val schemaForDocumentation = Schema(listOf(RestrictedType(type.toString(), "", listOf(type.toString()), SerializerFactory.primitiveTypeName(Binary::class.java)!!, descriptor, emptyList()))) override val schemaForDocumentation = Schema(listOf(RestrictedType(type.toString(), "", listOf(type.toString()), SerializerFactory.primitiveTypeName(ByteArray::class.java)!!, descriptor, emptyList())))
override fun writeDescribedObject(obj: PublicKey, data: Data, type: Type, output: SerializationOutput) { override fun writeDescribedObject(obj: PublicKey, data: Data, type: Type, output: SerializationOutput) {
// TODO: Instead of encoding to the default X509 format, we could have a custom per key type (space-efficient) serialiser. // TODO: Instead of encoding to the default X509 format, we could have a custom per key type (space-efficient) serialiser.
output.writeObject(Binary(obj.encoded), data, clazz) output.writeObject(obj.encoded, data, clazz)
} }
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): PublicKey { override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): PublicKey {
val A = input.readObject(obj, schema, ByteArray::class.java) as Binary val bits = input.readObject(obj, schema, ByteArray::class.java) as ByteArray
return Crypto.decodePublicKey(A.array) return Crypto.decodePublicKey(bits)
} }
} }

View File

@ -0,0 +1,25 @@
package net.corda.core.serialization.amqp.custom
import net.corda.core.serialization.amqp.*
import org.apache.qpid.proton.codec.Data
import org.bouncycastle.asn1.ASN1InputStream
import org.bouncycastle.asn1.x500.X500Name
import java.lang.reflect.Type
/**
* Custom serializer for X500 names that utilizes their ASN.1 encoding on the wire.
*/
object X500NameSerializer : CustomSerializer.Implements<X500Name>(X500Name::class.java) {
override val additionalSerializers: Iterable<CustomSerializer<out Any>> = emptyList()
override val schemaForDocumentation = Schema(listOf(RestrictedType(type.toString(), "", listOf(type.toString()), SerializerFactory.primitiveTypeName(ByteArray::class.java)!!, descriptor, emptyList())))
override fun writeDescribedObject(obj: X500Name, data: Data, type: Type, output: SerializationOutput) {
output.writeObject(obj.encoded, data, clazz)
}
override fun readObject(obj: Any, schema: Schema, input: DeserializationInput): X500Name {
val binary = input.readObject(obj, schema, ByteArray::class.java) as ByteArray
return X500Name.getInstance(ASN1InputStream(binary).readObject())
}
}

View File

@ -1,17 +1,26 @@
package net.corda.core.serialization.amqp package net.corda.core.serialization.amqp
import net.corda.core.contracts.*
import net.corda.core.crypto.SecureHash
import net.corda.core.flows.FlowException import net.corda.core.flows.FlowException
import net.corda.core.identity.AbstractParty
import net.corda.core.serialization.CordaSerializable import net.corda.core.serialization.CordaSerializable
import net.corda.core.serialization.EmptyWhitelist import net.corda.core.serialization.EmptyWhitelist
import net.corda.core.serialization.KryoAMQPSerializer
import net.corda.core.utilities.CordaRuntimeException
import net.corda.nodeapi.RPCException import net.corda.nodeapi.RPCException
import net.corda.testing.MEGA_CORP
import net.corda.testing.MEGA_CORP_PUBKEY import net.corda.testing.MEGA_CORP_PUBKEY
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.junit.Test import org.junit.Test
import java.io.IOException import java.io.IOException
import java.io.NotSerializableException import java.io.NotSerializableException
import java.math.BigDecimal
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.time.Instant
import java.util.* import java.util.*
import kotlin.test.assertEquals
import kotlin.test.assertNotNull import kotlin.test.assertNotNull
import kotlin.test.assertTrue import kotlin.test.assertTrue
@ -58,12 +67,38 @@ class SerializationOutputTests {
@Suppress("AddVarianceModifier") @Suppress("AddVarianceModifier")
data class GenericFoo<T>(val bar: String, val pub: T) data class GenericFoo<T>(val bar: String, val pub: T)
data class ContainsGenericFoo(val contain: GenericFoo<String>)
data class NestedGenericFoo<T>(val contain: GenericFoo<T>)
data class ContainsNestedGenericFoo(val contain: NestedGenericFoo<String>)
data class TreeMapWrapper(val tree: TreeMap<Int, Foo>) data class TreeMapWrapper(val tree: TreeMap<Int, Foo>)
data class NavigableMapWrapper(val tree: NavigableMap<Int, Foo>) data class NavigableMapWrapper(val tree: NavigableMap<Int, Foo>)
data class SortedSetWrapper(val set: SortedSet<Int>) data class SortedSetWrapper(val set: SortedSet<Int>)
open class InheritedGeneric<X>(val foo: X)
data class ExtendsGeneric(val bar: Int, val pub: String) : InheritedGeneric<String>(pub)
interface GenericInterface<X> {
val pub: X
}
data class ImplementsGenericString(val bar: Int, override val pub: String) : GenericInterface<String>
data class ImplementsGenericX<Y>(val bar: Int, override val pub: Y) : GenericInterface<Y>
abstract class AbstractGenericX<Z> : GenericInterface<Z>
data class InheritGenericX<A>(val duke: Double, override val pub: A) : AbstractGenericX<A>()
data class CapturesGenericX(val foo: GenericInterface<String>)
object KotlinObject
class Mismatch(fred: Int) { class Mismatch(fred: Int) {
val ginger: Int = fred val ginger: Int = fred
@ -85,7 +120,11 @@ class SerializationOutputTests {
data class PolymorphicProperty(val foo: FooInterface?) data class PolymorphicProperty(val foo: FooInterface?)
private fun serdes(obj: Any, factory: SerializerFactory = SerializerFactory(), freshDeserializationFactory: SerializerFactory = SerializerFactory(), expectedEqual: Boolean = true): Any { private fun serdes(obj: Any,
factory: SerializerFactory = SerializerFactory(),
freshDeserializationFactory: SerializerFactory = SerializerFactory(),
expectedEqual: Boolean = true,
expectDeserializedEqual: Boolean = true): Any {
val ser = SerializationOutput(factory) val ser = SerializationOutput(factory)
val bytes = ser.serialize(obj) val bytes = ser.serialize(obj)
@ -103,6 +142,7 @@ class SerializationOutputTests {
// Check that a vanilla AMQP decoder can deserialize without schema. // Check that a vanilla AMQP decoder can deserialize without schema.
val result = decoder.readObject() as Envelope val result = decoder.readObject() as Envelope
assertNotNull(result) assertNotNull(result)
println(result.schema)
val des = DeserializationInput(freshDeserializationFactory) val des = DeserializationInput(freshDeserializationFactory)
val desObj = des.deserialize(bytes) val desObj = des.deserialize(bytes)
@ -113,7 +153,7 @@ class SerializationOutputTests {
val des2 = DeserializationInput(factory) val des2 = DeserializationInput(factory)
val desObj2 = des2.deserialize(ser2.serialize(obj)) val desObj2 = des2.deserialize(ser2.serialize(obj))
assertTrue(Objects.deepEquals(obj, desObj2) == expectedEqual) assertTrue(Objects.deepEquals(obj, desObj2) == expectedEqual)
assertTrue(Objects.deepEquals(desObj, desObj2)) assertTrue(Objects.deepEquals(desObj, desObj2) == expectDeserializedEqual)
// TODO: add some schema assertions to check correctly formed. // TODO: add some schema assertions to check correctly formed.
return desObj2 return desObj2
@ -155,7 +195,7 @@ class SerializationOutputTests {
serdes(obj) serdes(obj)
} }
@Test @Test(expected = NotSerializableException::class)
fun `test top level list array`() { fun `test top level list array`() {
val obj = arrayOf(listOf("Fred", "Ginger"), listOf("Rogers", "Hammerstein")) val obj = arrayOf(listOf("Fred", "Ginger"), listOf("Rogers", "Hammerstein"))
serdes(obj) serdes(obj)
@ -197,12 +237,51 @@ class SerializationOutputTests {
serdes(obj) serdes(obj)
} }
@Test(expected = NotSerializableException::class) @Test
fun `test generic foo`() { fun `test generic foo`() {
val obj = GenericFoo("Fred", "Ginger") val obj = GenericFoo("Fred", "Ginger")
serdes(obj) serdes(obj)
} }
@Test
fun `test generic foo as property`() {
val obj = ContainsGenericFoo(GenericFoo("Fred", "Ginger"))
serdes(obj)
}
@Test
fun `test nested generic foo as property`() {
val obj = ContainsNestedGenericFoo(NestedGenericFoo(GenericFoo("Fred", "Ginger")))
serdes(obj)
}
// TODO: Generic interfaces / superclasses
@Test
fun `test extends generic`() {
val obj = ExtendsGeneric(1, "Ginger")
serdes(obj)
}
@Test
fun `test implements generic`() {
val obj = ImplementsGenericString(1, "Ginger")
serdes(obj)
}
@Test
fun `test implements generic captured`() {
val obj = CapturesGenericX(ImplementsGenericX(1, "Ginger"))
serdes(obj)
}
@Test
fun `test inherits generic captured`() {
val obj = CapturesGenericX(InheritGenericX(1.0, "Ginger"))
serdes(obj)
}
@Test(expected = NotSerializableException::class) @Test(expected = NotSerializableException::class)
fun `test TreeMap`() { fun `test TreeMap`() {
val obj = TreeMap<Int, Foo>() val obj = TreeMap<Int, Foo>()
@ -246,9 +325,9 @@ class SerializationOutputTests {
@Test @Test
fun `test custom serializers on public key`() { fun `test custom serializers on public key`() {
val factory = SerializerFactory() val factory = SerializerFactory()
factory.register(net.corda.core.serialization.amqp.custom.PublicKeySerializer()) factory.register(net.corda.core.serialization.amqp.custom.PublicKeySerializer)
val factory2 = SerializerFactory() val factory2 = SerializerFactory()
factory2.register(net.corda.core.serialization.amqp.custom.PublicKeySerializer()) factory2.register(net.corda.core.serialization.amqp.custom.PublicKeySerializer)
val obj = MEGA_CORP_PUBKEY val obj = MEGA_CORP_PUBKEY
serdes(obj, factory, factory2) serdes(obj, factory, factory2)
} }
@ -267,8 +346,9 @@ class SerializationOutputTests {
val factory2 = SerializerFactory() val factory2 = SerializerFactory()
factory2.register(net.corda.core.serialization.amqp.custom.ThrowableSerializer(factory2)) factory2.register(net.corda.core.serialization.amqp.custom.ThrowableSerializer(factory2))
val obj = IllegalAccessException("message").fillInStackTrace() val t = IllegalAccessException("message").fillInStackTrace()
serdes(obj, factory, factory2, false) val desThrowable = serdes(t, factory, factory2, false) as Throwable
assertSerializedThrowableEquivalent(t, desThrowable)
} }
@Test @Test
@ -286,7 +366,19 @@ class SerializationOutputTests {
throw IllegalStateException("Layer 2", t) throw IllegalStateException("Layer 2", t)
} }
} catch(t: Throwable) { } catch(t: Throwable) {
serdes(t, factory, factory2, false) val desThrowable = serdes(t, factory, factory2, false) as Throwable
assertSerializedThrowableEquivalent(t, desThrowable)
}
}
fun assertSerializedThrowableEquivalent(t: Throwable, desThrowable: Throwable) {
assertTrue(desThrowable is CordaRuntimeException) // Since we don't handle the other case(s) yet
if (desThrowable is CordaRuntimeException) {
assertEquals("${t.javaClass.name}: ${t.message}", desThrowable.message)
assertTrue(desThrowable is CordaRuntimeException)
assertTrue(Objects.deepEquals(t.stackTrace, desThrowable.stackTrace))
assertEquals(t.suppressed.size, desThrowable.suppressed.size)
t.suppressed.zip(desThrowable.suppressed).forEach { (before, after) -> assertSerializedThrowableEquivalent(before, after) }
} }
} }
@ -307,7 +399,8 @@ class SerializationOutputTests {
throw e throw e
} }
} catch(t: Throwable) { } catch(t: Throwable) {
serdes(t, factory, factory2, false) val desThrowable = serdes(t, factory, factory2, false) as Throwable
assertSerializedThrowableEquivalent(t, desThrowable)
} }
} }
@ -347,4 +440,88 @@ class SerializationOutputTests {
serdes(obj) serdes(obj)
} }
@Test
fun `test kotlin object`() {
serdes(KotlinObject)
}
object FooContract : Contract {
override fun verify(tx: TransactionForContract) {
}
override val legalContractReference: SecureHash = SecureHash.Companion.sha256("FooContractLegal")
}
class FooState : ContractState {
override val contract: Contract
get() = FooContract
override val participants: List<AbstractParty>
get() = emptyList()
}
@Test
fun `test transaction state`() {
val state = TransactionState<FooState>(FooState(), MEGA_CORP)
val factory = SerializerFactory()
KryoAMQPSerializer.registerCustomSerializers(factory)
val factory2 = SerializerFactory()
KryoAMQPSerializer.registerCustomSerializers(factory2)
val desState = serdes(state, factory, factory2, expectedEqual = false, expectDeserializedEqual = false)
assertTrue(desState is TransactionState<*>)
assertTrue((desState as TransactionState<*>).data is FooState)
assertTrue(desState.notary == state.notary)
assertTrue(desState.encumbrance == state.encumbrance)
}
@Test
fun `test currencies serialize`() {
val factory = SerializerFactory()
factory.register(net.corda.core.serialization.amqp.custom.CurrencySerializer)
val factory2 = SerializerFactory()
factory2.register(net.corda.core.serialization.amqp.custom.CurrencySerializer)
val obj = Currency.getInstance("USD")
serdes(obj, factory, factory2)
}
@Test
fun `test big decimals serialize`() {
val factory = SerializerFactory()
factory.register(net.corda.core.serialization.amqp.custom.BigDecimalSerializer)
val factory2 = SerializerFactory()
factory2.register(net.corda.core.serialization.amqp.custom.BigDecimalSerializer)
val obj = BigDecimal("100000000000000000000000000000.00")
serdes(obj, factory, factory2)
}
@Test
fun `test instants serialize`() {
val factory = SerializerFactory()
factory.register(net.corda.core.serialization.amqp.custom.InstantSerializer(factory))
val factory2 = SerializerFactory()
factory2.register(net.corda.core.serialization.amqp.custom.InstantSerializer(factory2))
val obj = Instant.now()
serdes(obj, factory, factory2)
}
@Test
fun `test StateRef serialize`() {
val factory = SerializerFactory()
factory.register(net.corda.core.serialization.amqp.custom.InstantSerializer(factory))
val factory2 = SerializerFactory()
factory2.register(net.corda.core.serialization.amqp.custom.InstantSerializer(factory2))
val obj = StateRef(SecureHash.randomSHA256(), 0)
serdes(obj, factory, factory2)
}
} }