Merge remote-tracking branch 'origin/release/os/4.6' into christians/ENT-5273-update-from-os-4.6

This commit is contained in:
Christian Sailer
2020-07-23 14:02:18 +01:00
55 changed files with 1579 additions and 292 deletions

View File

@ -0,0 +1,103 @@
package net.corda.nodeapi.internal.serialization.kryo
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.CheckpointCustomSerializer
import net.corda.serialization.internal.amqp.CORDAPP_TYPE
import java.lang.reflect.Type
import kotlin.reflect.jvm.javaType
import kotlin.reflect.jvm.jvmErasure
/**
* Adapts CheckpointCustomSerializer for use in Kryo
*/
internal class CustomSerializerCheckpointAdaptor<OBJ, PROXY>(private val userSerializer : CheckpointCustomSerializer<OBJ, PROXY>) : Serializer<OBJ>() {
/**
* The class name of the serializer we are adapting.
*/
val serializerName: String = userSerializer.javaClass.name
/**
* The input type of this custom serializer.
*/
val cordappType: Type
/**
* Check we have access to the types specified on the CheckpointCustomSerializer interface.
*
* Throws UnableToDetermineSerializerTypesException if the types are missing.
*/
init {
val types: List<Type> = userSerializer::class
.supertypes
.filter { it.jvmErasure == CheckpointCustomSerializer::class }
.flatMap { it.arguments }
.mapNotNull { it.type?.javaType }
// We are expecting a cordapp type and a proxy type.
// We will only use the cordapp type in this class
// but we want to check both are present.
val typeParameterCount = 2
if (types.size != typeParameterCount) {
throw UnableToDetermineSerializerTypesException("Unable to determine serializer parent types")
}
cordappType = types[CORDAPP_TYPE]
}
/**
* Serialize obj to the Kryo stream.
*/
override fun write(kryo: Kryo, output: Output, obj: OBJ) {
fun <T> writeToKryo(obj: T) = kryo.writeClassAndObject(output, obj)
// Write serializer type
writeToKryo(serializerName)
// Write proxy object
writeToKryo(userSerializer.toProxy(obj))
}
/**
* Deserialize an object from the Kryo stream.
*/
override fun read(kryo: Kryo, input: Input, type: Class<OBJ>): OBJ {
@Suppress("UNCHECKED_CAST")
fun <T> readFromKryo() = kryo.readClassAndObject(input) as T
// Check the serializer type
checkSerializerType(readFromKryo())
// Read the proxy object
return userSerializer.fromProxy(readFromKryo())
}
/**
* Throws a `CustomCheckpointSerializersHaveChangedException` if the serializer type in the kryo stream does not match the serializer
* type for this custom serializer.
*
* @param checkpointSerializerType Serializer type from the Kryo stream
*/
private fun checkSerializerType(checkpointSerializerType: String) {
if (checkpointSerializerType != serializerName)
throw CustomCheckpointSerializersHaveChangedException("The custom checkpoint serializers have changed while checkpoints exist. " +
"Please restore the CorDapps to when this checkpoint was created.")
}
}
/**
* Thrown when the input/output types are missing from the custom serializer.
*/
class UnableToDetermineSerializerTypesException(message: String) : RuntimeException(message)
/**
* Thrown when the custom serializer is found to be reading data from another type of custom serializer.
*
* This was expected to happen if the user adds or removes CorDapps while checkpoints exist but it turned out that registering serializers
* as default made the system reliable.
*/
class CustomCheckpointSerializersHaveChangedException(message: String) : RuntimeException(message)

View File

@ -10,12 +10,14 @@ import com.esotericsoftware.kryo.io.Output
import com.esotericsoftware.kryo.pool.KryoPool
import com.esotericsoftware.kryo.serializers.ClosureSerializer
import net.corda.core.internal.uncheckedCast
import net.corda.core.serialization.CheckpointCustomSerializer
import net.corda.core.serialization.ClassWhitelist
import net.corda.core.serialization.SerializationDefaults
import net.corda.core.serialization.SerializedBytes
import net.corda.core.serialization.internal.CheckpointSerializationContext
import net.corda.core.serialization.internal.CheckpointSerializer
import net.corda.core.utilities.ByteSequence
import net.corda.core.utilities.loggerFor
import net.corda.serialization.internal.AlwaysAcceptEncodingWhitelist
import net.corda.serialization.internal.ByteBufferInputStream
import net.corda.serialization.internal.CheckpointSerializationContextImpl
@ -40,10 +42,10 @@ private object AutoCloseableSerialisationDetector : Serializer<AutoCloseable>()
}
object KryoCheckpointSerializer : CheckpointSerializer {
private val kryoPoolsForContexts = ConcurrentHashMap<Pair<ClassWhitelist, ClassLoader>, KryoPool>()
private val kryoPoolsForContexts = ConcurrentHashMap<Triple<ClassWhitelist, ClassLoader, Iterable<CheckpointCustomSerializer<*,*>>>, KryoPool>()
private fun getPool(context: CheckpointSerializationContext): KryoPool {
return kryoPoolsForContexts.computeIfAbsent(Pair(context.whitelist, context.deserializationClassLoader)) {
return kryoPoolsForContexts.computeIfAbsent(Triple(context.whitelist, context.deserializationClassLoader, context.checkpointCustomSerializers)) {
KryoPool.Builder {
val serializer = Fiber.getFiberSerializer(false) as KryoSerializer
val classResolver = CordaClassResolver(context).apply { setKryo(serializer.kryo) }
@ -56,12 +58,60 @@ object KryoCheckpointSerializer : CheckpointSerializer {
addDefaultSerializer(AutoCloseable::class.java, AutoCloseableSerialisationDetector)
register(ClosureSerializer.Closure::class.java, CordaClosureSerializer)
classLoader = it.second
// Add custom serializers
val customSerializers = buildCustomSerializerAdaptors(context)
warnAboutDuplicateSerializers(customSerializers)
val classToSerializer = mapInputClassToCustomSerializer(context.deserializationClassLoader, customSerializers)
addDefaultCustomSerializers(this, classToSerializer)
}
}.build()
}
}
/**
* Returns a sorted list of CustomSerializerCheckpointAdaptor based on the custom serializers inside context.
*
* The adaptors are sorted by serializerName which maps to javaClass.name for the serializer class
*/
private fun buildCustomSerializerAdaptors(context: CheckpointSerializationContext) =
context.checkpointCustomSerializers.map { CustomSerializerCheckpointAdaptor(it) }.sortedBy { it.serializerName }
/**
* Returns a list of pairs where the first element is the input class of the custom serializer and the second element is the
* custom serializer.
*/
private fun mapInputClassToCustomSerializer(classLoader: ClassLoader, customSerializers: Iterable<CustomSerializerCheckpointAdaptor<*, *>>) =
customSerializers.map { getInputClassForCustomSerializer(classLoader, it) to it }
/**
* Returns the Class object for the serializers input type.
*/
private fun getInputClassForCustomSerializer(classLoader: ClassLoader, customSerializer: CustomSerializerCheckpointAdaptor<*, *>): Class<*> {
val typeNameWithoutGenerics = customSerializer.cordappType.typeName.substringBefore('<')
return classLoader.loadClass(typeNameWithoutGenerics)
}
/**
* Emit a warning if two or more custom serializers are found for the same input type.
*/
private fun warnAboutDuplicateSerializers(customSerializers: Iterable<CustomSerializerCheckpointAdaptor<*,*>>) =
customSerializers
.groupBy({ it.cordappType }, { it.serializerName })
.filter { (_, serializerNames) -> serializerNames.distinct().size > 1 }
.forEach { (inputType, serializerNames) -> loggerFor<KryoCheckpointSerializer>().warn("Duplicate custom checkpoint serializer for type $inputType. Serializers: ${serializerNames.joinToString(", ")}") }
/**
* Register all custom serializers as default, this class + subclass, registrations.
*
* Serializers registered before this will take priority. This needs to run after registrations we want to keep otherwise it may
* replace them.
*/
private fun addDefaultCustomSerializers(kryo: Kryo, classToSerializer: Iterable<Pair<Class<*>, CustomSerializerCheckpointAdaptor<*, *>>>) =
classToSerializer
.forEach { (clazz, customSerializer) -> kryo.addDefaultSerializer(clazz, customSerializer) }
private fun <T : Any> CheckpointSerializationContext.kryo(task: Kryo.() -> T): T {
return getPool(this).run { kryo ->
kryo.context.ensureCapacity(properties.size)