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/24 10:38:36 UTC

[GitHub] [ignite-3] SammyVimes commented on a change in pull request #521: IGNITE-16155 Implement (un)marshalling of built-ins

SammyVimes commented on a change in pull request #521:
URL: https://github.com/apache/ignite-3/pull/521#discussion_r774963933



##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/ClassDescriptor.java
##########
@@ -172,4 +172,50 @@ public boolean hasWriteReplace() {
     public boolean hasReadResolve() {
         return serialization.hasReadResolve();
     }
+
+    /**
+     * Returns {@code true} if this is the descriptor of {@code null} values.
+     *
+     * @return {@code true} if this is the descriptor of {@code null} values
+     */
+    public boolean isNull() {
+        return clazz == Null.class;
+    }
+
+    /**
+     * Returns {@code true} if this is the descriptor of {@link java.util.Collections#singletonList(Object)} type.
+     *
+     * @return {@code true} if this is the descriptor of {@link java.util.Collections#singletonList(Object)} type
+     */
+    public boolean isSingletonList() {
+        return descriptorId == BuiltinType.SINGLETON_LIST.descriptorId();

Review comment:
       why is it different from `isNull` in the means of comparison?

##########
File path: modules/network/src/test/java/org/apache/ignite/internal/network/serialization/marshal/DefaultUserObjectMarshallerWithBuiltinsTest.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.marshal;
+
+import static java.util.Collections.singletonList;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.jupiter.api.Assumptions.assumingThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+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.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.network.serialization.BuiltinType;
+import org.apache.ignite.internal.network.serialization.ClassDescriptor;
+import org.apache.ignite.internal.network.serialization.ClassDescriptorFactory;
+import org.apache.ignite.internal.network.serialization.ClassDescriptorFactoryContext;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests for how {@link DefaultUserObjectMarshaller} handles primitives.
+ */
+class DefaultUserObjectMarshallerWithBuiltinsTest {
+    private final ClassDescriptorFactoryContext descriptorRegistry = new ClassDescriptorFactoryContext();
+    private final ClassDescriptorFactory descriptorFactory = new ClassDescriptorFactory(descriptorRegistry);
+
+    private final DefaultUserObjectMarshaller marshaller = new DefaultUserObjectMarshaller(descriptorRegistry, descriptorFactory);
+
+    @Test
+    void marshalsAndUnmarshalsNull() throws Exception {
+        MarshalledObject marshalled = marshaller.marshal(null);
+
+        Object unmarshalled = marshaller.unmarshal(marshalled.bytes());
+
+        assertThat(unmarshalled, is(nullValue()));
+    }
+
+    @Test
+    void marshalsNullUsingOnlyNullDescriptor() throws Exception {
+        MarshalledObject marshalled = marshaller.marshal(null);
+
+        assertThat(marshalled.usedDescriptors(), equalTo(List.of(descriptorRegistry.getNullDescriptor())));
+    }
+
+    @Test
+    void marshalsNullWithCorrectDescriptorId() throws Exception {
+        MarshalledObject marshalled = marshaller.marshal(null);

Review comment:
       Maybe all tests for `null` value should be in one test ?

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/BuiltInContainerMarshallers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.marshal;
+
+import static java.util.Collections.singletonList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.IntFunction;
+import org.apache.ignite.internal.network.serialization.ClassDescriptor;
+
+/**
+ * Utility to (un)marshal built-in collections and maps.
+ */
+class BuiltInContainerMarshallers {
+    /**
+     * Map of all classes which are built-in collections AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     */
+    private final Map<Class<?>, IntFunction<? extends Collection<?>>> mutableBuiltInCollectionFactories = Map.of(
+            ArrayList.class, ArrayList::new,
+            LinkedList.class, len -> new LinkedList<>(),
+            HashSet.class, HashSet::new,
+            LinkedHashSet.class, LinkedHashSet::new
+    );
+
+    /**
+     * Map of all classes which are built-in maps AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty map, then read N entries
+     * and put each of them into the map.
+     */
+    private final Map<Class<?>, IntFunction<? extends Map<?, ?>>> mutableBuiltInMapFactories = Map.of(
+            HashMap.class, HashMap::new,
+            LinkedHashMap.class, LinkedHashMap::new
+    );
+
+    private final TrackingMarshaller trackingMarshaller;
+
+    BuiltInContainerMarshallers(TrackingMarshaller trackingMarshaller) {
+        this.trackingMarshaller = trackingMarshaller;
+    }
+
+    List<ClassDescriptor> writeGenericRefArray(Object[] array, ClassDescriptor arrayDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        output.writeUTF(array.getClass().getComponentType().getName());
+        return writeCollection(Arrays.asList(array), arrayDescriptor, output);
+    }
+
+    <T> T[] readGenericRefArray(DataInput input, ValueReader<T> elementReader)
+            throws IOException, UnmarshalException {
+        return BuiltInMarshalling.readGenericRefArray(input, elementReader);
+    }
+
+    List<ClassDescriptor> writeBuiltInCollection(Collection<?> object, ClassDescriptor descriptor, DataOutput output)
+            throws IOException, MarshalException {
+        if (supportsAsMutableBuiltInCollection(descriptor)) {
+            return writeCollection(object, descriptor, output);
+        } else if (descriptor.isSingletonList()) {
+            return writeSingletonList(object, descriptor, output);
+        } else {
+            throw new IllegalStateException("Marshalling of " + descriptor.clazz() + " is not supported, but it's marked as a built-in");
+        }
+    }
+
+    /**
+     * Returns {@code true} if the given descriptor is supported as a built-in mutable collection. Such types
+     * are eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     *
+     * @param descriptor the descriptor to check
+     * @return {@code true} if the given descriptor is supported as a built-in mutable collection
+     */
+    private boolean supportsAsMutableBuiltInCollection(ClassDescriptor descriptor) {
+        return mutableBuiltInCollectionFactories.containsKey(descriptor.clazz());
+    }
+
+    private List<ClassDescriptor> writeCollection(Collection<?> collection, ClassDescriptor collectionDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(collectionDescriptor);
+
+        BuiltInMarshalling.writeCollection(collection, output, writerAddingUsedDescriptor(usedDescriptors));
+
+        return List.copyOf(usedDescriptors);
+    }
+
+    private <T> ValueWriter<T> writerAddingUsedDescriptor(Set<ClassDescriptor> usedDescriptors) {
+        return (elem, out) -> {
+            List<ClassDescriptor> elementDescriptors = trackingMarshaller.marshal(elem, out);
+            usedDescriptors.addAll(elementDescriptors);
+        };
+    }
+
+    private List<ClassDescriptor> writeSingletonList(Collection<?> collection, ClassDescriptor listDescriptor, DataOutput output)
+            throws MarshalException, IOException {
+        assert collection.size() == 1;
+
+        Object element = collection.iterator().next();
+
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(listDescriptor);
+
+        List<ClassDescriptor> descriptorsFromElement = trackingMarshaller.marshal(element, output);
+        usedDescriptors.addAll(descriptorsFromElement);
+
+        return List.copyOf(usedDescriptors);
+    }
+
+    <T, C extends Collection<T>> C readBuiltInCollection(
+            ClassDescriptor collectionDescriptor,
+            ValueReader<T> elementReader,
+            DataInput input
+    ) throws UnmarshalException, IOException {
+        if (collectionDescriptor.isSingletonList()) {
+            @SuppressWarnings("unchecked")

Review comment:
       Looks really weird, how about putting it on the method? :D

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/BuiltInContainerMarshallers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.marshal;
+
+import static java.util.Collections.singletonList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.IntFunction;
+import org.apache.ignite.internal.network.serialization.ClassDescriptor;
+
+/**
+ * Utility to (un)marshal built-in collections and maps.
+ */
+class BuiltInContainerMarshallers {
+    /**
+     * Map of all classes which are built-in collections AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     */
+    private final Map<Class<?>, IntFunction<? extends Collection<?>>> mutableBuiltInCollectionFactories = Map.of(
+            ArrayList.class, ArrayList::new,
+            LinkedList.class, len -> new LinkedList<>(),
+            HashSet.class, HashSet::new,
+            LinkedHashSet.class, LinkedHashSet::new
+    );
+
+    /**
+     * Map of all classes which are built-in maps AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty map, then read N entries
+     * and put each of them into the map.
+     */
+    private final Map<Class<?>, IntFunction<? extends Map<?, ?>>> mutableBuiltInMapFactories = Map.of(
+            HashMap.class, HashMap::new,
+            LinkedHashMap.class, LinkedHashMap::new
+    );
+
+    private final TrackingMarshaller trackingMarshaller;
+
+    BuiltInContainerMarshallers(TrackingMarshaller trackingMarshaller) {
+        this.trackingMarshaller = trackingMarshaller;
+    }
+
+    List<ClassDescriptor> writeGenericRefArray(Object[] array, ClassDescriptor arrayDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        output.writeUTF(array.getClass().getComponentType().getName());
+        return writeCollection(Arrays.asList(array), arrayDescriptor, output);
+    }
+
+    <T> T[] readGenericRefArray(DataInput input, ValueReader<T> elementReader)
+            throws IOException, UnmarshalException {
+        return BuiltInMarshalling.readGenericRefArray(input, elementReader);
+    }
+
+    List<ClassDescriptor> writeBuiltInCollection(Collection<?> object, ClassDescriptor descriptor, DataOutput output)
+            throws IOException, MarshalException {
+        if (supportsAsMutableBuiltInCollection(descriptor)) {
+            return writeCollection(object, descriptor, output);
+        } else if (descriptor.isSingletonList()) {
+            return writeSingletonList(object, descriptor, output);
+        } else {
+            throw new IllegalStateException("Marshalling of " + descriptor.clazz() + " is not supported, but it's marked as a built-in");
+        }
+    }
+
+    /**
+     * Returns {@code true} if the given descriptor is supported as a built-in mutable collection. Such types
+     * are eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     *
+     * @param descriptor the descriptor to check
+     * @return {@code true} if the given descriptor is supported as a built-in mutable collection
+     */
+    private boolean supportsAsMutableBuiltInCollection(ClassDescriptor descriptor) {
+        return mutableBuiltInCollectionFactories.containsKey(descriptor.clazz());
+    }
+
+    private List<ClassDescriptor> writeCollection(Collection<?> collection, ClassDescriptor collectionDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(collectionDescriptor);
+
+        BuiltInMarshalling.writeCollection(collection, output, writerAddingUsedDescriptor(usedDescriptors));
+
+        return List.copyOf(usedDescriptors);
+    }
+
+    private <T> ValueWriter<T> writerAddingUsedDescriptor(Set<ClassDescriptor> usedDescriptors) {
+        return (elem, out) -> {
+            List<ClassDescriptor> elementDescriptors = trackingMarshaller.marshal(elem, out);
+            usedDescriptors.addAll(elementDescriptors);
+        };
+    }
+
+    private List<ClassDescriptor> writeSingletonList(Collection<?> collection, ClassDescriptor listDescriptor, DataOutput output)
+            throws MarshalException, IOException {
+        assert collection.size() == 1;
+
+        Object element = collection.iterator().next();
+
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(listDescriptor);
+
+        List<ClassDescriptor> descriptorsFromElement = trackingMarshaller.marshal(element, output);
+        usedDescriptors.addAll(descriptorsFromElement);
+
+        return List.copyOf(usedDescriptors);

Review comment:
       Why not simply return `usedDescriptors` set? 

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/BuiltInContainerMarshallers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.marshal;
+
+import static java.util.Collections.singletonList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.IntFunction;
+import org.apache.ignite.internal.network.serialization.ClassDescriptor;
+
+/**
+ * Utility to (un)marshal built-in collections and maps.
+ */
+class BuiltInContainerMarshallers {
+    /**
+     * Map of all classes which are built-in collections AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     */
+    private final Map<Class<?>, IntFunction<? extends Collection<?>>> mutableBuiltInCollectionFactories = Map.of(
+            ArrayList.class, ArrayList::new,
+            LinkedList.class, len -> new LinkedList<>(),
+            HashSet.class, HashSet::new,
+            LinkedHashSet.class, LinkedHashSet::new
+    );
+
+    /**
+     * Map of all classes which are built-in maps AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty map, then read N entries
+     * and put each of them into the map.
+     */
+    private final Map<Class<?>, IntFunction<? extends Map<?, ?>>> mutableBuiltInMapFactories = Map.of(
+            HashMap.class, HashMap::new,
+            LinkedHashMap.class, LinkedHashMap::new
+    );
+
+    private final TrackingMarshaller trackingMarshaller;
+
+    BuiltInContainerMarshallers(TrackingMarshaller trackingMarshaller) {
+        this.trackingMarshaller = trackingMarshaller;
+    }
+
+    List<ClassDescriptor> writeGenericRefArray(Object[] array, ClassDescriptor arrayDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        output.writeUTF(array.getClass().getComponentType().getName());
+        return writeCollection(Arrays.asList(array), arrayDescriptor, output);
+    }
+
+    <T> T[] readGenericRefArray(DataInput input, ValueReader<T> elementReader)
+            throws IOException, UnmarshalException {
+        return BuiltInMarshalling.readGenericRefArray(input, elementReader);
+    }
+
+    List<ClassDescriptor> writeBuiltInCollection(Collection<?> object, ClassDescriptor descriptor, DataOutput output)
+            throws IOException, MarshalException {
+        if (supportsAsMutableBuiltInCollection(descriptor)) {
+            return writeCollection(object, descriptor, output);
+        } else if (descriptor.isSingletonList()) {
+            return writeSingletonList(object, descriptor, output);
+        } else {
+            throw new IllegalStateException("Marshalling of " + descriptor.clazz() + " is not supported, but it's marked as a built-in");
+        }
+    }
+
+    /**
+     * Returns {@code true} if the given descriptor is supported as a built-in mutable collection. Such types
+     * are eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     *
+     * @param descriptor the descriptor to check
+     * @return {@code true} if the given descriptor is supported as a built-in mutable collection
+     */
+    private boolean supportsAsMutableBuiltInCollection(ClassDescriptor descriptor) {
+        return mutableBuiltInCollectionFactories.containsKey(descriptor.clazz());
+    }
+
+    private List<ClassDescriptor> writeCollection(Collection<?> collection, ClassDescriptor collectionDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(collectionDescriptor);
+
+        BuiltInMarshalling.writeCollection(collection, output, writerAddingUsedDescriptor(usedDescriptors));
+
+        return List.copyOf(usedDescriptors);
+    }
+
+    private <T> ValueWriter<T> writerAddingUsedDescriptor(Set<ClassDescriptor> usedDescriptors) {
+        return (elem, out) -> {
+            List<ClassDescriptor> elementDescriptors = trackingMarshaller.marshal(elem, out);
+            usedDescriptors.addAll(elementDescriptors);
+        };
+    }
+
+    private List<ClassDescriptor> writeSingletonList(Collection<?> collection, ClassDescriptor listDescriptor, DataOutput output)
+            throws MarshalException, IOException {
+        assert collection.size() == 1;
+
+        Object element = collection.iterator().next();

Review comment:
       I believe the argument's type can be `List` and here you can just do `.get(0)` to avoid the creation of an iterator

##########
File path: modules/network/src/main/java/org/apache/ignite/internal/network/serialization/marshal/BuiltInContainerMarshallers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.marshal;
+
+import static java.util.Collections.singletonList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.IntFunction;
+import org.apache.ignite.internal.network.serialization.ClassDescriptor;
+
+/**
+ * Utility to (un)marshal built-in collections and maps.
+ */
+class BuiltInContainerMarshallers {
+    /**
+     * Map of all classes which are built-in collections AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     */
+    private final Map<Class<?>, IntFunction<? extends Collection<?>>> mutableBuiltInCollectionFactories = Map.of(
+            ArrayList.class, ArrayList::new,
+            LinkedList.class, len -> new LinkedList<>(),
+            HashSet.class, HashSet::new,
+            LinkedHashSet.class, LinkedHashSet::new
+    );
+
+    /**
+     * Map of all classes which are built-in maps AND may have different sizes AND are mutable. This makes
+     * them eligible for a generic unmarshal algorithm: read length, create an empty map, then read N entries
+     * and put each of them into the map.
+     */
+    private final Map<Class<?>, IntFunction<? extends Map<?, ?>>> mutableBuiltInMapFactories = Map.of(
+            HashMap.class, HashMap::new,
+            LinkedHashMap.class, LinkedHashMap::new
+    );
+
+    private final TrackingMarshaller trackingMarshaller;
+
+    BuiltInContainerMarshallers(TrackingMarshaller trackingMarshaller) {
+        this.trackingMarshaller = trackingMarshaller;
+    }
+
+    List<ClassDescriptor> writeGenericRefArray(Object[] array, ClassDescriptor arrayDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        output.writeUTF(array.getClass().getComponentType().getName());
+        return writeCollection(Arrays.asList(array), arrayDescriptor, output);
+    }
+
+    <T> T[] readGenericRefArray(DataInput input, ValueReader<T> elementReader)
+            throws IOException, UnmarshalException {
+        return BuiltInMarshalling.readGenericRefArray(input, elementReader);
+    }
+
+    List<ClassDescriptor> writeBuiltInCollection(Collection<?> object, ClassDescriptor descriptor, DataOutput output)
+            throws IOException, MarshalException {
+        if (supportsAsMutableBuiltInCollection(descriptor)) {
+            return writeCollection(object, descriptor, output);
+        } else if (descriptor.isSingletonList()) {
+            return writeSingletonList(object, descriptor, output);
+        } else {
+            throw new IllegalStateException("Marshalling of " + descriptor.clazz() + " is not supported, but it's marked as a built-in");
+        }
+    }
+
+    /**
+     * Returns {@code true} if the given descriptor is supported as a built-in mutable collection. Such types
+     * are eligible for a generic unmarshal algorithm: read length, create an empty collection, then read N elements
+     * and add each of them into the collection.
+     *
+     * @param descriptor the descriptor to check
+     * @return {@code true} if the given descriptor is supported as a built-in mutable collection
+     */
+    private boolean supportsAsMutableBuiltInCollection(ClassDescriptor descriptor) {
+        return mutableBuiltInCollectionFactories.containsKey(descriptor.clazz());
+    }
+
+    private List<ClassDescriptor> writeCollection(Collection<?> collection, ClassDescriptor collectionDescriptor, DataOutput output)
+            throws IOException, MarshalException {
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(collectionDescriptor);
+
+        BuiltInMarshalling.writeCollection(collection, output, writerAddingUsedDescriptor(usedDescriptors));
+
+        return List.copyOf(usedDescriptors);
+    }
+
+    private <T> ValueWriter<T> writerAddingUsedDescriptor(Set<ClassDescriptor> usedDescriptors) {
+        return (elem, out) -> {
+            List<ClassDescriptor> elementDescriptors = trackingMarshaller.marshal(elem, out);
+            usedDescriptors.addAll(elementDescriptors);
+        };
+    }
+
+    private List<ClassDescriptor> writeSingletonList(Collection<?> collection, ClassDescriptor listDescriptor, DataOutput output)
+            throws MarshalException, IOException {
+        assert collection.size() == 1;
+
+        Object element = collection.iterator().next();
+
+        Set<ClassDescriptor> usedDescriptors = new HashSet<>();
+        usedDescriptors.add(listDescriptor);
+
+        List<ClassDescriptor> descriptorsFromElement = trackingMarshaller.marshal(element, output);
+        usedDescriptors.addAll(descriptorsFromElement);
+
+        return List.copyOf(usedDescriptors);
+    }
+
+    <T, C extends Collection<T>> C readBuiltInCollection(
+            ClassDescriptor collectionDescriptor,
+            ValueReader<T> elementReader,
+            DataInput input
+    ) throws UnmarshalException, IOException {
+        if (collectionDescriptor.isSingletonList()) {
+            @SuppressWarnings("unchecked")
+            C castResult = (C) singletonList(elementReader.read(input));

Review comment:
       castResult?




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