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/05/25 10:03:37 UTC

[GitHub] [ignite-3] ibessonov commented on a change in pull request #130: IGNITE-14649 Annotation processor for network message (de-)serializers

ibessonov commented on a change in pull request #130:
URL: https://github.com/apache/ignite-3/pull/130#discussion_r638587678



##########
File path: modules/network-annotation-processor/src/integrationTest/java/org/apache/ignite/network/messages/internal/processor/ITAutoSerializableProcessorTest.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.network.processor.internal;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.tools.JavaFileObject;
+import com.google.testing.compile.Compilation;
+import com.google.testing.compile.Compiler;
+import com.google.testing.compile.JavaFileObjects;
+import org.apache.ignite.network.NetworkMessage;
+import org.junit.jupiter.api.Test;
+
+import static com.google.testing.compile.CompilationSubject.assertThat;
+
+/**
+ * Integration tests for {@link AutoSerializableProcessor}.
+ */
+public class ITAutoSerializableProcessorTest {
+    /** Package name of the test sources. */
+    private static final String RESOURCE_PACKAGE_NAME = "org.apache.ignite.network.processor.internal.";
+
+    /** Compiler instance configured with the annotation processor being tested. */
+    private final Compiler compiler = Compiler.javac().withProcessors(new AutoSerializableProcessor());
+
+    /**
+     * Compiles the network message with all supported directly marshallable types and checks that the compilation
+     * completed successfully.
+     */
+    @Test
+    void testCompileAllTypesMessage() {
+        Compilation compilation = compiler.compile(
+            getSources("AllTypesMessage", "AllTypesMessageImpl", "AllTypesMessageFactory")
+        );
+
+        assertThat(compilation).succeededWithoutWarnings();
+
+        assertThat(compilation).generatedSourceFile(RESOURCE_PACKAGE_NAME + "AllTypesMessageSerializer");
+        assertThat(compilation).generatedSourceFile(RESOURCE_PACKAGE_NAME + "AllTypesMessageDeserializer");
+        assertThat(compilation).generatedSourceFile(RESOURCE_PACKAGE_NAME + "AllTypesMessageSerializationFactory");
+    }
+
+    /**
+     * Compiles a test message that doesn't extend {@link NetworkMessage}.
+     */
+    @Test
+    void testInvalidAnnotatedTypeMessage() {
+        Compilation compilation = compiler.compile(
+            getSources("InvalidAnnotatedTypeMessage", "AllTypesMessageImpl", "AllTypesMessageFactory")
+        );
+
+        assertThat(compilation).hadErrorContaining("annotation must only be present on interfaces that extend");
+    }
+
+    /**
+     * Compiles a test message that contains an unsupported content type.
+     */
+    @Test
+    void testUnsupportedTypeMessage() {
+        Compilation compilation = compiler.compile(
+            getSources("UnsupportedTypeMessage", "AllTypesMessageImpl", "AllTypesMessageFactory")
+        );
+
+        assertThat(compilation).hadErrorContaining("Unsupported reference type for message (de-)serialization: java.util.ArrayList");
+    }
+
+    /**
+     * Compiles a test message that violates the message contract by not declaring a {@code Builder} interface.
+     */
+    @Test
+    void testMissingBuilderMessage() {
+        Compilation compilation = compiler.compile(
+            getSources("MissingBuilderMessage", "AllTypesMessageImpl", "AllTypesMessageFactory")
+        );
+
+        assertThat(compilation).hadErrorContaining("No nested Builder interface found");

Review comment:
       Can we avoid this requirement and generate builders by ourselves?

##########
File path: modules/network-annotation-processor/src/integrationTest/resources/org/apache/ignite/network/processor/internal/AllTypesMessage.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.network.processor.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+
+@AutoSerializable(messageFactory = AllTypesMessageFactory.class)
+public interface AllTypesMessage extends NetworkMessage {
+    short TYPE = 123;

Review comment:
       What's the purpose of this constant?

##########
File path: modules/network-annotation-processor/src/integrationTest/resources/org/apache/ignite/network/processor/internal/AllTypesMessage.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.network.processor.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+
+@AutoSerializable(messageFactory = AllTypesMessageFactory.class)

Review comment:
       Similar question - we should at least consider the possibility of generating this factory.

##########
File path: modules/network-annotation-processor/src/main/java/org/apache/ignite/network/processor/internal/AutoSerializableProcessor.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.network.processor.internal;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedSourceVersion;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.Name;
+import javax.lang.model.element.PackageElement;
+import javax.lang.model.element.TypeElement;
+import javax.tools.Diagnostic;
+import com.squareup.javapoet.ClassName;
+import com.squareup.javapoet.JavaFile;
+import com.squareup.javapoet.MethodSpec;
+import com.squareup.javapoet.TypeName;
+import com.squareup.javapoet.TypeSpec;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+import org.apache.ignite.network.serialization.MessageDeserializer;
+import org.apache.ignite.network.serialization.MessageSerializationFactory;
+import org.apache.ignite.network.serialization.MessageSerializationRegistry;
+import org.apache.ignite.network.serialization.MessageSerializer;
+
+/**
+ * Annotation processor for generating (de-)serializers for network messages marked with the {@link AutoSerializable}
+ * annotation.
+ */
+@SupportedSourceVersion(SourceVersion.RELEASE_11)
+public class AutoSerializableProcessor extends AbstractProcessor {
+    /** {@inheritDoc} */
+    @Override public Set<String> getSupportedAnnotationTypes() {
+        return Set.of(AutoSerializable.class.getName());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean process(Set<? extends TypeElement> annotations, RoundEnvironment roundEnv) {
+        Set<TypeElement> annotatedElements = annotations.stream()
+            .map(roundEnv::getElementsAnnotatedWith)
+            .flatMap(Collection::stream)
+            .map(TypeElement.class::cast)
+            .collect(Collectors.toUnmodifiableSet());
+
+        if (annotatedElements.isEmpty()) {
+            return true;
+        }
+
+        try {
+            generateSources(annotatedElements);
+        } catch (IOException e) {
+            throw new IllegalStateException("IO exception during annotation processing", e);
+        }
+
+        return true;
+    }
+
+    /**
+     * Generates serialization-related classes for the given elements.
+     */
+    private void generateSources(Set<TypeElement> annotatedElements) throws IOException {
+        var factories = new HashMap<TypeElement, TypeSpec>();
+
+        for (var messageClass : annotatedElements) {
+            try {
+                if (isValidElement(messageClass)) {
+                    String packageName = ClassName.get(messageClass).packageName();
+
+                    TypeSpec serializer = generateSerializer(messageClass);
+                    writeToFile(packageName, serializer);
+
+                    TypeSpec deserializer = generateDeseralizer(messageClass);
+                    writeToFile(packageName, deserializer);
+
+                    TypeSpec factory = generateFactory(messageClass, serializer, deserializer);
+                    writeToFile(packageName, factory);
+
+                    factories.put(messageClass, factory);
+                } else {
+                    processingEnv.getMessager().printMessage(
+                        Diagnostic.Kind.ERROR,
+                        String.format(
+                            "%s annotation must only be present on interfaces that extend %s",
+                            AutoSerializable.class, NetworkMessage.class
+                        ),
+                        messageClass
+                    );
+                }
+            } catch (ProcessingException e) {
+                processingEnv.getMessager().printMessage(Diagnostic.Kind.ERROR, e.getMessage(), messageClass);
+            }
+        }
+
+        TypeSpec registryInitializer = generateRegistryInitializer(factories);
+        writeToFile(getParentPackage(annotatedElements), registryInitializer);
+    }
+
+    /**
+     * Generates a {@link MessageSerializer}.
+     */
+    private TypeSpec generateSerializer(TypeElement messageClass) {
+        processingEnv.getMessager().printMessage(Diagnostic.Kind.NOTE, "Generating a MessageSerializer", messageClass);
+
+        return new MessageSerializerGenerator(processingEnv, messageClass).generateSerializer();
+    }
+
+    /**
+     * Generates a {@link MessageDeserializer}.
+     */
+    private TypeSpec generateDeseralizer(TypeElement messageClass) {
+        processingEnv.getMessager().printMessage(Diagnostic.Kind.NOTE, "Generating a MessageDeserializer", messageClass);
+
+        return new MessageDeserializerGenerator(processingEnv, messageClass).generateDeserializer();
+    }
+
+    /**
+     * Generates a {@link MessageSerializationFactory}.
+     */
+    private TypeSpec generateFactory(TypeElement messageClass, TypeSpec serializer, TypeSpec deserializer) {
+        processingEnv.getMessager().printMessage(Diagnostic.Kind.NOTE, "Generating a MessageSerializationFactory", messageClass);
+
+        return new SerializationFactoryGenerator(messageClass).generateFactory(serializer, deserializer);
+    }
+
+    /**
+     * Generates a class for registering all generated {@link MessageSerializationFactory} for the current module.
+     */
+    // TODO: refactor this method to use module names as part of the generated class,
+    //  see https://issues.apache.org/jira/browse/IGNITE-14715
+    private static TypeSpec generateRegistryInitializer(Map<TypeElement, TypeSpec> factoriesByMessageType) {
+        MethodSpec.Builder initializeMethod = MethodSpec.methodBuilder("initialize")
+            .addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+            .addParameter(TypeName.get(MessageSerializationRegistry.class), "serializationRegistry");
+
+        factoriesByMessageType.forEach((messageClass, factory) -> {
+            var factoryPackage = ClassName.get(messageClass).packageName();
+            var factoryType = ClassName.get(factoryPackage, factory.name);
+
+            initializeMethod.addStatement("serializationRegistry.registerFactory($T.TYPE, new $T())", messageClass, factoryType);

Review comment:
       Oh, I see now why you need this constant. Very non-intuitive.

##########
File path: modules/network-annotation-processor/src/main/java/org/apache/ignite/network/processor/internal/AutoSerializableProcessor.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.network.processor.internal;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedSourceVersion;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.Name;
+import javax.lang.model.element.PackageElement;
+import javax.lang.model.element.TypeElement;
+import javax.tools.Diagnostic;
+import com.squareup.javapoet.ClassName;
+import com.squareup.javapoet.JavaFile;
+import com.squareup.javapoet.MethodSpec;
+import com.squareup.javapoet.TypeName;
+import com.squareup.javapoet.TypeSpec;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+import org.apache.ignite.network.serialization.MessageDeserializer;
+import org.apache.ignite.network.serialization.MessageSerializationFactory;
+import org.apache.ignite.network.serialization.MessageSerializationRegistry;
+import org.apache.ignite.network.serialization.MessageSerializer;
+
+/**
+ * Annotation processor for generating (de-)serializers for network messages marked with the {@link AutoSerializable}
+ * annotation.
+ */
+@SupportedSourceVersion(SourceVersion.RELEASE_11)
+public class AutoSerializableProcessor extends AbstractProcessor {
+    /** {@inheritDoc} */
+    @Override public Set<String> getSupportedAnnotationTypes() {
+        return Set.of(AutoSerializable.class.getName());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean process(Set<? extends TypeElement> annotations, RoundEnvironment roundEnv) {
+        Set<TypeElement> annotatedElements = annotations.stream()
+            .map(roundEnv::getElementsAnnotatedWith)
+            .flatMap(Collection::stream)
+            .map(TypeElement.class::cast)
+            .collect(Collectors.toUnmodifiableSet());
+
+        if (annotatedElements.isEmpty()) {
+            return true;
+        }
+
+        try {
+            generateSources(annotatedElements);
+        } catch (IOException e) {
+            throw new IllegalStateException("IO exception during annotation processing", e);
+        }
+
+        return true;
+    }
+
+    /**
+     * Generates serialization-related classes for the given elements.
+     */
+    private void generateSources(Set<TypeElement> annotatedElements) throws IOException {
+        var factories = new HashMap<TypeElement, TypeSpec>();
+
+        for (var messageClass : annotatedElements) {
+            try {
+                if (isValidElement(messageClass)) {
+                    String packageName = ClassName.get(messageClass).packageName();
+
+                    TypeSpec serializer = generateSerializer(messageClass);
+                    writeToFile(packageName, serializer);
+
+                    TypeSpec deserializer = generateDeseralizer(messageClass);
+                    writeToFile(packageName, deserializer);
+
+                    TypeSpec factory = generateFactory(messageClass, serializer, deserializer);
+                    writeToFile(packageName, factory);
+
+                    factories.put(messageClass, factory);
+                } else {
+                    processingEnv.getMessager().printMessage(
+                        Diagnostic.Kind.ERROR,
+                        String.format(
+                            "%s annotation must only be present on interfaces that extend %s",
+                            AutoSerializable.class, NetworkMessage.class
+                        ),
+                        messageClass
+                    );
+                }
+            } catch (ProcessingException e) {
+                processingEnv.getMessager().printMessage(Diagnostic.Kind.ERROR, e.getMessage(), messageClass);
+            }
+        }
+
+        TypeSpec registryInitializer = generateRegistryInitializer(factories);
+        writeToFile(getParentPackage(annotatedElements), registryInitializer);
+    }
+
+    /**
+     * Generates a {@link MessageSerializer}.
+     */
+    private TypeSpec generateSerializer(TypeElement messageClass) {
+        processingEnv.getMessager().printMessage(Diagnostic.Kind.NOTE, "Generating a MessageSerializer", messageClass);
+
+        return new MessageSerializerGenerator(processingEnv, messageClass).generateSerializer();
+    }
+
+    /**
+     * Generates a {@link MessageDeserializer}.
+     */
+    private TypeSpec generateDeseralizer(TypeElement messageClass) {
+        processingEnv.getMessager().printMessage(Diagnostic.Kind.NOTE, "Generating a MessageDeserializer", messageClass);
+
+        return new MessageDeserializerGenerator(processingEnv, messageClass).generateDeserializer();
+    }
+
+    /**
+     * Generates a {@link MessageSerializationFactory}.
+     */
+    private TypeSpec generateFactory(TypeElement messageClass, TypeSpec serializer, TypeSpec deserializer) {
+        processingEnv.getMessager().printMessage(Diagnostic.Kind.NOTE, "Generating a MessageSerializationFactory", messageClass);
+
+        return new SerializationFactoryGenerator(messageClass).generateFactory(serializer, deserializer);
+    }
+
+    /**
+     * Generates a class for registering all generated {@link MessageSerializationFactory} for the current module.
+     */
+    // TODO: refactor this method to use module names as part of the generated class,
+    //  see https://issues.apache.org/jira/browse/IGNITE-14715
+    private static TypeSpec generateRegistryInitializer(Map<TypeElement, TypeSpec> factoriesByMessageType) {
+        MethodSpec.Builder initializeMethod = MethodSpec.methodBuilder("initialize")
+            .addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+            .addParameter(TypeName.get(MessageSerializationRegistry.class), "serializationRegistry");
+
+        factoriesByMessageType.forEach((messageClass, factory) -> {
+            var factoryPackage = ClassName.get(messageClass).packageName();
+            var factoryType = ClassName.get(factoryPackage, factory.name);
+
+            initializeMethod.addStatement("serializationRegistry.registerFactory($T.TYPE, new $T())", messageClass, factoryType);
+        });
+
+        return TypeSpec.classBuilder("MessageSerializationRegistryInitializer")
+            .addModifiers(Modifier.PUBLIC)
+            .addMethod(initializeMethod.build())
+            .build();
+    }
+
+    /**
+     * Returns the longest common package name among the given elements' packages.
+     */
+    private String getParentPackage(Collection<TypeElement> messageClasses) {
+        List<String[]> packageNames = messageClasses.stream()
+            .map(processingEnv.getElementUtils()::getPackageOf)
+            .map(PackageElement::getQualifiedName)
+            .map(Name::toString)
+            .map(packageName -> packageName.split("\\."))
+            .collect(Collectors.toUnmodifiableList());
+
+        int minNameLength = packageNames.stream().mapToInt(arr -> arr.length).min().getAsInt();
+
+        var result = new StringJoiner(".");
+
+        for (int i = 0; i < minNameLength; ++i) {
+            var distinctSubPackageNames = new HashSet<String>();
+
+            for (String[] packageName : packageNames) {
+                distinctSubPackageNames.add(packageName[i]);
+            }
+
+            if (distinctSubPackageNames.size() == 1) {
+                result.add(distinctSubPackageNames.iterator().next());
+            } else {

Review comment:
       Wrong formatting

##########
File path: modules/network-annotation-processor/src/main/java/org/apache/ignite/network/processor/internal/BaseMethodNameResolver.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.network.processor.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import javax.annotation.processing.ProcessingEnvironment;
+import javax.lang.model.type.ArrayType;
+import javax.lang.model.type.DeclaredType;
+import javax.lang.model.type.TypeKind;
+import javax.lang.model.type.TypeMirror;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for resolving a "base" part of a (de-)serialization method based on the message type. This part is then used
+ * by concrete method resolvers by prepending a "read"/"write" prefix and adding call arguments.
+ *
+ * @see MessageReaderMethodResolver
+ * @see MessageWriterMethodResolver
+ */
+class BaseMethodNameResolver {
+    /** */
+    private final ProcessingEnvironment processingEnvironment;
+
+    /** */
+    BaseMethodNameResolver(ProcessingEnvironment processingEnvironment) {
+        this.processingEnvironment = processingEnvironment;
+    }
+
+    /**
+     * Resolves a "base" part of a (de-)serialization method.
+     */
+    String resolveBaseMethodName(TypeMirror parameterType) {
+        if (parameterType.getKind().isPrimitive()) {
+            return resolvePrimitiveMethodName(parameterType);
+        } else if (parameterType.getKind() == TypeKind.ARRAY) {

Review comment:
       you know the deal

##########
File path: modules/network-annotation-processor/src/main/java/org/apache/ignite/network/processor/internal/AutoSerializableProcessor.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.network.processor.internal;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedSourceVersion;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.Name;
+import javax.lang.model.element.PackageElement;
+import javax.lang.model.element.TypeElement;
+import javax.tools.Diagnostic;
+import com.squareup.javapoet.ClassName;
+import com.squareup.javapoet.JavaFile;
+import com.squareup.javapoet.MethodSpec;
+import com.squareup.javapoet.TypeName;
+import com.squareup.javapoet.TypeSpec;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+import org.apache.ignite.network.serialization.MessageDeserializer;
+import org.apache.ignite.network.serialization.MessageSerializationFactory;
+import org.apache.ignite.network.serialization.MessageSerializationRegistry;
+import org.apache.ignite.network.serialization.MessageSerializer;
+
+/**
+ * Annotation processor for generating (de-)serializers for network messages marked with the {@link AutoSerializable}
+ * annotation.
+ */
+@SupportedSourceVersion(SourceVersion.RELEASE_11)
+public class AutoSerializableProcessor extends AbstractProcessor {
+    /** {@inheritDoc} */
+    @Override public Set<String> getSupportedAnnotationTypes() {
+        return Set.of(AutoSerializable.class.getName());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean process(Set<? extends TypeElement> annotations, RoundEnvironment roundEnv) {
+        Set<TypeElement> annotatedElements = annotations.stream()
+            .map(roundEnv::getElementsAnnotatedWith)
+            .flatMap(Collection::stream)
+            .map(TypeElement.class::cast)
+            .collect(Collectors.toUnmodifiableSet());
+
+        if (annotatedElements.isEmpty()) {
+            return true;
+        }
+
+        try {
+            generateSources(annotatedElements);
+        } catch (IOException e) {
+            throw new IllegalStateException("IO exception during annotation processing", e);
+        }
+
+        return true;
+    }
+
+    /**
+     * Generates serialization-related classes for the given elements.
+     */
+    private void generateSources(Set<TypeElement> annotatedElements) throws IOException {
+        var factories = new HashMap<TypeElement, TypeSpec>();
+
+        for (var messageClass : annotatedElements) {
+            try {
+                if (isValidElement(messageClass)) {
+                    String packageName = ClassName.get(messageClass).packageName();
+
+                    TypeSpec serializer = generateSerializer(messageClass);
+                    writeToFile(packageName, serializer);
+
+                    TypeSpec deserializer = generateDeseralizer(messageClass);
+                    writeToFile(packageName, deserializer);
+
+                    TypeSpec factory = generateFactory(messageClass, serializer, deserializer);
+                    writeToFile(packageName, factory);
+
+                    factories.put(messageClass, factory);
+                } else {

Review comment:
       Wrong formatting

##########
File path: modules/network-annotation-processor/src/integrationTest/resources/org/apache/ignite/network/processor/internal/AllTypesMessage.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.network.processor.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+
+@AutoSerializable(messageFactory = AllTypesMessageFactory.class)
+public interface AllTypesMessage extends NetworkMessage {

Review comment:
       I don't see any support for arbitrary serializable fields. We will need it in the future, 100%

##########
File path: modules/network-api/pom.xml
##########
@@ -0,0 +1,62 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent/pom.xml</relativePath>
+    </parent>
+
+    <artifactId>ignite-network-api</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+        </dependency>
+
+        <dependency>

Review comment:
       Please add a comment declaring that following dependencies are test-only.

##########
File path: modules/network-annotation-processor/src/main/java/org/apache/ignite/network/processor/internal/AutoSerializableProcessor.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.network.processor.internal;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedSourceVersion;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.Name;
+import javax.lang.model.element.PackageElement;
+import javax.lang.model.element.TypeElement;
+import javax.tools.Diagnostic;
+import com.squareup.javapoet.ClassName;
+import com.squareup.javapoet.JavaFile;
+import com.squareup.javapoet.MethodSpec;
+import com.squareup.javapoet.TypeName;
+import com.squareup.javapoet.TypeSpec;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+import org.apache.ignite.network.serialization.MessageDeserializer;
+import org.apache.ignite.network.serialization.MessageSerializationFactory;
+import org.apache.ignite.network.serialization.MessageSerializationRegistry;
+import org.apache.ignite.network.serialization.MessageSerializer;
+
+/**
+ * Annotation processor for generating (de-)serializers for network messages marked with the {@link AutoSerializable}
+ * annotation.
+ */
+@SupportedSourceVersion(SourceVersion.RELEASE_11)
+public class AutoSerializableProcessor extends AbstractProcessor {
+    /** {@inheritDoc} */
+    @Override public Set<String> getSupportedAnnotationTypes() {
+        return Set.of(AutoSerializable.class.getName());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean process(Set<? extends TypeElement> annotations, RoundEnvironment roundEnv) {
+        Set<TypeElement> annotatedElements = annotations.stream()
+            .map(roundEnv::getElementsAnnotatedWith)
+            .flatMap(Collection::stream)
+            .map(TypeElement.class::cast)
+            .collect(Collectors.toUnmodifiableSet());
+
+        if (annotatedElements.isEmpty()) {
+            return true;
+        }
+
+        try {
+            generateSources(annotatedElements);
+        } catch (IOException e) {
+            throw new IllegalStateException("IO exception during annotation processing", e);
+        }
+
+        return true;
+    }
+
+    /**
+     * Generates serialization-related classes for the given elements.
+     */
+    private void generateSources(Set<TypeElement> annotatedElements) throws IOException {
+        var factories = new HashMap<TypeElement, TypeSpec>();
+
+        for (var messageClass : annotatedElements) {
+            try {
+                if (isValidElement(messageClass)) {
+                    String packageName = ClassName.get(messageClass).packageName();
+
+                    TypeSpec serializer = generateSerializer(messageClass);
+                    writeToFile(packageName, serializer);
+
+                    TypeSpec deserializer = generateDeseralizer(messageClass);
+                    writeToFile(packageName, deserializer);
+
+                    TypeSpec factory = generateFactory(messageClass, serializer, deserializer);
+                    writeToFile(packageName, factory);
+
+                    factories.put(messageClass, factory);
+                } else {
+                    processingEnv.getMessager().printMessage(
+                        Diagnostic.Kind.ERROR,
+                        String.format(
+                            "%s annotation must only be present on interfaces that extend %s",
+                            AutoSerializable.class, NetworkMessage.class
+                        ),
+                        messageClass
+                    );
+                }
+            } catch (ProcessingException e) {
+                processingEnv.getMessager().printMessage(Diagnostic.Kind.ERROR, e.getMessage(), messageClass);
+            }
+        }
+
+        TypeSpec registryInitializer = generateRegistryInitializer(factories);
+        writeToFile(getParentPackage(annotatedElements), registryInitializer);

Review comment:
       Can we put it to the same package where builder factory is located?

##########
File path: modules/network-annotation-processor/src/integrationTest/resources/org/apache/ignite/network/processor/internal/AllTypesMessage.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.network.processor.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.processor.annotations.AutoSerializable;
+
+@AutoSerializable(messageFactory = AllTypesMessageFactory.class)
+public interface AllTypesMessage extends NetworkMessage {
+    short TYPE = 123;
+
+    byte a();
+
+    short b();
+
+    int c();
+
+    long d();
+
+    float e();
+
+    double f();
+
+    char g();
+
+    boolean h();
+
+    byte[] i();
+
+    short[] j();
+
+    int[] k();
+
+    long[] l();
+
+    float[] m();
+
+    double[] n();
+
+    char[] o();
+
+    boolean[] p();
+
+    String q();
+
+    BitSet r();
+
+    UUID s();
+
+    IgniteUuid t();
+
+    NetworkMessage u();
+
+    NetworkMessage[] v();
+
+    Collection<NetworkMessage> w();
+
+    Map<String, NetworkMessage> x();
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override public default short directType() {
+        return 5555;
+    }
+
+    interface Builder {
+        AllTypesMessage build();
+
+        Builder a(byte a);
+
+        Builder b(short b);
+
+        Builder c(int c);
+
+        Builder d(long d);
+
+        Builder e(float e);
+
+        Builder f(double f);
+
+        Builder g(char g);
+
+        Builder h(boolean h);
+
+        Builder i(byte[] i);
+
+        Builder j(short[] j);
+
+        Builder k(int[] k);
+
+        Builder l(long[] l);
+
+        Builder m(float[] m);
+
+        Builder n(double[] n);
+
+        Builder o(char[] o);
+
+        Builder p(boolean[] p);
+
+        Builder q(String q);
+
+        Builder r(BitSet r);
+
+        Builder s(UUID s);
+
+        Builder t(IgniteUuid t);
+
+        Builder u(NetworkMessage u);

Review comment:
       We might want to have directly marshallable classes that can't be used as messages. So we should think of other name for new superinterface.

##########
File path: modules/network/src/test/java/org/apache/ignite/network/internal/AllTypesMessageImpl.java
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.network.internal;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+
+/**
+ * Message with all types supported by Direct Marshalling.
+ */
+class AllTypesMessageImpl implements AllTypesMessage, AllTypesMessage.Builder {

Review comment:
       Why do we need impl class if it's auto-generated?




-- 
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.

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