You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sd...@apache.org on 2022/01/11 23:56:36 UTC

[ignite-3] 08/27: - make sure Serializable constructor is not called on unmarshalling, while first non-serializable parent constructor gets called

This is an automated email from the ASF dual-hosted git repository.

sdanilov pushed a commit to branch uos-network
in repository https://gitbox.apache.org/repos/asf/ignite-3.git

commit 5d77e863d0964abd17a50672d903ae184a481acc
Author: Roman Puchkovskiy <ro...@gmail.com>
AuthorDate: Tue Jan 11 11:38:27 2022 +0400

    - make sure Serializable constructor is not called on unmarshalling, while first non-serializable parent constructor gets called
---
 .../marshal/ArbitraryObjectMarshaller.java         |  6 +-
 .../marshal/DefaultUserObjectMarshaller.java       |  6 +-
 .../marshal/SerializableInstantiation.java         | 99 ++++++----------------
 .../marshal/SerializableInstantiationTest.java     | 36 ++------
 4 files changed, 36 insertions(+), 111 deletions(-)

diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/ArbitraryObjectMarshaller.java b/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/ArbitraryObjectMarshaller.java
index 3ec8eeb..1ad2e39 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/ArbitraryObjectMarshaller.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/ArbitraryObjectMarshaller.java
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import org.apache.ignite.internal.network.serialization.ClassDescriptor;
-import org.apache.ignite.internal.network.serialization.ClassIndexedDescriptors;
 import org.apache.ignite.internal.network.serialization.FieldDescriptor;
 import org.apache.ignite.internal.network.serialization.SpecialMethodInvocationException;
 
@@ -38,13 +37,12 @@ class ArbitraryObjectMarshaller implements DefaultFieldsReaderWriter {
 
     private final Instantiation instantiation;
 
-    ArbitraryObjectMarshaller(ClassIndexedDescriptors descriptors, TypedValueWriter valueWriter, ValueReader<Object> valueReader) {
+    ArbitraryObjectMarshaller(TypedValueWriter valueWriter, ValueReader<Object> valueReader) {
         this.valueWriter = valueWriter;
         this.valueReader = valueReader;
 
         instantiation = new BestEffortInstantiation(
-                new NoArgConstructorInstantiation(),
-                new SerializableInstantiation(descriptors),
+                new SerializableInstantiation(),
                 new UnsafeInstantiation()
         );
     }
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/DefaultUserObjectMarshaller.java b/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/DefaultUserObjectMarshaller.java
index 82c3973..e1dc6a1 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/DefaultUserObjectMarshaller.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/DefaultUserObjectMarshaller.java
@@ -65,11 +65,7 @@ public class DefaultUserObjectMarshaller implements UserObjectMarshaller {
         this.descriptorRegistry = descriptorRegistry;
         this.descriptorFactory = descriptorFactory;
 
-        arbitraryObjectMarshaller = new ArbitraryObjectMarshaller(
-                descriptorRegistry,
-                this::marshalToOutput,
-                this::unmarshalFromInput
-        );
+        arbitraryObjectMarshaller = new ArbitraryObjectMarshaller(this::marshalToOutput, this::unmarshalFromInput);
 
         externalizableMarshaller = new ExternalizableMarshaller(this::unmarshalFromInput, this::marshalToOutput, arbitraryObjectMarshaller);
     }
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiation.java b/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiation.java
index 665bfb1..f31448d 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiation.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiation.java
@@ -17,99 +17,56 @@
 
 package org.apache.ignite.internal.network.serialization.marshal;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.io.ObjectStreamClass;
-import java.io.ObjectStreamConstants;
 import java.io.Serializable;
-import org.apache.ignite.internal.network.serialization.ClassDescriptor;
-import org.apache.ignite.internal.network.serialization.ClassIndexedDescriptors;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
 
 /**
- * Instantiates {@link Serializable} classes (they are the only ones supported) by crafting a representation of
- * a serialized object of a given class (without any field data) and then deserializing it using the standard
- * Java Serialization.
+ * Instantiates {@link Serializable} classes using the mechanism defined by Java Serialization. That is,
+ * for an {@link java.io.Externalizable}, its no-arg constructor is invoked. For a non-Externalizable {@link Serializable},
+ * a new constructor is generated that invokes the no-arg constructor of the deepest non-serializable ancestor in the hierarchy.
  */
 class SerializableInstantiation implements Instantiation {
 
-    private static final int STREAM_VERSION = 5;
+    private static final MethodHandle STREAM_CLASS_NEW_INSTANCE;
 
-    private final ClassIndexedDescriptors descriptors;
+    static {
+        try {
+            STREAM_CLASS_NEW_INSTANCE = streamClassNewInstanceMethodHandle();
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
 
-    SerializableInstantiation(ClassIndexedDescriptors descriptors) {
-        this.descriptors = descriptors;
+    private static MethodHandle streamClassNewInstanceMethodHandle() throws NoSuchMethodException, IllegalAccessException {
+        MethodHandles.Lookup lookup = MethodHandles.privateLookupIn(ObjectStreamClass.class, MethodHandles.lookup());
+        return lookup.findVirtual(ObjectStreamClass.class, "newInstance", MethodType.methodType(Object.class));
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean supports(Class<?> objectClass) {
-        if (!Serializable.class.isAssignableFrom(objectClass)) {
-            return false;
-        }
-
-        ClassDescriptor descriptor = descriptors.getRequiredDescriptor(objectClass);
-        return !descriptor.hasWriteReplace() && !descriptor.hasReadResolve();
+        return Serializable.class.isAssignableFrom(objectClass);
     }
 
     /** {@inheritDoc} */
     @Override
     public Object newInstance(Class<?> objectClass) throws InstantiationException {
-        byte[] jdkSerialization = jdkSerializationOfEmptyInstanceOf(objectClass);
-
-        try (ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(jdkSerialization))) {
-            return ois.readObject();
-        } catch (IOException | ClassNotFoundException e) {
-            throw new InstantiationException("Cannot deserialize JDK serialization of an empty instance", e);
-        }
-    }
-
-    private byte[] jdkSerializationOfEmptyInstanceOf(Class<?> objectClass) throws InstantiationException {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        try (DataOutputStream dos = new DataOutputStream(baos)) {
-            writeSignature(dos);
-
-            dos.writeByte(ObjectStreamConstants.TC_OBJECT);
-            dos.writeByte(ObjectStreamConstants.TC_CLASSDESC);
-            dos.writeUTF(objectClass.getName());
-
-            dos.writeLong(serialVersionUid(objectClass));
+        // Using the standard machinery (ObjectStreamClass) to instantiate an object to avoid generating excessive constructors
+        // (as the standard machinery caches the constructors effectively).
 
-            writeFlags(dos);
+        ObjectStreamClass desc = ObjectStreamClass.lookup(objectClass);
 
-            writeZeroFields(dos);
+        // But as ObjectStreamClass#newInstance() is package-local, we have to resort to reflection/method handles magic.
 
-            dos.writeByte(ObjectStreamConstants.TC_ENDBLOCKDATA);
-            writeNullForNoParentDescriptor(dos);
-        } catch (IOException e) {
-            throw new InstantiationException("Cannot create JDK serialization of an empty instance", e);
+        try {
+            return STREAM_CLASS_NEW_INSTANCE.invokeExact(desc);
+        } catch (Error e) {
+            throw e;
+        } catch (Throwable e) {
+            throw new InstantiationException("Cannot instantiate", e);
         }
-
-        return baos.toByteArray();
-    }
-
-    private void writeSignature(DataOutputStream dos) throws IOException {
-        dos.writeShort(ObjectStreamConstants.STREAM_MAGIC);
-        dos.writeShort(STREAM_VERSION);
-    }
-
-    private long serialVersionUid(Class<?> objectClass) {
-        ObjectStreamClass descriptor = ObjectStreamClass.lookup(objectClass);
-        return descriptor.getSerialVersionUID();
-    }
-
-    private void writeFlags(DataOutputStream dos) throws IOException {
-        dos.writeByte(ObjectStreamConstants.SC_SERIALIZABLE);
-    }
-
-    private void writeZeroFields(DataOutputStream dos) throws IOException {
-        dos.writeShort(0);
-    }
-
-    private void writeNullForNoParentDescriptor(DataOutputStream dos) throws IOException {
-        dos.writeByte(ObjectStreamConstants.TC_NULL);
     }
 }
diff --git a/modules/network/src/test/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiationTest.java b/modules/network/src/test/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiationTest.java
index fbcb148..c0dbe03 100644
--- a/modules/network/src/test/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiationTest.java
+++ b/modules/network/src/test/java/org/apache/ignite/internal/network/serialization/marshal/SerializableInstantiationTest.java
@@ -22,17 +22,10 @@ import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.notNullValue;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.lenient;
 
 import java.io.Serializable;
-import org.apache.ignite.internal.network.serialization.ClassDescriptor;
-import org.apache.ignite.internal.network.serialization.ClassIndexedDescriptors;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
-import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
 /**
@@ -40,22 +33,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
  */
 @ExtendWith(MockitoExtension.class)
 class SerializableInstantiationTest {
-    @Mock
-    private ClassIndexedDescriptors descriptors;
-    @Mock
-    private ClassDescriptor descriptor;
-
-    private Instantiation instantiation;
-
-    @BeforeEach
-    void initMocks() {
-        lenient().when(descriptors.getRequiredDescriptor(any())).thenReturn(descriptor);
-    }
-
-    @BeforeEach
-    void createInstantiation() {
-        instantiation = new SerializableInstantiation(descriptors);
-    }
+    private final Instantiation instantiation = new SerializableInstantiation();
 
     @Test
     void doesNotSupportNonSerializableClasses() {
@@ -68,17 +46,13 @@ class SerializableInstantiationTest {
     }
 
     @Test
-    void doesNotSupportClassesWithWriteReplace() {
-        doReturn(true).when(descriptor).hasWriteReplace();
-
-        assertFalse(instantiation.supports(WithWriteReplace.class));
+    void supportsClassesWithWriteReplace() {
+        assertTrue(instantiation.supports(WithWriteReplace.class));
     }
 
     @Test
-    void doesNotSupportClassesWithReadResolve() {
-        doReturn(true).when(descriptor).hasReadResolve();
-
-        assertFalse(instantiation.supports(WithReadResolve.class));
+    void supportsClassesWithReadResolve() {
+        assertTrue(instantiation.supports(WithReadResolve.class));
     }
 
     // TODO: IGNITE-16165 - test that it does not support instantiation of Serializable classes with writeObject()/readObject()