You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/12/11 14:21:37 UTC

[GitHub] [ignite-3] sashapolo commented on a change in pull request #495: IGNITE-15940 Class descriptor parser

sashapolo commented on a change in pull request #495:
URL: https://github.com/apache/ignite-3/pull/495#discussion_r767153391



##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptor.java
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.util.List;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Class descriptor for the user object serialization.
+ */
+public class ClassDescriptor {
+    /**
+     * Name of the class.
+     */
+    @NotNull
+    private final String className;
+
+    /**
+     * Class.
+     */
+    @NotNull
+    private final Class<?> clazz;
+
+    /**
+     * Descriptor id.
+     */
+    private final int descriptorId;
+
+    /**
+     * List of the classes fields' descriptors.
+     */
+    @NotNull
+    private final List<FieldDescriptor> fields;
+
+    /**
+     * The type of the serialization mechanism for the class.
+     */
+    @NotNull
+    private final SerializationType serializationType;
+
+    /**
+     * Whether the class is final.
+     */
+    private final boolean isFinal;
+
+    /**
+     * Constructor.
+     */
+    public ClassDescriptor(@NotNull String className, @NotNull Class<?> clazz, int descriptorId,
+            @NotNull List<FieldDescriptor> fields, @NotNull SerializationType serializationType,
+            boolean isFinal) {
+        this.className = className;
+        this.clazz = clazz;
+        this.descriptorId = descriptorId;
+        this.fields = List.copyOf(fields);
+        this.serializationType = serializationType;
+        this.isFinal = isFinal;
+    }
+
+    /**
+     * Returns descriptor id.
+     *
+     * @return Descriptor id.
+     */
+    public int descriptorId() {
+        return descriptorId;
+    }
+
+    /**
+     * Returns fields' descriptors.
+     *
+     * @return Fields' descriptors.
+     */
+    @NotNull
+    public List<FieldDescriptor> fields() {
+        return fields;
+    }
+
+    /**
+     * Returns classes name.

Review comment:
       ```suggestion
        * Returns class' name.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptor.java
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.util.List;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Class descriptor for the user object serialization.
+ */
+public class ClassDescriptor {
+    /**
+     * Name of the class.
+     */
+    @NotNull
+    private final String className;
+
+    /**
+     * Class.
+     */
+    @NotNull
+    private final Class<?> clazz;
+
+    /**
+     * Descriptor id.
+     */
+    private final int descriptorId;
+
+    /**
+     * List of the classes fields' descriptors.
+     */
+    @NotNull
+    private final List<FieldDescriptor> fields;
+
+    /**
+     * The type of the serialization mechanism for the class.
+     */
+    @NotNull
+    private final SerializationType serializationType;
+
+    /**
+     * Whether the class is final.
+     */
+    private final boolean isFinal;
+
+    /**
+     * Constructor.
+     */
+    public ClassDescriptor(@NotNull String className, @NotNull Class<?> clazz, int descriptorId,
+            @NotNull List<FieldDescriptor> fields, @NotNull SerializationType serializationType,
+            boolean isFinal) {
+        this.className = className;
+        this.clazz = clazz;
+        this.descriptorId = descriptorId;
+        this.fields = List.copyOf(fields);
+        this.serializationType = serializationType;
+        this.isFinal = isFinal;
+    }
+
+    /**
+     * Returns descriptor id.
+     *
+     * @return Descriptor id.
+     */
+    public int descriptorId() {
+        return descriptorId;
+    }
+
+    /**
+     * Returns fields' descriptors.
+     *
+     * @return Fields' descriptors.
+     */
+    @NotNull
+    public List<FieldDescriptor> fields() {
+        return fields;
+    }
+
+    /**
+     * Returns classes name.
+     *
+     * @return Classes name.

Review comment:
       ```suggestion
        * @return Class' name.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {

Review comment:
       This method should be annotated with `@Nullable`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {

Review comment:
       This also fits on one line

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),

Review comment:
       Do I understand correctly that we still need the fields because even thought the class is `Serializable`, we will still use our own serialization for them?

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {

Review comment:
       I think using streams here will be much shorter:
   ```
   return Arrays.stream(clazz.getDeclaredFields())
           .filter(field -> {
               int modifiers = field.getModifiers();
   
               return !Modifier.isStatic(modifiers) && !Modifier.isTransient(modifiers);
           })
           .map(field -> {
               Class<?> type = field.getType();
   
               return new FieldDescriptor(field.getName(), type, context.getId(type));
           })
           .collect(Collectors.toList());
   ```
   
   The current code is fine, though, you can leave it if you like it        

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {

Review comment:
       fits on one lime -_-

##########
File path: modules/network/src/test/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParserTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DefaultType;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Set of tests for the class descriptor parser.
+ */
+public class ClassDescriptorParserTest {
+    /**
+     * Descriptor parser context.
+     */
+    private final ClassDescriptorParserContext context = new ClassDescriptorParserContext();
+
+    /**
+     * Descriptor parser.
+     */
+    private final ClassDescriptorParser parser = new ClassDescriptorParser(context);
+
+    static class SerializableClass implements Serializable {
+        private static final long serialVersionUID = 0L;
+    }
+
+    @Test
+    public void testSerializable() {
+        ClassDescriptor descriptor = parser.parse(SerializableClass.class);
+
+        assertEquals(SerializationType.SERIALIZABLE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableOverride implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private void writeObject(ObjectOutputStream out) throws IOException {
+            // No-op.
+        }
+
+        private void readObject(ObjectInputStream in)
+                throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+
+        private void readObjectNoData() throws ObjectStreamException {
+            // No-op.
+        }
+    }
+
+    @Test
+    public void testSerializableOverride() {
+        ClassDescriptor descriptor = parser.parse(SerializableOverride.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_OVERRIDE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableOverrideWriteReplace implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private void writeObject(ObjectOutputStream out) throws IOException {
+            // No-op.
+        }
+
+        private void readObject(ObjectInputStream in)
+                throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+
+        private void readObjectNoData() throws ObjectStreamException {
+            // No-op.
+        }
+
+        private Object writeReplace() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+    }
+
+    @Test
+    public void testSerializableOverrideWriteReplace() {
+        ClassDescriptor descriptor = parser.parse(SerializableOverrideWriteReplace.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE,
+                descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableOverrideReadResolve implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private void writeObject(ObjectOutputStream out) throws IOException {
+            // No-op.
+        }
+
+        private void readObject(ObjectInputStream in)
+                throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+
+        private void readObjectNoData() throws ObjectStreamException {
+            // No-op.
+        }
+
+        private Object readResolve() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+    }
+
+    @Test
+    public void testSerializableOverrideReadResolve() {
+        ClassDescriptor descriptor = parser.parse(SerializableOverrideReadResolve.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE,
+                descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableOverrideWriteReplaceReadResolve implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private void writeObject(ObjectOutputStream out) throws IOException {
+            // No-op.
+        }
+
+        private void readObject(ObjectInputStream in)
+                throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+
+        private void readObjectNoData() throws ObjectStreamException {
+            // No-op.
+        }
+
+        private Object writeReplace() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+
+        private Object readResolve() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+    }
+
+    @Test
+    public void testSerializableOverrideWriteReplaceReadResolve() {
+        ClassDescriptor descriptor = parser.parse(
+                SerializableOverrideWriteReplaceReadResolve.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE,
+                descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableWriteReplace implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private Object writeReplace() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+    }
+
+    @Test
+    public void testSerializableWriteReplace() {
+        ClassDescriptor descriptor = parser.parse(SerializableWriteReplace.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_WRITE_REPLACE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    @SuppressWarnings("CheckStyle")
+    static class SerializableReadResolve implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private Object readResolve() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+    }
+
+    @Test
+    public void testSerializableReadResolve() {
+        ClassDescriptor descriptor = parser.parse(SerializableReadResolve.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_READ_RESOLVE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableWriteReplaceReadResolve implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private Object writeReplace() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+
+        private Object readResolve() throws ObjectStreamException {
+            // No-op.
+            return null;
+        }
+    }
+
+    @Test
+    public void testSerializableWriteReplaceReadResolve() {
+        ClassDescriptor descriptor = parser.parse(SerializableWriteReplaceReadResolve.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE,
+                descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    @SuppressWarnings("RedundantThrows")
+    static class ExternalizableClass implements Externalizable {
+        public ExternalizableClass() {
+        }
+
+        @Override
+        public void writeExternal(ObjectOutput out) throws IOException {
+            // No-op.
+        }
+
+        @Override
+        public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+    }
+
+    @Test
+    public void testExternalizable() {
+        ClassDescriptor descriptor = parser.parse(ExternalizableClass.class);
+
+        assertEquals(SerializationType.EXTERNALIZABLE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class ArbitraryClass {
+    }
+
+    @Test
+    public void testArbitrary() {
+        ClassDescriptor descriptor = parser.parse(ArbitraryClass.class);
+
+        assertEquals(SerializationType.ARBITRARY, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    @SuppressWarnings({"ExternalizableWithoutPublicNoArgConstructor", "RedundantThrows"})
+    static class ExternalizableWithoutPublicNoArgConstructor implements Externalizable {
+        public ExternalizableWithoutPublicNoArgConstructor(@SuppressWarnings("unused") int unused) {
+        }
+
+        @Override
+        public void writeExternal(ObjectOutput out) throws IOException {
+            // No-op.
+        }
+
+        @Override
+        public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+    }
+
+    @Test
+    public void testExternalizableWithoutConstructor() {
+        assertThrows(IgniteException.class,
+                () -> parser.parse(ExternalizableWithoutPublicNoArgConstructor.class));
+    }
+
+    /** Class that holds a set of all types parseable by the class descriptor parser. */
+    @SuppressWarnings("unused")
+    static class Holder {
+        private SerializableClass field1;
+
+        private ExternalizableClass field2;
+
+        private ArbitraryClass field3;
+
+        private int primitiveField;
+    }
+
+    @Test
+    public void testHolderClass() {
+        ClassDescriptor holderDescriptor = parser.parse(Holder.class);
+
+        ClassDescriptor serializableDescriptor = context.getDescriptor(SerializableClass.class);
+        assertNotNull(serializableDescriptor);
+
+        ClassDescriptor externalizableDescriptor = context.getDescriptor(ExternalizableClass.class);
+        assertNotNull(externalizableDescriptor);
+
+        ClassDescriptor arbitraryDescriptor = context.getDescriptor(ArbitraryClass.class);
+        assertNotNull(arbitraryDescriptor);
+
+        ClassDescriptor intDescriptor = context.getDescriptor(DefaultType.INT.descriptorId());
+        assertNotNull(intDescriptor);
+
+        List<FieldDescriptor> fields = holderDescriptor.fields();
+        assertEquals(4, fields.size());
+
+        Map<String, FieldDescriptor> fieldsMap = fields.stream()

Review comment:
       Why do you need a map here? I think using an array is enough

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {

Review comment:
       This method should be annotated with `@Nullable`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptor.java
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.util.List;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Class descriptor for the user object serialization.
+ */
+public class ClassDescriptor {
+    /**
+     * Name of the class.
+     */
+    @NotNull
+    private final String className;
+
+    /**
+     * Class.
+     */
+    @NotNull
+    private final Class<?> clazz;
+
+    /**
+     * Descriptor id.
+     */
+    private final int descriptorId;
+
+    /**
+     * List of the classes fields' descriptors.

Review comment:
       ```suggestion
        * List of the class fields' descriptors.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)

Review comment:
       ```suggestion
       private static void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/SerializationType.java
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DefaultType;
+
+/**
+ * Serialization type.
+ */
+public enum SerializationType {

Review comment:
       Can we use bit flags instead? This way you will not have to create enums for combinations like `Serializable with readObject/writeObject and writeReplace`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {

Review comment:
       I would suggest to call this class `ClassDescriptorFactory`, since it creates descriptors based on a class.

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {

Review comment:
       this fits on one line

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());

Review comment:
       you should use `ArrayDeque` instead

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObjectNoData");
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the method's declaration contains throwing only of
+     * specified exceptions.
+     *
+     * @param method Method.
+     * @param exceptions List of exceptions.
+     * @return If the method throws exceptions.
+     */
+    @SuppressWarnings("BooleanMethodIsAlwaysInverted")
+    private boolean methodDeclaresExactExceptions(Method method,
+            List<Class<? extends Throwable>> exceptions) {
+        Class<?>[] exceptionTypes = method.getExceptionTypes();
+
+        if (exceptionTypes.length != exceptions.size()) {
+            return false;
+        }
+
+        return Arrays.asList(exceptionTypes).containsAll(exceptions);
+    }
+
+    /**
+     * Gets classes name.
+     *
+     * @param clazz Class.
+     * @return Classes name.
+     */
+    public static String className(Class<?> clazz) {
+        return clazz.getCanonicalName();

Review comment:
       `canonicalName` can return `null`, maybe we should handle this case

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);

Review comment:
       you can put this line under `else`, that would look a little more organic =)

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/DefaultClassDescriptors.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Class that holds default types and creates descriptors for them.
+ */
+public class DefaultClassDescriptors {
+    /** Quantity of descriptor ids reserved for the default descriptors. */
+    public static final int DEFAULT_DESCRIPTORS_OFFSET_COUNT = 1000;
+
+    /**
+     * Creates a descriptor for one of the default types.
+     *
+     * @param type Default type.
+     * @return Descriptor.
+     */
+    public static ClassDescriptor createDefaultDescriptor(DefaultType type) {

Review comment:
       I would suggest moving this method to `ClassDescriptorParser` (especially if we will rename it to `Factory`), that will make the design cleaner. We can even get rid of this class altogether.

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {

Review comment:
       Why are these cases actually important? Are they going to be used to validate descriptor compatibility?

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {

Review comment:
       do these methods have to return `Method`? Can they return a `boolean` instead?

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptor.java
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.util.List;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Class descriptor for the user object serialization.
+ */
+public class ClassDescriptor {
+    /**
+     * Name of the class.
+     */
+    @NotNull
+    private final String className;
+
+    /**
+     * Class.
+     */
+    @NotNull
+    private final Class<?> clazz;
+
+    /**
+     * Descriptor id.
+     */
+    private final int descriptorId;
+
+    /**
+     * List of the classes fields' descriptors.
+     */
+    @NotNull
+    private final List<FieldDescriptor> fields;
+
+    /**
+     * The type of the serialization mechanism for the class.
+     */
+    @NotNull
+    private final SerializationType serializationType;
+
+    /**
+     * Whether the class is final.
+     */
+    private final boolean isFinal;
+
+    /**
+     * Constructor.
+     */
+    public ClassDescriptor(@NotNull String className, @NotNull Class<?> clazz, int descriptorId,
+            @NotNull List<FieldDescriptor> fields, @NotNull SerializationType serializationType,
+            boolean isFinal) {

Review comment:
       Do we really need this parameter? Can we compute it ourselves? Same applies to `className`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/DefaultClassDescriptors.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Class that holds default types and creates descriptors for them.
+ */
+public class DefaultClassDescriptors {
+    /** Quantity of descriptor ids reserved for the default descriptors. */
+    public static final int DEFAULT_DESCRIPTORS_OFFSET_COUNT = 1000;
+
+    /**
+     * Creates a descriptor for one of the default types.
+     *
+     * @param type Default type.
+     * @return Descriptor.
+     */
+    public static ClassDescriptor createDefaultDescriptor(DefaultType type) {
+        Class<?> clazz = type.clazz;
+        int descriptorId = type.descriptorId;
+        return new ClassDescriptor(
+            ClassDescriptorParser.className(clazz),
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.DEFAULT,
+            ClassDescriptorParser.isFinal(clazz)
+        );
+    }
+
+    /**
+     * Default types.
+     */
+    public enum DefaultType {
+        BYTE(0, byte.class),

Review comment:
       Can we use enum ordinals instead and prohibit the reordering of this enum?

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {

Review comment:
       This method should be annotated with `@Nullable`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObjectNoData");
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the method's declaration contains throwing only of
+     * specified exceptions.
+     *
+     * @param method Method.
+     * @param exceptions List of exceptions.
+     * @return If the method throws exceptions.
+     */
+    @SuppressWarnings("BooleanMethodIsAlwaysInverted")
+    private boolean methodDeclaresExactExceptions(Method method,
+            List<Class<? extends Throwable>> exceptions) {

Review comment:
       I think we can accept a `Set` here...

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObjectNoData");
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(ObjectStreamException.class))) {

Review comment:
       You can use `List.of` that's both more consistent with similar method and shorter to write

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/FieldDescriptor.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Field descriptor for the user object serialization.
+ */
+public class FieldDescriptor {
+    /**
+     * Name of the field.
+     */
+    @NotNull
+    private final String name;
+
+    /**
+     * Type of the field.
+     */
+    @NotNull
+    private final Class<?> clazz;
+
+    /**
+     * Field type's descriptor id.
+     */
+    private final int typeDescriptorId;
+
+    /**
+     * Constructor.
+     */
+    public FieldDescriptor(@NotNull String name, @NotNull Class<?> clazz, int typeDescriptorId) {

Review comment:
       Maybe it would be cleaner to accept a `Field` instead of name and class...

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParserContext.java
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import static org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DEFAULT_DESCRIPTORS_OFFSET_COUNT;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DefaultType;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Class parser context.
+ */
+public class ClassDescriptorParserContext {
+    /** Sequential id generator for class descriptors. */
+    private final AtomicInteger idGenerator = new AtomicInteger(DEFAULT_DESCRIPTORS_OFFSET_COUNT);
+
+    /** Map class -> descriptor id. */
+    private final ConcurrentMap<Class<?>, Integer> idMap = new ConcurrentHashMap<>();
+
+    /** Map descriptor id -> class descriptor. */
+    private final ConcurrentMap<Integer, ClassDescriptor> descriptorMap = new ConcurrentHashMap<>();

Review comment:
       Shall we use an `AtomicReferenceArray` instead here?

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(

Review comment:
       If you add the new descriptor on each step instead of collecting them in a map, you would be able to get rid of the `localDescs` map completely

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParserContext.java
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import static org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DEFAULT_DESCRIPTORS_OFFSET_COUNT;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DefaultType;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Class parser context.
+ */
+public class ClassDescriptorParserContext {
+    /** Sequential id generator for class descriptors. */
+    private final AtomicInteger idGenerator = new AtomicInteger(DEFAULT_DESCRIPTORS_OFFSET_COUNT);
+
+    /** Map class -> descriptor id. */
+    private final ConcurrentMap<Class<?>, Integer> idMap = new ConcurrentHashMap<>();
+
+    /** Map descriptor id -> class descriptor. */
+    private final ConcurrentMap<Integer, ClassDescriptor> descriptorMap = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     */
+    public ClassDescriptorParserContext() {
+        for (DefaultType value : DefaultType.values()) {
+            ClassDescriptor defaultDescriptor = DefaultClassDescriptors.createDefaultDescriptor(
+                    value);

Review comment:
       this fits on one line. I'm starting to think that your IDEA is misconfigured

##########
File path: modules/network/src/test/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParserTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DefaultType;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Set of tests for the class descriptor parser.
+ */
+public class ClassDescriptorParserTest {
+    /**
+     * Descriptor parser context.
+     */
+    private final ClassDescriptorParserContext context = new ClassDescriptorParserContext();
+
+    /**
+     * Descriptor parser.
+     */
+    private final ClassDescriptorParser parser = new ClassDescriptorParser(context);
+
+    static class SerializableClass implements Serializable {
+        private static final long serialVersionUID = 0L;
+    }
+
+    @Test
+    public void testSerializable() {
+        ClassDescriptor descriptor = parser.parse(SerializableClass.class);
+
+        assertEquals(SerializationType.SERIALIZABLE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableOverride implements Serializable {
+        private static final long serialVersionUID = 0L;
+
+        private void writeObject(ObjectOutputStream out) throws IOException {
+            // No-op.
+        }
+
+        private void readObject(ObjectInputStream in)
+                throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+
+        private void readObjectNoData() throws ObjectStreamException {
+            // No-op.
+        }
+    }
+
+    @Test
+    public void testSerializableOverride() {
+        ClassDescriptor descriptor = parser.parse(SerializableOverride.class);
+
+        assertEquals(SerializationType.SERIALIZABLE_OVERRIDE, descriptor.serializationType());
+        assertEquals(0, descriptor.fields().size());
+    }
+
+    static class SerializableOverrideWriteReplace implements Serializable {

Review comment:
       I guess you should also test cases when signatures of these methods do not match the required signatures

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);

Review comment:
       Should we check the context in case the descriptor has already been generated?

##########
File path: modules/network/src/test/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParserTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.network.serialization.DefaultClassDescriptors.DefaultType;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Set of tests for the class descriptor parser.
+ */
+public class ClassDescriptorParserTest {
+    /**
+     * Descriptor parser context.
+     */
+    private final ClassDescriptorParserContext context = new ClassDescriptorParserContext();
+
+    /**
+     * Descriptor parser.
+     */
+    private final ClassDescriptorParser parser = new ClassDescriptorParser(context);
+
+    static class SerializableClass implements Serializable {

Review comment:
       all these inner classes can be `private`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObjectNoData");
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the method's declaration contains throwing only of
+     * specified exceptions.
+     *
+     * @param method Method.
+     * @param exceptions List of exceptions.
+     * @return If the method throws exceptions.
+     */
+    @SuppressWarnings("BooleanMethodIsAlwaysInverted")
+    private boolean methodDeclaresExactExceptions(Method method,

Review comment:
       ```suggestion
       private static boolean methodDeclaresExactExceptions(Method method,
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObjectNoData");
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the method's declaration contains throwing only of
+     * specified exceptions.
+     *
+     * @param method Method.
+     * @param exceptions List of exceptions.
+     * @return If the method throws exceptions.
+     */
+    @SuppressWarnings("BooleanMethodIsAlwaysInverted")
+    private boolean methodDeclaresExactExceptions(Method method,

Review comment:
       also, maybe we can rename this method to `declaresExactExceptions`

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptorParser.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.network.serialization;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.ignite.lang.IgniteException;
+
+/**
+ * Class parser for the user object serialization.
+ */
+public class ClassDescriptorParser {
+
+    /**
+     * Parser context.
+     */
+    private final ClassDescriptorParserContext context;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public ClassDescriptorParser(ClassDescriptorParserContext ctx) {
+        this.context = ctx;
+    }
+
+    /**
+     * Parses the class definition and definitions of classes fields if they're not already parsed.
+     *
+     * @param clazz Class definition.
+     * @return Class descriptor.
+     */
+    public ClassDescriptor parse(Class<?> clazz) {
+        ClassDescriptor classDesc = parse0(clazz);
+
+        Map<Integer, ClassDescriptor> localDescs = new HashMap<>();
+        localDescs.put(classDesc.descriptorId(), classDesc);
+
+        Queue<FieldDescriptor> fieldDescriptors = new LinkedList<>(classDesc.fields());
+
+        while (!fieldDescriptors.isEmpty()) {
+            FieldDescriptor fieldDescriptor = fieldDescriptors.remove();
+
+            int typeDescriptorId = fieldDescriptor.typeDescriptorId();
+
+            if (context.hasDescriptor(typeDescriptorId) || localDescs.containsKey(
+                    typeDescriptorId)) {
+                continue;
+            }
+
+            Class<?> fieldClass = fieldDescriptor.clazz();
+
+            ClassDescriptor fieldClassDesc = parse0(fieldClass);
+
+            localDescs.put(typeDescriptorId, fieldClassDesc);
+
+            fieldDescriptors.addAll(fieldClassDesc.fields());
+        }
+
+        context.addDescriptors(localDescs.values());
+
+        return classDesc;
+    }
+
+    /**
+     * Parses the class definition.
+     *
+     * @param clazz Class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor parse0(Class<?> clazz) {
+        assert !clazz.isPrimitive() :
+            clazz + " is a primitive, there should be a default descriptor";
+
+        int descriptorId = context.getId(clazz);
+
+        if (Externalizable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return externalizable(descriptorId, (Class<? extends Externalizable>) clazz);
+        } else if (Serializable.class.isAssignableFrom(clazz)) {
+            //noinspection unchecked
+            return serializable(descriptorId, (Class<? extends Serializable>) clazz);
+        }
+
+        return arbitrary(descriptorId, clazz);
+    }
+
+    /**
+     * Parses the externalizable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz        Externalizable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor externalizable(int descriptorId,
+            Class<? extends Externalizable> clazz) {
+        String className = className(clazz);
+
+        checkHasPublicNoArgConstructor(clazz);
+
+        return new ClassDescriptor(
+            className,
+            clazz,
+            descriptorId,
+            Collections.emptyList(),
+            SerializationType.EXTERNALIZABLE,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Checks if a class has a public no-arg constructor.
+     *
+     * @param clazz Class.
+     */
+    private void checkHasPublicNoArgConstructor(Class<? extends Externalizable> clazz)
+            throws IgniteException {
+        boolean hasPublicNoArgConstructor = true;
+
+        try {
+            Constructor<? extends Externalizable> ctor = clazz.getConstructor();
+
+            if (!Modifier.isPublic(ctor.getModifiers())) {
+                hasPublicNoArgConstructor = false;
+            }
+        } catch (NoSuchMethodException e) {
+            hasPublicNoArgConstructor = false;
+        }
+
+        if (!hasPublicNoArgConstructor) {
+            throw new IgniteException(
+                "Externalizable class " + className(clazz) + " has no public no-arg constructor");
+        }
+    }
+
+    /**
+     * Parses the serializable class definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Serializable class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor serializable(int descriptorId, Class<? extends Serializable> clazz) {
+        Method writeObject = getWriteObject(clazz);
+        Method readObject = getReadObject(clazz);
+        Method readObjectNoData = getReadObjectNoData(clazz);
+
+        boolean overrideSerialization = writeObject != null && readObject != null
+                && readObjectNoData != null;
+
+        Method writeReplace = getWriteReplace(clazz);
+        Method readResolve = getReadResolve(clazz);
+
+        boolean hasWriteReplaceOrReadResolve = writeReplace != null || readResolve != null;
+        boolean hasOnlyWriteReplace = writeReplace != null && readResolve == null;
+        boolean hasOnlyReadResolve = readResolve != null && writeReplace == null;
+
+        SerializationType type;
+
+        if (overrideSerialization) {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_OVERRIDE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        } else {
+            if (!hasWriteReplaceOrReadResolve) {
+                type = SerializationType.SERIALIZABLE;
+            } else if (hasOnlyWriteReplace) {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE;
+            } else if (hasOnlyReadResolve) {
+                type = SerializationType.SERIALIZABLE_READ_RESOLVE;
+            } else {
+                type = SerializationType.SERIALIZABLE_WRITE_REPLACE_READ_RESOLVE;
+            }
+        }
+
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            type,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Parses the arbitrary class (not serializable or externalizable) definition.
+     *
+     * @param descriptorId Descriptor id of the class.
+     * @param clazz Arbitrary class.
+     * @return Class descriptor.
+     */
+    private ClassDescriptor arbitrary(int descriptorId, Class<?> clazz) {
+        return new ClassDescriptor(
+            className(clazz),
+            clazz,
+            descriptorId,
+            fields(clazz),
+            SerializationType.ARBITRARY,
+            isFinal(clazz)
+        );
+    }
+
+    /**
+     * Gets field descriptors of the class. If a field's type doesn't have an id yet, generates it.
+     *
+     * @param clazz Class.
+     * @return List of field descriptor.
+     */
+    private List<FieldDescriptor> fields(Class<?> clazz) {
+        if (clazz.getSuperclass() != Object.class) {
+            // TODO: IGNITE-15945 add support for the inheritance
+            throw new UnsupportedOperationException("IGNITE-15945");
+        }
+
+        Field[] fields = clazz.getDeclaredFields();
+
+        List<FieldDescriptor> descs = new ArrayList<>(fields.length);
+
+        for (Field field : fields) {
+            Class<?> type = field.getType();
+
+            int modifiers = field.getModifiers();
+
+            // Ignore static and transient field.
+            if (Modifier.isStatic(modifiers) || Modifier.isTransient(modifiers)) {
+                continue;
+            }
+
+            String name = field.getName();
+
+            // Get or create type id.
+            int typeId = context.getId(type);
+
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(name, type, typeId);
+
+            descs.add(fieldDescriptor);
+        }
+
+        return descs;
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object writeReplace() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteReplace(Class<? extends Serializable> clazz) {
+        try {
+            Method writeReplace = clazz.getDeclaredMethod("writeReplace");
+
+            if (!methodDeclaresExactExceptions(writeReplace,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeReplace;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code ANY-ACCESS-MODIFIER Object readResolve() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadResolve(Class<? extends Serializable> clazz) {
+        try {
+            Method readResolve = clazz.getDeclaredMethod("readResolve");
+
+            if (!methodDeclaresExactExceptions(readResolve,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return readResolve;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void writeObject(java.io.ObjectOutputStream out) throws IOException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getWriteObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(IOException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObject(java.io.ObjectInputStream in) throws IOException,
+     * ClassNotFoundException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObject(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    List.of(IOException.class, ClassNotFoundException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Gets a method with the signature
+     * {@code private void readObjectNoData() throws ObjectStreamException}.
+     *
+     * @param clazz Class.
+     * @return Method.
+     */
+    private Method getReadObjectNoData(Class<? extends Serializable> clazz) {
+        try {
+            Method writeObject = clazz.getDeclaredMethod("readObjectNoData");
+
+            if (!Modifier.isPrivate(writeObject.getModifiers())) {
+                return null;
+            }
+
+            if (!methodDeclaresExactExceptions(writeObject,
+                    Collections.singletonList(ObjectStreamException.class))) {
+                return null;
+            }
+
+            return writeObject;
+        } catch (NoSuchMethodException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the method's declaration contains throwing only of
+     * specified exceptions.
+     *
+     * @param method Method.
+     * @param exceptions List of exceptions.
+     * @return If the method throws exceptions.
+     */
+    @SuppressWarnings("BooleanMethodIsAlwaysInverted")
+    private boolean methodDeclaresExactExceptions(Method method,
+            List<Class<? extends Throwable>> exceptions) {

Review comment:
       also, fits on one line




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org