This is an automated email from the ASF dual-hosted git repository.
av pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git
The following commit(s) were added to refs/heads/master by this push:
new c9d7c438e8c IGNITE-27953 Use message serializer for
CacheAffinityChangeMessage (#12867)
c9d7c438e8c is described below
commit c9d7c438e8c49055220cc672d5204647f6d0a5d2
Author: Anton Vinogradov <[email protected]>
AuthorDate: Tue Mar 10 23:06:42 2026 +0300
IGNITE-27953 Use message serializer for CacheAffinityChangeMessage (#12867)
---
.../internal/MessageSerializerGenerator.java | 124 ++++++++++++---------
.../internal/direct/DirectMessageReader.java | 31 ++----
.../internal/direct/DirectMessageWriter.java | 25 ++---
.../direct/stream/DirectByteBufferStream.java | 120 ++++++++++----------
.../discovery/DiscoveryMessageFactory.java | 3 +
.../cache/CacheAffinityChangeMessage.java | 37 ++++--
.../extensions/communication/MessageArrayType.java | 51 +++++++++
.../communication/MessageCollectionItemType.java | 13 ++-
.../communication/MessageCollectionType.java | 51 +++++++++
.../extensions/communication/MessageItemType.java | 36 ++++++
.../extensions/communication/MessageMapType.java | 61 ++++++++++
.../extensions/communication/MessageReader.java | 38 ++-----
.../extensions/communication/MessageType.java | 26 +++++
.../extensions/communication/MessageWriter.java | 33 ++----
.../AbstractMessageSerializationTest.java | 42 +++----
.../communication/GridCacheMessageSelfTest.java | 7 +-
.../codegen/TestCollectionsMessageSerializer.java | 104 ++++++++---------
.../src/test/resources/codegen/TestMapMessage.java | 4 +
.../codegen/TestMapMessageSerializer.java | 115 +++++++++++--------
.../resources/codegen/TestMessageSerializer.java | 14 ++-
20 files changed, 589 insertions(+), 346 deletions(-)
diff --git
a/modules/codegen/src/main/java/org/apache/ignite/internal/MessageSerializerGenerator.java
b/modules/codegen/src/main/java/org/apache/ignite/internal/MessageSerializerGenerator.java
index 355072e0b30..41d93c4e260 100644
---
a/modules/codegen/src/main/java/org/apache/ignite/internal/MessageSerializerGenerator.java
+++
b/modules/codegen/src/main/java/org/apache/ignite/internal/MessageSerializerGenerator.java
@@ -28,6 +28,7 @@ import java.io.Writer;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collection;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@@ -400,10 +401,7 @@ public class MessageSerializerGenerator {
return;
}
-
imports.add("org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType");
-
- returnFalseIfWriteFailed(write, field, "writer.writeObjectArray",
getExpr,
- "MessageCollectionItemType." +
messageCollectionItemType(componentType));
+ returnFalseIfWriteFailed(write, field, "writer.writeObjectArray",
getExpr, messageCollectionItemTypes(type));
return;
}
@@ -425,16 +423,10 @@ public class MessageSerializerGenerator {
returnFalseIfWriteFailed(write, field,
"writer.writeAffinityTopologyVersion", getExpr);
else if (assignableFrom(erasedType(type),
type(Map.class.getName()))) {
- List<? extends TypeMirror> typeArgs =
((DeclaredType)type).getTypeArguments();
-
- assert typeArgs.size() == 2;
-
-
imports.add("org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType");
-
List<String> args = new ArrayList<>();
+
args.add(getExpr);
- args.add("MessageCollectionItemType." +
messageCollectionItemType(typeArgs.get(0)));
- args.add("MessageCollectionItemType." +
messageCollectionItemType(typeArgs.get(1)));
+ args.add(messageCollectionItemTypes(type));
if (compress)
args.add("true"); // the value of the compress argument in
the MessageWriter#writeMap method
@@ -461,20 +453,8 @@ public class MessageSerializerGenerator {
returnFalseIfWriteFailed(write, field,
"writer.writeMessage", getExpr);
}
- else if (assignableFrom(erasedType(type),
type(Collection.class.getName()))) {
- List<? extends TypeMirror> typeArgs =
((DeclaredType)type).getTypeArguments();
-
- assert typeArgs.size() == 1;
-
-
imports.add("org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType");
-
- String collectionWriter = assignableFrom(erasedType(type),
type(Set.class.getName()))
- ? "writer.writeSet"
- : "writer.writeCollection";
-
- returnFalseIfWriteFailed(write, field, collectionWriter,
getExpr,
- "MessageCollectionItemType." +
messageCollectionItemType(typeArgs.get(0)));
- }
+ else if (assignableFrom(erasedType(type),
type(Collection.class.getName())))
+ returnFalseIfWriteFailed(write, field,
"writer.writeCollection", getExpr, messageCollectionItemTypes(type));
else if (enumType(env, type)) {
Element element = env.getTypeUtils().asElement(type);
@@ -641,13 +621,7 @@ public class MessageSerializerGenerator {
}
if (componentType.getKind() == TypeKind.ARRAY) {
- TypeMirror ctype =
((ArrayType)componentType).getComponentType();
-
- assert ctype.getKind().isPrimitive();
-
- returnFalseIfReadFailed(field, "reader.readObjectArray",
- "MessageCollectionItemType." +
messageCollectionItemType(ctype),
- ctype.getKind().name().toLowerCase() + "[].class");
+ returnFalseIfReadFailed(field, "reader.readObjectArray",
messageCollectionItemTypes(type));
return;
}
@@ -655,11 +629,7 @@ public class MessageSerializerGenerator {
if (componentType.getKind() == TypeKind.DECLARED) {
Element componentElement =
((DeclaredType)componentType).asElement();
- String cls = componentElement.getSimpleName().toString();
-
- returnFalseIfReadFailed(field, "reader.readObjectArray",
- "MessageCollectionItemType." +
messageCollectionItemType(componentType),
- cls + ".class");
+ returnFalseIfReadFailed(field, "reader.readObjectArray",
messageCollectionItemTypes(type));
if
(!"java.lang".equals(env.getElementUtils().getPackageOf(componentElement).getQualifiedName().toString()))
{
String importCls =
((QualifiedNameable)componentElement).getQualifiedName().toString();
@@ -693,9 +663,8 @@ public class MessageSerializerGenerator {
assert typeArgs.size() == 2;
List<String> args = new ArrayList<>();
- args.add("MessageCollectionItemType." +
messageCollectionItemType(typeArgs.get(0)));
- args.add("MessageCollectionItemType." +
messageCollectionItemType(typeArgs.get(1)));
- args.add("false"); // the value of the linked argument in the
MessageReader#readMap method
+
+ args.add(messageCollectionItemTypes(type));
if (compress)
args.add("true"); // the value of the compress argument in
the MessageReader#readMap method
@@ -723,16 +692,7 @@ public class MessageSerializerGenerator {
}
else if (assignableFrom(erasedType(type),
type(Collection.class.getName()))) {
- List<? extends TypeMirror> typeArgs =
((DeclaredType)type).getTypeArguments();
-
- assert typeArgs.size() == 1;
-
- String collectionReader = assignableFrom(erasedType(type),
type(Set.class.getName()))
- ? "reader.readSet"
- : "reader.readCollection";
-
- returnFalseIfReadFailed(field, collectionReader,
- "MessageCollectionItemType." +
messageCollectionItemType(typeArgs.get(0)));
+ returnFalseIfReadFailed(field, "reader.readCollection",
messageCollectionItemTypes(type));
}
else if (enumType(env, type)) {
String fieldPrefix =
typeNameToFieldName(env.getTypeUtils().asElement(type).getSimpleName().toString());
@@ -754,6 +714,68 @@ public class MessageSerializerGenerator {
throw new IllegalArgumentException("Unsupported type kind: " +
type.getKind());
}
+ /** */
+ private String messageCollectionItemTypes(TypeMirror type) throws
Exception {
+
imports.add("org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType");
+
+ if (type.getKind() == TypeKind.ARRAY) {
+ ArrayType arrType = (ArrayType)type;
+ TypeMirror componentType = arrType.getComponentType();
+
+ String clazz;
+
+ if (componentType.getKind() == TypeKind.ARRAY) {
+ TypeMirror ctype =
((ArrayType)componentType).getComponentType();
+
+ clazz = ctype.getKind().name().toLowerCase() + "[].class";
+ }
+ else if (componentType.getKind() == TypeKind.DECLARED) {
+ Element componentElement =
((DeclaredType)componentType).asElement();
+
+ clazz = componentElement.getSimpleName() + ".class";
+ }
+ else {
+ assert componentType.getKind().isPrimitive();
+
+
imports.add("org.apache.ignite.plugin.extensions.communication.MessageItemType");
+
+ return "new MessageItemType(MessageCollectionItemType." +
messageCollectionItemType(componentType) + "_ARR)";
+ }
+
+
imports.add("org.apache.ignite.plugin.extensions.communication.MessageArrayType");
+
+ return "new MessageArrayType(" +
messageCollectionItemTypes(componentType) + ", " + clazz + ")";
+ }
+ else if (assignableFrom(erasedType(type), type(Map.class.getName()))) {
+
imports.add("org.apache.ignite.plugin.extensions.communication.MessageMapType");
+
+ List<? extends TypeMirror> typeArgs =
((DeclaredType)type).getTypeArguments();
+
+ assert typeArgs.size() == 2;
+
+ return "new MessageMapType(" +
+ messageCollectionItemTypes(typeArgs.get(0)) + ", " +
+ messageCollectionItemTypes(typeArgs.get(1)) + ", " +
+ assignableFrom(erasedType(type),
type(LinkedHashMap.class.getName())) + ")";
+ }
+ else if (assignableFrom(erasedType(type),
type(Collection.class.getName()))) {
+
imports.add("org.apache.ignite.plugin.extensions.communication.MessageCollectionType");
+
+ List<? extends TypeMirror> typeArgs =
((DeclaredType)type).getTypeArguments();
+
+ assert typeArgs.size() == 1;
+
+ return "new MessageCollectionType(" +
+ messageCollectionItemTypes(typeArgs.get(0)) + ", " +
+ assignableFrom(erasedType(type), type(Set.class.getName())) +
")";
+ }
+ else {
+
imports.add("org.apache.ignite.plugin.extensions.communication.MessageItemType");
+
+ return "new MessageItemType(MessageCollectionItemType." +
messageCollectionItemType(type) + ")";
+ }
+ }
+
/**
* Find MessageCollectionItemType for a given type.
* <p>
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
index 3217d7630a0..cce73aee5b1 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
import java.util.BitSet;
import java.util.Collection;
import java.util.Map;
-import java.util.Set;
import java.util.UUID;
import java.util.function.Function;
import org.apache.ignite.internal.direct.state.DirectMessageState;
@@ -38,8 +37,10 @@ import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.lang.IgniteOutClosure;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.plugin.extensions.communication.Message;
-import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageArrayType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
import org.apache.ignite.plugin.extensions.communication.MessageFactory;
+import org.apache.ignite.plugin.extensions.communication.MessageMapType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.jetbrains.annotations.Nullable;
@@ -379,10 +380,10 @@ public class DirectMessageReader implements MessageReader
{
}
/** {@inheritDoc} */
- @Override public <T> T[] readObjectArray(MessageCollectionItemType
itemType, Class<T> itemCls) {
+ @Override public <T> T[] readObjectArray(MessageArrayType type) {
DirectByteBufferStream stream = state.item().stream;
- T[] msg = stream.readObjectArray(itemType, itemCls, this);
+ T[] msg = stream.readObjectArray(type, this);
lastRead = stream.lastFinished();
@@ -390,10 +391,10 @@ public class DirectMessageReader implements MessageReader
{
}
/** {@inheritDoc} */
- @Override public <C extends Collection<?>> C
readCollection(MessageCollectionItemType itemType) {
+ @Override public <C extends Collection<?>> C
readCollection(MessageCollectionType type) {
DirectByteBufferStream stream = state.item().stream;
- C col = stream.readList(itemType, this);
+ C col = stream.readCollection(type, this);
lastRead = stream.lastFinished();
@@ -401,19 +402,7 @@ public class DirectMessageReader implements MessageReader {
}
/** {@inheritDoc} */
- @Override public <SET extends Set<?>> SET
readSet(MessageCollectionItemType itemType) {
- DirectByteBufferStream stream = state.item().stream;
-
- SET set = stream.readSet(itemType, this);
-
- lastRead = stream.lastFinished();
-
- return set;
- }
-
- /** {@inheritDoc} */
- @Override public <M extends Map<?, ?>> M readMap(MessageCollectionItemType
keyType,
- MessageCollectionItemType valType, boolean linked, boolean compress) {
+ @Override public <M extends Map<?, ?>> M readMap(MessageMapType type,
boolean compress) {
DirectByteBufferStream stream = state.item().stream;
M map;
@@ -421,10 +410,10 @@ public class DirectMessageReader implements MessageReader
{
if (compress)
map = readCompressedMessageAndDeserialize(
stream,
- tmpReader -> tmpReader.state.item().stream.readMap(keyType,
valType, linked, tmpReader)
+ tmpReader -> tmpReader.state.item().stream.readMap(type,
tmpReader)
);
else {
- map = stream.readMap(keyType, valType, linked, this);
+ map = stream.readMap(type, this);
lastRead = stream.lastFinished();
}
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
index 2cb363f1bd4..db35c93e165 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageWriter.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
import java.util.BitSet;
import java.util.Collection;
import java.util.Map;
-import java.util.Set;
import java.util.UUID;
import java.util.function.Consumer;
import org.apache.ignite.internal.direct.state.DirectMessageState;
@@ -37,8 +36,10 @@ import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.lang.IgniteOutClosure;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.plugin.extensions.communication.Message;
-import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageArrayType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
import org.apache.ignite.plugin.extensions.communication.MessageFactory;
+import org.apache.ignite.plugin.extensions.communication.MessageMapType;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
import org.jetbrains.annotations.Nullable;
@@ -360,41 +361,35 @@ public class DirectMessageWriter implements MessageWriter
{
}
/** {@inheritDoc} */
- @Override public <T> boolean writeObjectArray(T[] arr,
MessageCollectionItemType itemType) {
+ @Override public <T> boolean writeObjectArray(T[] arr, MessageArrayType
type) {
DirectByteBufferStream stream = state.item().stream;
- stream.writeObjectArray(arr, itemType, this);
+ stream.writeObjectArray(arr, type, this);
return stream.lastFinished();
}
/** {@inheritDoc} */
- @Override public <T> boolean writeCollection(Collection<T> col,
MessageCollectionItemType itemType) {
+ @Override public <T> boolean writeCollection(Collection<T> col,
MessageCollectionType type) {
DirectByteBufferStream stream = state.item().stream;
- stream.writeCollection(col, itemType, this);
+ stream.writeCollection(col, type, this);
return stream.lastFinished();
}
/** {@inheritDoc} */
- @Override public <T> boolean writeSet(Set<T> set,
MessageCollectionItemType itemType) {
- return writeCollection(set, itemType);
- }
-
- /** {@inheritDoc} */
- @Override public <K, V> boolean writeMap(Map<K, V> map,
MessageCollectionItemType keyType,
- MessageCollectionItemType valType, boolean compress) {
+ @Override public <K, V> boolean writeMap(Map<K, V> map, MessageMapType
type, boolean compress) {
DirectByteBufferStream stream = state.item().stream;
if (compress)
writeCompressedMessage(
- tmpWriter -> tmpWriter.state.item().stream.writeMap(map,
keyType, valType, tmpWriter),
+ tmpWriter -> tmpWriter.state.item().stream.writeMap(map, type,
tmpWriter),
map == null,
stream
);
else
- stream.writeMap(map, keyType, valType, this);
+ stream.writeMap(map, type, this);
return stream.lastFinished();
}
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
index ca3ecfc7fbc..6890200d5f7 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/DirectByteBufferStream.java
@@ -27,7 +27,6 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.RandomAccess;
-import java.util.Set;
import java.util.UUID;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
@@ -43,9 +42,12 @@ import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.plugin.extensions.communication.Message;
-import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageArrayType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
import org.apache.ignite.plugin.extensions.communication.MessageFactory;
+import org.apache.ignite.plugin.extensions.communication.MessageMapType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageType;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
import org.jetbrains.annotations.Nullable;
@@ -938,10 +940,10 @@ public class DirectByteBufferStream {
/**
* @param arr Array.
- * @param itemType Component type.
+ * @param type Type.
* @param writer Writer.
*/
- public <T> void writeObjectArray(T[] arr, MessageCollectionItemType
itemType, MessageWriter writer) {
+ public <T> void writeObjectArray(T[] arr, MessageArrayType type,
MessageWriter writer) {
if (arr != null) {
int len = arr.length;
@@ -958,7 +960,7 @@ public class DirectByteBufferStream {
if (arrCur == NULL)
arrCur = arr[arrPos++];
- write(itemType, arrCur, writer);
+ write(type.valueType(), arrCur, writer);
if (!lastFinished)
return;
@@ -974,13 +976,13 @@ public class DirectByteBufferStream {
/**
* @param col Collection.
- * @param itemType Component type.
+ * @param type Type.
* @param writer Writer.
*/
- public <T> void writeCollection(Collection<T> col,
MessageCollectionItemType itemType, MessageWriter writer) {
+ public <T> void writeCollection(Collection<T> col, MessageCollectionType
type, MessageWriter writer) {
if (col != null) {
if (col instanceof List && col instanceof RandomAccess)
- writeRandomAccessList((List<T>)col, itemType, writer);
+ writeRandomAccessList((List<T>)col, type, writer);
else {
if (it == null) {
writeInt(col.size());
@@ -995,7 +997,7 @@ public class DirectByteBufferStream {
if (cur == NULL)
cur = it.next();
- write(itemType, cur, writer);
+ write(type.valueType(), cur, writer);
if (!lastFinished)
return;
@@ -1012,10 +1014,10 @@ public class DirectByteBufferStream {
/**
* @param list List.
- * @param itemType Component type.
+ * @param type Type.
* @param writer Writer.
*/
- private <T> void writeRandomAccessList(List<T> list,
MessageCollectionItemType itemType, MessageWriter writer) {
+ private <T> void writeRandomAccessList(List<T> list, MessageCollectionType
type, MessageWriter writer) {
assert list instanceof RandomAccess;
int size = list.size();
@@ -1033,7 +1035,7 @@ public class DirectByteBufferStream {
if (arrCur == NULL)
arrCur = list.get(arrPos++);
- write(itemType, arrCur, writer);
+ write(type.valueType(), arrCur, writer);
if (!lastFinished)
return;
@@ -1046,11 +1048,10 @@ public class DirectByteBufferStream {
/**
* @param map Map.
- * @param keyType Key type.
- * @param valType Value type.
+ * @param type Type.
* @param writer Writer.
*/
- public <K, V> void writeMap(Map<K, V> map, MessageCollectionItemType
keyType, MessageCollectionItemType valType, MessageWriter writer) {
+ public <K, V> void writeMap(Map<K, V> map, MessageMapType type,
MessageWriter writer) {
if (map != null) {
if (mapIt == null) {
writeInt(map.size());
@@ -1070,7 +1071,7 @@ public class DirectByteBufferStream {
e = (Map.Entry<K, V>)mapCur;
if (!keyDone) {
- write(keyType, e.getKey(), writer);
+ write(type.keyType(), e.getKey(), writer);
if (!lastFinished)
return;
@@ -1078,7 +1079,7 @@ public class DirectByteBufferStream {
keyDone = true;
}
- write(valType, e.getValue(), writer);
+ write(type.valueType(), e.getValue(), writer);
if (!lastFinished)
return;
@@ -1597,12 +1598,11 @@ public class DirectByteBufferStream {
}
/**
- * @param itemType Item type.
- * @param itemCls Item class.
+ * @param type Item type.
* @param reader Reader.
* @return Array.
*/
- public <T> T[] readObjectArray(MessageCollectionItemType itemType,
Class<T> itemCls, MessageReader reader) {
+ public <T> T[] readObjectArray(MessageArrayType type, MessageReader
reader) {
if (readSize == -1) {
int size = readInt();
@@ -1614,10 +1614,10 @@ public class DirectByteBufferStream {
if (readSize >= 0) {
if (objArr == null)
- objArr = itemCls != null ?
(Object[])Array.newInstance(itemCls, readSize) : new Object[readSize];
+ objArr = type.clazz() != null ?
(Object[])Array.newInstance(type.clazz(), readSize) : new Object[readSize];
for (int i = readItems; i < readSize; i++) {
- Object item = read(itemType, reader);
+ Object item = read(type.valueType(), reader);
if (!lastFinished)
return null;
@@ -1639,37 +1639,14 @@ public class DirectByteBufferStream {
return objArr0;
}
- /**
- * Reads collection as an {@link ArrayList}.
- *
- * @param itemType Item type.
- * @param reader Reader.
- * @return {@link ArrayList}.
- */
- public <L extends List<?>> L readList(MessageCollectionItemType itemType,
MessageReader reader) {
- return readCollection(itemType, reader, false);
- }
-
- /**
- * Reads collection as a {@link HashSet}.
- *
- * @param itemType Item type.
- * @param reader Reader.
- * @return {@link HashSet}.
- */
- public <SET extends Set<?>> SET readSet(MessageCollectionItemType
itemType, MessageReader reader) {
- return readCollection(itemType, reader, true);
- }
-
/**
* Reads collection eather as a {@link ArrayList} or a {@link HashSet}.
*
- * @param itemType Item type.
+ * @param type Item type.
* @param reader Reader.
- * @param set Read-as-Set flag.
* @return {@link ArrayList} or a {@link HashSet}.
*/
- private <C extends Collection<?>> C
readCollection(MessageCollectionItemType itemType, MessageReader reader,
boolean set) {
+ public <C extends Collection<?>> C readCollection(MessageCollectionType
type, MessageReader reader) {
if (readSize == -1) {
int size = readInt();
@@ -1681,10 +1658,10 @@ public class DirectByteBufferStream {
if (readSize >= 0) {
if (col == null)
- col = set ? U.newHashSet(readSize) : new ArrayList<>(readSize);
+ col = type.set() ? U.newHashSet(readSize) : new
ArrayList<>(readSize);
for (int i = readItems; i < readSize; i++) {
- Object item = read(itemType, reader);
+ Object item = read(type.valueType(), reader);
if (!lastFinished)
return null;
@@ -1707,14 +1684,11 @@ public class DirectByteBufferStream {
}
/**
- * @param keyType Key type.
- * @param valType Value type.
- * @param linked Whether linked map should be created.
+ * @param type Value type.
* @param reader Reader.
* @return Map.
*/
- public <M extends Map<?, ?>> M readMap(MessageCollectionItemType keyType,
MessageCollectionItemType valType,
- boolean linked, MessageReader
reader) {
+ public <M extends Map<?, ?>> M readMap(MessageMapType type, MessageReader
reader) {
if (readSize == -1) {
int size = readInt();
@@ -1726,11 +1700,11 @@ public class DirectByteBufferStream {
if (readSize >= 0) {
if (map == null)
- map = linked ? U.newLinkedHashMap(readSize) :
U.newHashMap(readSize);
+ map = type.linked() ? U.newLinkedHashMap(readSize) :
U.newHashMap(readSize);
for (int i = readItems; i < readSize; i++) {
if (!keyDone) {
- Object key = read(keyType, reader);
+ Object key = read(type.keyType(), reader);
if (!lastFinished)
return null;
@@ -1739,7 +1713,7 @@ public class DirectByteBufferStream {
keyDone = true;
}
- Object val = read(valType, reader);
+ Object val = read(type.valueType(), reader);
if (!lastFinished)
return null;
@@ -2020,8 +1994,8 @@ public class DirectByteBufferStream {
* @param val Value.
* @param writer Writer.
*/
- protected void write(MessageCollectionItemType type, Object val,
MessageWriter writer) {
- switch (type) {
+ protected <K, V> void write(MessageType type, Object val, MessageWriter
writer) {
+ switch (type.type()) {
case BYTE:
writeByte((Byte)val);
@@ -2142,6 +2116,21 @@ public class DirectByteBufferStream {
break;
+ case MAP:
+ writeMap((Map<K, V>)val, (MessageMapType)type, writer);
+
+ break;
+
+ case COLLECTION:
+ writeCollection((Collection<V>)val,
(MessageCollectionType)type, writer);
+
+ break;
+
+ case ARRAY:
+ writeObjectArray((V[])val, (MessageArrayType)type, writer);
+
+ break;
+
case MSG:
try {
if (val != null)
@@ -2166,8 +2155,8 @@ public class DirectByteBufferStream {
* @param reader Reader.
* @return Value.
*/
- protected Object read(MessageCollectionItemType type, MessageReader
reader) {
- switch (type) {
+ protected Object read(MessageType type, MessageReader reader) {
+ switch (type.type()) {
case BYTE:
return readByte();
@@ -2240,6 +2229,15 @@ public class DirectByteBufferStream {
case GRID_LONG_LIST:
return readGridLongList();
+ case MAP:
+ return readMap((MessageMapType)type, reader);
+
+ case COLLECTION:
+ return readCollection((MessageCollectionType)type, reader);
+
+ case ARRAY:
+ return readObjectArray((MessageArrayType)type, reader);
+
case MSG:
return readMessage(reader);
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java
b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java
index 14272ac0f0e..83a6271af1c 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java
@@ -27,6 +27,8 @@ import
org.apache.ignite.internal.processors.authentication.UserManagementOperat
import
org.apache.ignite.internal.processors.authentication.UserProposedMessage;
import
org.apache.ignite.internal.processors.authentication.UserProposedMessageSerializer;
import org.apache.ignite.internal.processors.authentication.UserSerializer;
+import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
+import
org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessageSerializer;
import
org.apache.ignite.internal.processors.cache.CacheStatisticsModeChangeMessage;
import
org.apache.ignite.internal.processors.cache.CacheStatisticsModeChangeMessageSerializer;
import
org.apache.ignite.internal.processors.cache.TxTimeoutOnPartitionMapExchangeChangeMessage;
@@ -217,5 +219,6 @@ public class DiscoveryMessageFactory implements
MessageFactoryProvider {
factory.register((short)512, ChangeGlobalStateFinishMessage::new, new
ChangeGlobalStateFinishMessageSerializer());
factory.register((short)513, StopRoutineAckDiscoveryMessage::new, new
StopRoutineAckDiscoveryMessageSerializer());
factory.register((short)514, StopRoutineDiscoveryMessage::new, new
StopRoutineDiscoveryMessageSerializer());
+ factory.register((short)515, CacheAffinityChangeMessage::new, new
CacheAffinityChangeMessageSerializer());
}
}
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
index 748693473b4..6026449475c 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
import java.util.List;
import java.util.Map;
import java.util.UUID;
+import org.apache.ignite.internal.Order;
import org.apache.ignite.internal.managers.discovery.DiscoCache;
import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
@@ -28,6 +29,7 @@ import
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
import
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
import org.jetbrains.annotations.Nullable;
@@ -35,30 +37,37 @@ import org.jetbrains.annotations.Nullable;
* CacheAffinityChangeMessage represent a message that switches to a new
affinity assignmentafter rebalance is finished.
* This message should not be mutated in any way outside the
"disco-notifier-worker" thread.
*/
-public class CacheAffinityChangeMessage implements DiscoveryCustomMessage {
+public class CacheAffinityChangeMessage implements DiscoveryCustomMessage,
Message {
/** */
private static final long serialVersionUID = 0L;
/** */
- private IgniteUuid id = IgniteUuid.randomUuid();
+ @Order(0)
+ IgniteUuid id;
/** */
- private AffinityTopologyVersion topVer;
+ @Order(1)
+ AffinityTopologyVersion topVer;
/** */
- private GridDhtPartitionExchangeId exchId;
+ @Order(2)
+ GridDhtPartitionExchangeId exchId;
/** */
- private Map<Integer, Map<Integer, List<UUID>>> assignmentChange;
+ @Order(3)
+ Map<Integer, Map<Integer, List<UUID>>> assignmentChange;
/** */
- private Map<Integer, IgniteUuid> cacheDeploymentIds;
+ @Order(4)
+ Map<Integer, IgniteUuid> cacheDeploymentIds;
/** */
- private GridDhtPartitionsFullMessage partsMsg;
+ @Order(5)
+ GridDhtPartitionsFullMessage partsMsg;
/** If this flag is {@code true} then this message should lead to
partition map exchnage. */
- private boolean exchangeNeeded;
+ @Order(6)
+ boolean exchangeNeeded;
/**
* This flag indicates that this message should not be passed to other
nodes except the coordinator.
@@ -68,7 +77,10 @@ public class CacheAffinityChangeMessage implements
DiscoveryCustomMessage {
* This flag is used when discovery SPI does not support mutable custom
messages.
* See {@link DiscoverySpiMutableCustomMessageSupport}.
*/
- private transient boolean stopProc;
+ private boolean stopProc;
+
+ /** */
+ public CacheAffinityChangeMessage() {}
/**
* Constructor used when message is created after cache rebalance finished.
@@ -77,6 +89,7 @@ public class CacheAffinityChangeMessage implements
DiscoveryCustomMessage {
* @param cacheDeploymentIds Cache deployment ID.
*/
public CacheAffinityChangeMessage(AffinityTopologyVersion topVer,
Map<Integer, IgniteUuid> cacheDeploymentIds) {
+ id = IgniteUuid.randomUuid();
this.topVer = topVer;
this.cacheDeploymentIds = cacheDeploymentIds;
}
@@ -92,6 +105,7 @@ public class CacheAffinityChangeMessage implements
DiscoveryCustomMessage {
GridDhtPartitionExchangeId exchId,
GridDhtPartitionsFullMessage partsMsg,
Map<Integer, Map<Integer, List<UUID>>> assignmentChange) {
+ id = IgniteUuid.randomUuid();
this.exchId = exchId;
this.partsMsg = partsMsg;
this.assignmentChange = assignmentChange;
@@ -194,6 +208,11 @@ public class CacheAffinityChangeMessage implements
DiscoveryCustomMessage {
return discoCache.copy(topVer, null);
}
+ /** {@inheritDoc} */
+ @Override public short directType() {
+ return 515;
+ }
+
/** {@inheritDoc} */
@Override public String toString() {
return S.toString(CacheAffinityChangeMessage.class, this);
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageArrayType.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageArrayType.java
new file mode 100644
index 00000000000..1fa3fe3d216
--- /dev/null
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageArrayType.java
@@ -0,0 +1,51 @@
+/*
+ * 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.plugin.extensions.communication;
+
+/** */
+public class MessageArrayType implements MessageType {
+ /** */
+ private final MessageType valType;
+
+ /** */
+ private final Class<?> clazz;
+
+ /**
+ * @param valType Value type.
+ * @param clazz Class.
+ */
+ public MessageArrayType(MessageType valType, Class<?> clazz) {
+ this.valType = valType;
+ this.clazz = clazz;
+ }
+
+ /** @return Value type. */
+ public MessageType valueType() {
+ return valType;
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageCollectionItemType type() {
+ return MessageCollectionItemType.ARRAY;
+ }
+
+ /** @return Class. */
+ public Class<?> clazz() {
+ return clazz;
+ }
+}
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
index b994809b77d..b28e67297ce 100644
---
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
@@ -95,8 +95,17 @@ public enum MessageCollectionItemType {
/** Cache object. */
CACHE_OBJECT,
- /** GridLongList */
- GRID_LONG_LIST;
+ /** GridLongList. */
+ GRID_LONG_LIST,
+
+ /** Map. */
+ MAP,
+
+ /** Collection. */
+ COLLECTION,
+
+ /** Array. */
+ ARRAY;
/** Enum values. */
private static final MessageCollectionItemType[] VALS = values();
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionType.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionType.java
new file mode 100644
index 00000000000..f6d982de72a
--- /dev/null
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionType.java
@@ -0,0 +1,51 @@
+/*
+ * 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.plugin.extensions.communication;
+
+/** */
+public class MessageCollectionType implements MessageType {
+ /** */
+ private final MessageType valType;
+
+ /** */
+ private final boolean set;
+
+ /**
+ * @param valType Value type.
+ * @param set Is set.
+ */
+ public MessageCollectionType(MessageType valType, boolean set) {
+ this.valType = valType;
+ this.set = set;
+ }
+
+ /** @return Value type. */
+ public MessageType valueType() {
+ return valType;
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageCollectionItemType type() {
+ return MessageCollectionItemType.COLLECTION;
+ }
+
+ /** @return Is set. */
+ public boolean set() {
+ return set;
+ }
+}
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageItemType.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageItemType.java
new file mode 100644
index 00000000000..77664ebba95
--- /dev/null
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageItemType.java
@@ -0,0 +1,36 @@
+/*
+ * 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.plugin.extensions.communication;
+
+/** */
+public class MessageItemType implements MessageType {
+ /** */
+ private final MessageCollectionItemType type;
+
+ /**
+ @param type Type.
+ */
+ public MessageItemType(MessageCollectionItemType type) {
+ this.type = type;
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageCollectionItemType type() {
+ return type;
+ }
+}
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageMapType.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageMapType.java
new file mode 100644
index 00000000000..978a101e60b
--- /dev/null
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageMapType.java
@@ -0,0 +1,61 @@
+/*
+ * 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.plugin.extensions.communication;
+
+/** */
+public class MessageMapType implements MessageType {
+ /** */
+ private final MessageType keyType;
+
+ /** */
+ private final MessageType valType;
+
+ /** */
+ private final boolean linked;
+
+ /**
+ * @param keyType Key type.
+ * @param valType Value type.
+ * @param linked Is linked hash map.
+ */
+ public MessageMapType(MessageType keyType, MessageType valType, boolean
linked) {
+ this.keyType = keyType;
+ this.valType = valType;
+ this.linked = linked;
+ }
+
+ /** @return Key type. */
+ public MessageType keyType() {
+ return keyType;
+ }
+
+ /** @return Value type. */
+ public MessageType valueType() {
+ return valType;
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageCollectionItemType type() {
+ return MessageCollectionItemType.MAP;
+ }
+
+ /** @return Is linked hash map. */
+ public boolean linked() {
+ return linked;
+ }
+}
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
index eda96716fac..48cf3355ce6 100644
---
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java
@@ -20,9 +20,7 @@ package org.apache.ignite.plugin.extensions.communication;
import java.nio.ByteBuffer;
import java.util.BitSet;
import java.util.Collection;
-import java.util.LinkedHashMap;
import java.util.Map;
-import java.util.Set;
import java.util.UUID;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.CacheObject;
@@ -236,59 +234,41 @@ public interface MessageReader {
/**
* Reads array of objects.
*
- * @param itemType Array component type.
- * @param itemCls Array component class.
+ * @param type Array component type.
* @param <T> Type of the read object.
* @return Array of objects.
*/
- public <T> T[] readObjectArray(MessageCollectionItemType itemType,
Class<T> itemCls);
+ public <T> T[] readObjectArray(MessageArrayType type);
/**
* Reads any collection.
*
- * @param itemType Collection item type.
+ * @param type Collection item type.
* @param <C> Type of the read collection.
* @return Collection.
*/
- public <C extends Collection<?>> C
readCollection(MessageCollectionItemType itemType);
-
- /**
- * Reads any collection and provides it as a set.
- *
- * @param itemType Set item type.
- * @param <S> Type of the read set.
- * @return Set.
- */
- public <S extends Set<?>> S readSet(MessageCollectionItemType itemType);
+ public <C extends Collection<?>> C readCollection(MessageCollectionType
type);
/**
* Reads map.
*
- * @param keyType Map key type.
- * @param valType Map value type.
- * @param linked Whether {@link LinkedHashMap} should be created.
+ * @param type Map type.
* @param <M> Type of the read map.
* @return Map.
*/
- // TODO: IGNITE-26329 — switch to the new readMap method without the
linked flag parameter
- public default <M extends Map<?, ?>> M readMap(MessageCollectionItemType
keyType,
- MessageCollectionItemType valType, boolean linked) {
- return readMap(keyType, valType, linked, false);
+ public default <M extends Map<?, ?>> M readMap(MessageMapType type) {
+ return readMap(type, false);
}
/**
* Reads map.
*
- * @param keyType Map key type.
- * @param valType Map value type.
- * @param linked Whether {@link LinkedHashMap} should be created.
+ * @param type Map type.
* @param compress Whether map should be compressed.
* @param <M> Type of the read map.
* @return Map.
*/
- // TODO: IGNITE-26329 — switch to the new readMap method without the
linked flag parameter
- public <M extends Map<?, ?>> M readMap(MessageCollectionItemType keyType,
- MessageCollectionItemType valType, boolean linked, boolean compress);
+ public <M extends Map<?, ?>> M readMap(MessageMapType type, boolean
compress);
/**
* Tells whether last invocation of any of {@code readXXX(...)}
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageType.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageType.java
new file mode 100644
index 00000000000..c9f241f6e38
--- /dev/null
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageType.java
@@ -0,0 +1,26 @@
+/*
+ * 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.plugin.extensions.communication;
+
+/** */
+public interface MessageType {
+ /**
+ * @return Type.
+ */
+ public MessageCollectionItemType type();
+}
diff --git
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
index e073ca83134..a189a195b8e 100644
---
a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
+++
b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageWriter.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
import java.util.BitSet;
import java.util.Collection;
import java.util.Map;
-import java.util.Set;
import java.util.UUID;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.CacheObject;
@@ -288,60 +287,46 @@ public interface MessageWriter {
* Writes array of objects.
*
* @param arr Array of objects.
- * @param itemType Array component type.
+ * @param type Array component type.
* @param <T> Type of the objects that array contains.
* @return Whether array was fully written.
*/
- public <T> boolean writeObjectArray(T[] arr, MessageCollectionItemType
itemType);
+ public <T> boolean writeObjectArray(T[] arr, MessageArrayType type);
/**
* Writes collection with its elements order.
*
* @param col Collection.
- * @param itemType Collection item type.
+ * @param type Collection item type.
* @param <T> Type of the objects that collection contains.
* @return Whether value was fully written.
*/
- public <T> boolean writeCollection(Collection<T> col,
MessageCollectionItemType itemType);
-
- /**
- * Writes set with its elements order.
- *
- * @param set Set.
- * @param itemType Set item type.
- * @param <T> Type of the objects that set contains.
- * @return Whether value was fully written.
- */
- public <T> boolean writeSet(Set<T> set, MessageCollectionItemType
itemType);
+ public <T> boolean writeCollection(Collection<T> col,
MessageCollectionType type);
/**
* Writes map.
*
* @param map Map.
- * @param keyType Map key type.
- * @param valType Map value type.
+ * @param type Map type.
* @param <K> Initial key types of the map to write.
* @param <V> Initial value types of the map to write.
* @return Whether value was fully written.
*/
- public default <K, V> boolean writeMap(Map<K, V> map,
MessageCollectionItemType keyType,
- MessageCollectionItemType valType) {
- return writeMap(map, keyType, valType, false);
+ public default <K, V> boolean writeMap(Map<K, V> map, MessageMapType type)
{
+ return writeMap(map, type, false);
}
/**
* Writes map.
*
* @param map Map.
- * @param keyType Map key type.
- * @param valType Map value type.
+ * @param type Map type.
* @param compress Whether map should be compressed.
* @param <K> Initial key types of the map to write.
* @param <V> Initial value types of the map to write.
* @return Whether value was fully written.
*/
- public <K, V> boolean writeMap(Map<K, V> map, MessageCollectionItemType
keyType,
- MessageCollectionItemType valType, boolean compress);
+ public <K, V> boolean writeMap(Map<K, V> map, MessageMapType type, boolean
compress);
/**
* @return Whether header of current message is already written.
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/AbstractMessageSerializationTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/AbstractMessageSerializationTest.java
index 2f28cfc4146..1104b04e217 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/AbstractMessageSerializationTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/AbstractMessageSerializationTest.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.managers.communication;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
@@ -29,9 +31,11 @@ import
org.apache.ignite.internal.processors.cache.KeyCacheObject;
import org.apache.ignite.internal.util.GridLongList;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.plugin.extensions.communication.Message;
-import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageArrayType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
import org.apache.ignite.plugin.extensions.communication.MessageFactory;
import
org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider;
+import org.apache.ignite.plugin.extensions.communication.MessageMapType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage;
@@ -291,24 +295,18 @@ public abstract class AbstractMessageSerializationTest {
}
/** {@inheritDoc} */
- @Override public <T> boolean writeObjectArray(T[] arr,
MessageCollectionItemType itemType) {
+ @Override public <T> boolean writeObjectArray(T[] arr,
MessageArrayType type) {
return writeField(Object[].class);
}
/** {@inheritDoc} */
- @Override public <T> boolean writeCollection(Collection<T> col,
MessageCollectionItemType itemType) {
- return writeField(Collection.class);
+ @Override public <T> boolean writeCollection(Collection<T> col,
MessageCollectionType type) {
+ return writeField(type.set() ? Set.class : Collection.class);
}
/** {@inheritDoc} */
- @Override public <T> boolean writeSet(Set<T> set,
MessageCollectionItemType itemType) {
- return writeField(Set.class);
- }
-
- /** {@inheritDoc} */
- @Override public <K, V> boolean writeMap(Map<K, V> map,
MessageCollectionItemType keyType,
- MessageCollectionItemType valType, boolean compress) {
- return writeField(Map.class);
+ @Override public <K, V> boolean writeMap(Map<K, V> map, MessageMapType
type, boolean compress) {
+ return writeField(type.linked() ? LinkedHashMap.class :
HashMap.class);
}
/** {@inheritDoc} */
@@ -552,30 +550,22 @@ public abstract class AbstractMessageSerializationTest {
}
/** {@inheritDoc} */
- @Override public <T> T[] readObjectArray(MessageCollectionItemType
itemType, Class<T> itemCls) {
+ @Override public <T> T[] readObjectArray(MessageArrayType type) {
readField(Object[].class);
return null;
}
/** {@inheritDoc} */
- @Override public <C extends Collection<?>> C
readCollection(MessageCollectionItemType itemType) {
- readField(Collection.class);
+ @Override public <C extends Collection<?>> C
readCollection(MessageCollectionType type) {
+ readField(type.set() ? Set.class : Collection.class);
return null;
}
-
- /** {@inheritDoc} */
- @Override public <S extends Set<?>> S
readSet(MessageCollectionItemType itemType) {
- readField(Set.class);
-
- return null;
- }
-
+
/** {@inheritDoc} */
- @Override public <M extends Map<?, ?>> M
readMap(MessageCollectionItemType keyType,
- MessageCollectionItemType valType, boolean linked, boolean
compress) {
- readField(Map.class);
+ @Override public <M extends Map<?, ?>> M readMap(MessageMapType type,
boolean compress) {
+ readField(type.linked() ? LinkedHashMap.class : HashMap.class);
return null;
}
diff --git
a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
index 3ad36b8f8f3..6acfc4c101c 100644
---
a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
@@ -42,8 +42,10 @@ import org.apache.ignite.plugin.ExtensionRegistry;
import org.apache.ignite.plugin.PluginContext;
import org.apache.ignite.plugin.extensions.communication.Message;
import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
import org.apache.ignite.plugin.extensions.communication.MessageFactory;
import
org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider;
+import org.apache.ignite.plugin.extensions.communication.MessageItemType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -268,7 +270,8 @@ public class GridCacheMessageSelfTest extends
GridCommonAbstractTest {
switch (writer.state()) {
case 3:
- if (!writer.writeCollection(entries,
MessageCollectionItemType.MSG))
+ if (!writer.writeCollection(entries,
+ new MessageCollectionType(new
MessageItemType(MessageCollectionItemType.MSG), false)))
return false;
writer.incrementState();
@@ -287,7 +290,7 @@ public class GridCacheMessageSelfTest extends
GridCommonAbstractTest {
switch (reader.state()) {
case 3:
- entries =
reader.readCollection(MessageCollectionItemType.MSG);
+ entries = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.MSG),
false));
if (!reader.isLastRead())
return false;
diff --git
a/modules/core/src/test/resources/codegen/TestCollectionsMessageSerializer.java
b/modules/core/src/test/resources/codegen/TestCollectionsMessageSerializer.java
index 2f4f365cc83..d27220d6b25 100644
---
a/modules/core/src/test/resources/codegen/TestCollectionsMessageSerializer.java
+++
b/modules/core/src/test/resources/codegen/TestCollectionsMessageSerializer.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal;
import org.apache.ignite.internal.TestCollectionsMessage;
import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
+import org.apache.ignite.plugin.extensions.communication.MessageItemType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageSerializer;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -40,151 +42,151 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
switch (writer.state()) {
case 0:
- if (!writer.writeCollection(msg.booleanArrayList,
MessageCollectionItemType.BOOLEAN_ARR))
+ if (!writer.writeCollection(msg.booleanArrayList, new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.BOOLEAN_ARR), false)))
return false;
writer.incrementState();
case 1:
- if (!writer.writeCollection(msg.byteArrayList,
MessageCollectionItemType.BYTE_ARR))
+ if (!writer.writeCollection(msg.byteArrayList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BYTE_ARR),
false)))
return false;
writer.incrementState();
case 2:
- if (!writer.writeCollection(msg.shortArrayList,
MessageCollectionItemType.SHORT_ARR))
+ if (!writer.writeCollection(msg.shortArrayList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.SHORT_ARR),
false)))
return false;
writer.incrementState();
case 3:
- if (!writer.writeCollection(msg.intArrayList,
MessageCollectionItemType.INT_ARR))
+ if (!writer.writeCollection(msg.intArrayList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.INT_ARR),
false)))
return false;
writer.incrementState();
case 4:
- if (!writer.writeCollection(msg.longArrayList,
MessageCollectionItemType.LONG_ARR))
+ if (!writer.writeCollection(msg.longArrayList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.LONG_ARR),
false)))
return false;
writer.incrementState();
case 5:
- if (!writer.writeCollection(msg.charArrayList,
MessageCollectionItemType.CHAR_ARR))
+ if (!writer.writeCollection(msg.charArrayList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.CHAR_ARR),
false)))
return false;
writer.incrementState();
case 6:
- if (!writer.writeCollection(msg.floatArrayList,
MessageCollectionItemType.FLOAT_ARR))
+ if (!writer.writeCollection(msg.floatArrayList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.FLOAT_ARR),
false)))
return false;
writer.incrementState();
case 7:
- if (!writer.writeCollection(msg.doubleArrayList,
MessageCollectionItemType.DOUBLE_ARR))
+ if (!writer.writeCollection(msg.doubleArrayList, new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.DOUBLE_ARR), false)))
return false;
writer.incrementState();
case 8:
- if (!writer.writeCollection(msg.stringList,
MessageCollectionItemType.STRING))
+ if (!writer.writeCollection(msg.stringList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.STRING),
false)))
return false;
writer.incrementState();
case 9:
- if (!writer.writeCollection(msg.uuidList,
MessageCollectionItemType.UUID))
+ if (!writer.writeCollection(msg.uuidList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.UUID),
false)))
return false;
writer.incrementState();
case 10:
- if (!writer.writeCollection(msg.bitSetList,
MessageCollectionItemType.BIT_SET))
+ if (!writer.writeCollection(msg.bitSetList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BIT_SET),
false)))
return false;
writer.incrementState();
case 11:
- if (!writer.writeCollection(msg.igniteUuidList,
MessageCollectionItemType.IGNITE_UUID))
+ if (!writer.writeCollection(msg.igniteUuidList, new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.IGNITE_UUID), false)))
return false;
writer.incrementState();
case 12:
- if (!writer.writeCollection(msg.affTopVersionList,
MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION))
+ if (!writer.writeCollection(msg.affTopVersionList, new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION), false)))
return false;
writer.incrementState();
case 13:
- if (!writer.writeCollection(msg.boxedBooleanList,
MessageCollectionItemType.BOOLEAN))
+ if (!writer.writeCollection(msg.boxedBooleanList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BOOLEAN),
false)))
return false;
writer.incrementState();
case 14:
- if (!writer.writeCollection(msg.boxedByteList,
MessageCollectionItemType.BYTE))
+ if (!writer.writeCollection(msg.boxedByteList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BYTE),
false)))
return false;
writer.incrementState();
case 15:
- if (!writer.writeCollection(msg.boxedShortList,
MessageCollectionItemType.SHORT))
+ if (!writer.writeCollection(msg.boxedShortList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.SHORT),
false)))
return false;
writer.incrementState();
case 16:
- if (!writer.writeCollection(msg.boxedIntList,
MessageCollectionItemType.INT))
+ if (!writer.writeCollection(msg.boxedIntList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.INT),
false)))
return false;
writer.incrementState();
case 17:
- if (!writer.writeCollection(msg.boxedLongList,
MessageCollectionItemType.LONG))
+ if (!writer.writeCollection(msg.boxedLongList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.LONG),
false)))
return false;
writer.incrementState();
case 18:
- if (!writer.writeCollection(msg.boxedCharList,
MessageCollectionItemType.CHAR))
+ if (!writer.writeCollection(msg.boxedCharList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.CHAR),
false)))
return false;
writer.incrementState();
case 19:
- if (!writer.writeCollection(msg.boxedFloatList,
MessageCollectionItemType.FLOAT))
+ if (!writer.writeCollection(msg.boxedFloatList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.FLOAT),
false)))
return false;
writer.incrementState();
case 20:
- if (!writer.writeCollection(msg.boxedDoubleList,
MessageCollectionItemType.DOUBLE))
+ if (!writer.writeCollection(msg.boxedDoubleList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.DOUBLE),
false)))
return false;
writer.incrementState();
case 21:
- if (!writer.writeCollection(msg.messageList,
MessageCollectionItemType.MSG))
+ if (!writer.writeCollection(msg.messageList, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.MSG),
false)))
return false;
writer.incrementState();
case 22:
- if (!writer.writeCollection(msg.gridLongListList,
MessageCollectionItemType.GRID_LONG_LIST))
+ if (!writer.writeCollection(msg.gridLongListList, new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.GRID_LONG_LIST), false)))
return false;
writer.incrementState();
case 23:
- if (!writer.writeSet(msg.boxedIntegerSet,
MessageCollectionItemType.INT))
+ if (!writer.writeCollection(msg.boxedIntegerSet, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.INT),
true)))
return false;
writer.incrementState();
case 24:
- if (!writer.writeSet(msg.bitSetSet,
MessageCollectionItemType.BIT_SET))
+ if (!writer.writeCollection(msg.bitSetSet, new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BIT_SET),
true)))
return false;
writer.incrementState();
@@ -197,7 +199,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
@Override public boolean readFrom(TestCollectionsMessage msg,
MessageReader reader) {
switch (reader.state()) {
case 0:
- msg.booleanArrayList =
reader.readCollection(MessageCollectionItemType.BOOLEAN_ARR);
+ msg.booleanArrayList = reader.readCollection(new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.BOOLEAN_ARR), false));
if (!reader.isLastRead())
return false;
@@ -205,7 +207,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 1:
- msg.byteArrayList =
reader.readCollection(MessageCollectionItemType.BYTE_ARR);
+ msg.byteArrayList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BYTE_ARR),
false));
if (!reader.isLastRead())
return false;
@@ -213,7 +215,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 2:
- msg.shortArrayList =
reader.readCollection(MessageCollectionItemType.SHORT_ARR);
+ msg.shortArrayList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.SHORT_ARR),
false));
if (!reader.isLastRead())
return false;
@@ -221,7 +223,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 3:
- msg.intArrayList =
reader.readCollection(MessageCollectionItemType.INT_ARR);
+ msg.intArrayList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.INT_ARR),
false));
if (!reader.isLastRead())
return false;
@@ -229,7 +231,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 4:
- msg.longArrayList =
reader.readCollection(MessageCollectionItemType.LONG_ARR);
+ msg.longArrayList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.LONG_ARR),
false));
if (!reader.isLastRead())
return false;
@@ -237,7 +239,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 5:
- msg.charArrayList =
reader.readCollection(MessageCollectionItemType.CHAR_ARR);
+ msg.charArrayList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.CHAR_ARR),
false));
if (!reader.isLastRead())
return false;
@@ -245,7 +247,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 6:
- msg.floatArrayList =
reader.readCollection(MessageCollectionItemType.FLOAT_ARR);
+ msg.floatArrayList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.FLOAT_ARR),
false));
if (!reader.isLastRead())
return false;
@@ -253,7 +255,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 7:
- msg.doubleArrayList =
reader.readCollection(MessageCollectionItemType.DOUBLE_ARR);
+ msg.doubleArrayList = reader.readCollection(new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.DOUBLE_ARR), false));
if (!reader.isLastRead())
return false;
@@ -261,7 +263,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 8:
- msg.stringList =
reader.readCollection(MessageCollectionItemType.STRING);
+ msg.stringList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.STRING),
false));
if (!reader.isLastRead())
return false;
@@ -269,7 +271,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 9:
- msg.uuidList =
reader.readCollection(MessageCollectionItemType.UUID);
+ msg.uuidList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.UUID),
false));
if (!reader.isLastRead())
return false;
@@ -277,7 +279,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 10:
- msg.bitSetList =
reader.readCollection(MessageCollectionItemType.BIT_SET);
+ msg.bitSetList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BIT_SET),
false));
if (!reader.isLastRead())
return false;
@@ -285,7 +287,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 11:
- msg.igniteUuidList =
reader.readCollection(MessageCollectionItemType.IGNITE_UUID);
+ msg.igniteUuidList = reader.readCollection(new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.IGNITE_UUID), false));
if (!reader.isLastRead())
return false;
@@ -293,7 +295,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 12:
- msg.affTopVersionList =
reader.readCollection(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION);
+ msg.affTopVersionList = reader.readCollection(new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION), false));
if (!reader.isLastRead())
return false;
@@ -301,7 +303,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 13:
- msg.boxedBooleanList =
reader.readCollection(MessageCollectionItemType.BOOLEAN);
+ msg.boxedBooleanList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BOOLEAN),
false));
if (!reader.isLastRead())
return false;
@@ -309,7 +311,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 14:
- msg.boxedByteList =
reader.readCollection(MessageCollectionItemType.BYTE);
+ msg.boxedByteList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BYTE),
false));
if (!reader.isLastRead())
return false;
@@ -317,7 +319,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 15:
- msg.boxedShortList =
reader.readCollection(MessageCollectionItemType.SHORT);
+ msg.boxedShortList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.SHORT),
false));
if (!reader.isLastRead())
return false;
@@ -325,7 +327,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 16:
- msg.boxedIntList =
reader.readCollection(MessageCollectionItemType.INT);
+ msg.boxedIntList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.INT),
false));
if (!reader.isLastRead())
return false;
@@ -333,7 +335,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 17:
- msg.boxedLongList =
reader.readCollection(MessageCollectionItemType.LONG);
+ msg.boxedLongList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.LONG),
false));
if (!reader.isLastRead())
return false;
@@ -341,7 +343,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 18:
- msg.boxedCharList =
reader.readCollection(MessageCollectionItemType.CHAR);
+ msg.boxedCharList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.CHAR),
false));
if (!reader.isLastRead())
return false;
@@ -349,7 +351,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 19:
- msg.boxedFloatList =
reader.readCollection(MessageCollectionItemType.FLOAT);
+ msg.boxedFloatList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.FLOAT),
false));
if (!reader.isLastRead())
return false;
@@ -357,7 +359,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 20:
- msg.boxedDoubleList =
reader.readCollection(MessageCollectionItemType.DOUBLE);
+ msg.boxedDoubleList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.DOUBLE),
false));
if (!reader.isLastRead())
return false;
@@ -365,7 +367,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 21:
- msg.messageList =
reader.readCollection(MessageCollectionItemType.MSG);
+ msg.messageList = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.MSG),
false));
if (!reader.isLastRead())
return false;
@@ -373,7 +375,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 22:
- msg.gridLongListList =
reader.readCollection(MessageCollectionItemType.GRID_LONG_LIST);
+ msg.gridLongListList = reader.readCollection(new
MessageCollectionType(new
MessageItemType(MessageCollectionItemType.GRID_LONG_LIST), false));
if (!reader.isLastRead())
return false;
@@ -381,7 +383,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 23:
- msg.boxedIntegerSet =
reader.readSet(MessageCollectionItemType.INT);
+ msg.boxedIntegerSet = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.INT),
true));
if (!reader.isLastRead())
return false;
@@ -389,7 +391,7 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
reader.incrementState();
case 24:
- msg.bitSetSet =
reader.readSet(MessageCollectionItemType.BIT_SET);
+ msg.bitSetSet = reader.readCollection(new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.BIT_SET),
true));
if (!reader.isLastRead())
return false;
@@ -399,4 +401,4 @@ public class TestCollectionsMessageSerializer implements
MessageSerializer<TestC
return true;
}
-}
+}
\ No newline at end of file
diff --git a/modules/core/src/test/resources/codegen/TestMapMessage.java
b/modules/core/src/test/resources/codegen/TestMapMessage.java
index 8fc709ba0c9..71a20da9237 100644
--- a/modules/core/src/test/resources/codegen/TestMapMessage.java
+++ b/modules/core/src/test/resources/codegen/TestMapMessage.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal;
import java.util.BitSet;
+import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -99,6 +100,9 @@ public class TestMapMessage implements Message {
@Order(23)
Map<GridLongList, Integer> gridLongListIntegerMap;
+ @Order(24)
+ Map<GridLongList, Map<UUID, List<Double>>> gridlistDoubleMapUuidMap;
+
public short directType() {
return 0;
}
diff --git
a/modules/core/src/test/resources/codegen/TestMapMessageSerializer.java
b/modules/core/src/test/resources/codegen/TestMapMessageSerializer.java
index 42d0773b7e2..afa07ce5787 100644
--- a/modules/core/src/test/resources/codegen/TestMapMessageSerializer.java
+++ b/modules/core/src/test/resources/codegen/TestMapMessageSerializer.java
@@ -19,6 +19,9 @@ package org.apache.ignite.internal;
import org.apache.ignite.internal.TestMapMessage;
import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionType;
+import org.apache.ignite.plugin.extensions.communication.MessageItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageMapType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageSerializer;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -40,145 +43,151 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
switch (writer.state()) {
case 0:
- if (!writer.writeMap(msg.booleanArrayBoxedLongMap,
MessageCollectionItemType.BOOLEAN_ARR, MessageCollectionItemType.LONG))
+ if (!writer.writeMap(msg.booleanArrayBoxedLongMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.BOOLEAN_ARR), new
MessageItemType(MessageCollectionItemType.LONG), false)))
return false;
writer.incrementState();
case 1:
- if (!writer.writeMap(msg.byteArrayBooleanArrayMap,
MessageCollectionItemType.BYTE_ARR, MessageCollectionItemType.BOOLEAN_ARR))
+ if (!writer.writeMap(msg.byteArrayBooleanArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.BYTE_ARR), new
MessageItemType(MessageCollectionItemType.BOOLEAN_ARR), false)))
return false;
writer.incrementState();
case 2:
- if (!writer.writeMap(msg.shortArrayByteArrayMap,
MessageCollectionItemType.SHORT_ARR, MessageCollectionItemType.BYTE_ARR))
+ if (!writer.writeMap(msg.shortArrayByteArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.SHORT_ARR), new
MessageItemType(MessageCollectionItemType.BYTE_ARR), false)))
return false;
writer.incrementState();
case 3:
- if (!writer.writeMap(msg.intArrayShortArrayMap,
MessageCollectionItemType.INT_ARR, MessageCollectionItemType.SHORT_ARR))
+ if (!writer.writeMap(msg.intArrayShortArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.INT_ARR), new
MessageItemType(MessageCollectionItemType.SHORT_ARR), false)))
return false;
writer.incrementState();
case 4:
- if (!writer.writeMap(msg.longArrayIntArrayMap,
MessageCollectionItemType.LONG_ARR, MessageCollectionItemType.INT_ARR))
+ if (!writer.writeMap(msg.longArrayIntArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.LONG_ARR), new
MessageItemType(MessageCollectionItemType.INT_ARR), false)))
return false;
writer.incrementState();
case 5:
- if (!writer.writeMap(msg.charArrayLongArrayMap,
MessageCollectionItemType.CHAR_ARR, MessageCollectionItemType.LONG_ARR))
+ if (!writer.writeMap(msg.charArrayLongArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.CHAR_ARR), new
MessageItemType(MessageCollectionItemType.LONG_ARR), false)))
return false;
writer.incrementState();
case 6:
- if (!writer.writeMap(msg.floatArrayCharArrayMap,
MessageCollectionItemType.FLOAT_ARR, MessageCollectionItemType.CHAR_ARR))
+ if (!writer.writeMap(msg.floatArrayCharArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.FLOAT_ARR), new
MessageItemType(MessageCollectionItemType.CHAR_ARR), false)))
return false;
writer.incrementState();
case 7:
- if (!writer.writeMap(msg.doubleArrayFloatArrayMap,
MessageCollectionItemType.DOUBLE_ARR, MessageCollectionItemType.FLOAT_ARR))
+ if (!writer.writeMap(msg.doubleArrayFloatArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.DOUBLE_ARR), new
MessageItemType(MessageCollectionItemType.FLOAT_ARR), false)))
return false;
writer.incrementState();
case 8:
- if (!writer.writeMap(msg.stringDoubleArrayMap,
MessageCollectionItemType.STRING, MessageCollectionItemType.DOUBLE_ARR))
+ if (!writer.writeMap(msg.stringDoubleArrayMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.STRING), new
MessageItemType(MessageCollectionItemType.DOUBLE_ARR), false)))
return false;
writer.incrementState();
case 9:
- if (!writer.writeMap(msg.uuidStringMap,
MessageCollectionItemType.UUID, MessageCollectionItemType.STRING))
+ if (!writer.writeMap(msg.uuidStringMap, new MessageMapType(new
MessageItemType(MessageCollectionItemType.UUID), new
MessageItemType(MessageCollectionItemType.STRING), false)))
return false;
writer.incrementState();
case 10:
- if (!writer.writeMap(msg.bitSetUuidMap,
MessageCollectionItemType.BIT_SET, MessageCollectionItemType.UUID))
+ if (!writer.writeMap(msg.bitSetUuidMap, new MessageMapType(new
MessageItemType(MessageCollectionItemType.BIT_SET), new
MessageItemType(MessageCollectionItemType.UUID), false)))
return false;
writer.incrementState();
case 11:
- if (!writer.writeMap(msg.igniteUuidBitSetMap,
MessageCollectionItemType.IGNITE_UUID, MessageCollectionItemType.BIT_SET))
+ if (!writer.writeMap(msg.igniteUuidBitSetMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.IGNITE_UUID), new
MessageItemType(MessageCollectionItemType.BIT_SET), false)))
return false;
writer.incrementState();
case 12:
- if (!writer.writeMap(msg.affTopVersionIgniteUuidMap,
MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION,
MessageCollectionItemType.IGNITE_UUID))
+ if (!writer.writeMap(msg.affTopVersionIgniteUuidMap, new
MessageMapType(new
MessageItemType(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION), new
MessageItemType(MessageCollectionItemType.IGNITE_UUID), false)))
return false;
writer.incrementState();
case 13:
- if (!writer.writeMap(msg.boxedBooleanAffTopVersionMap,
MessageCollectionItemType.BOOLEAN,
MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION))
+ if (!writer.writeMap(msg.boxedBooleanAffTopVersionMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.BOOLEAN), new
MessageItemType(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION), false)))
return false;
writer.incrementState();
case 14:
- if (!writer.writeMap(msg.boxedByteBoxedBooleanMap,
MessageCollectionItemType.BYTE, MessageCollectionItemType.BOOLEAN))
+ if (!writer.writeMap(msg.boxedByteBoxedBooleanMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.BYTE), new
MessageItemType(MessageCollectionItemType.BOOLEAN), false)))
return false;
writer.incrementState();
case 15:
- if (!writer.writeMap(msg.boxedShortBoxedByteMap,
MessageCollectionItemType.SHORT, MessageCollectionItemType.BYTE))
+ if (!writer.writeMap(msg.boxedShortBoxedByteMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.SHORT), new
MessageItemType(MessageCollectionItemType.BYTE), false)))
return false;
writer.incrementState();
case 16:
- if (!writer.writeMap(msg.boxedIntBoxedShortMap,
MessageCollectionItemType.INT, MessageCollectionItemType.SHORT))
+ if (!writer.writeMap(msg.boxedIntBoxedShortMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.INT), new
MessageItemType(MessageCollectionItemType.SHORT), false)))
return false;
writer.incrementState();
case 17:
- if (!writer.writeMap(msg.boxedLongBoxedIntMap,
MessageCollectionItemType.LONG, MessageCollectionItemType.INT))
+ if (!writer.writeMap(msg.boxedLongBoxedIntMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.LONG), new
MessageItemType(MessageCollectionItemType.INT), false)))
return false;
writer.incrementState();
case 18:
- if (!writer.writeMap(msg.boxedCharBoxedLongMap,
MessageCollectionItemType.CHAR, MessageCollectionItemType.LONG))
+ if (!writer.writeMap(msg.boxedCharBoxedLongMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.CHAR), new
MessageItemType(MessageCollectionItemType.LONG), false)))
return false;
writer.incrementState();
case 19:
- if (!writer.writeMap(msg.boxedFloatBoxedCharMap,
MessageCollectionItemType.FLOAT, MessageCollectionItemType.CHAR))
+ if (!writer.writeMap(msg.boxedFloatBoxedCharMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.FLOAT), new
MessageItemType(MessageCollectionItemType.CHAR), false)))
return false;
writer.incrementState();
case 20:
- if (!writer.writeMap(msg.boxedDoubleBoxedFloatMap,
MessageCollectionItemType.DOUBLE, MessageCollectionItemType.FLOAT))
+ if (!writer.writeMap(msg.boxedDoubleBoxedFloatMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.DOUBLE), new
MessageItemType(MessageCollectionItemType.FLOAT), false)))
return false;
writer.incrementState();
case 21:
- if (!writer.writeMap(msg.messageBoxedDoubleMap,
MessageCollectionItemType.MSG, MessageCollectionItemType.DOUBLE))
+ if (!writer.writeMap(msg.messageBoxedDoubleMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.MSG), new
MessageItemType(MessageCollectionItemType.DOUBLE), false)))
return false;
writer.incrementState();
case 22:
- if (!writer.writeMap(msg.integerGridLongListMap,
MessageCollectionItemType.INT, MessageCollectionItemType.GRID_LONG_LIST))
+ if (!writer.writeMap(msg.integerGridLongListMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.INT), new
MessageItemType(MessageCollectionItemType.GRID_LONG_LIST), false)))
return false;
writer.incrementState();
case 23:
- if (!writer.writeMap(msg.gridLongListIntegerMap,
MessageCollectionItemType.GRID_LONG_LIST, MessageCollectionItemType.INT))
+ if (!writer.writeMap(msg.gridLongListIntegerMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.GRID_LONG_LIST),
new MessageItemType(MessageCollectionItemType.INT), false)))
+ return false;
+
+ writer.incrementState();
+
+ case 24:
+ if (!writer.writeMap(msg.gridlistDoubleMapUuidMap, new
MessageMapType(new MessageItemType(MessageCollectionItemType.GRID_LONG_LIST),
new MessageMapType(new MessageItemType(MessageCollectionItemType.UUID), new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.DOUBLE),
false), false), false)))
return false;
writer.incrementState();
@@ -191,7 +200,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
@Override public boolean readFrom(TestMapMessage msg, MessageReader
reader) {
switch (reader.state()) {
case 0:
- msg.booleanArrayBoxedLongMap =
reader.readMap(MessageCollectionItemType.BOOLEAN_ARR,
MessageCollectionItemType.LONG, false);
+ msg.booleanArrayBoxedLongMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.BOOLEAN_ARR), new
MessageItemType(MessageCollectionItemType.LONG), false));
if (!reader.isLastRead())
return false;
@@ -199,7 +208,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 1:
- msg.byteArrayBooleanArrayMap =
reader.readMap(MessageCollectionItemType.BYTE_ARR,
MessageCollectionItemType.BOOLEAN_ARR, false);
+ msg.byteArrayBooleanArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.BYTE_ARR), new
MessageItemType(MessageCollectionItemType.BOOLEAN_ARR), false));
if (!reader.isLastRead())
return false;
@@ -207,7 +216,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 2:
- msg.shortArrayByteArrayMap =
reader.readMap(MessageCollectionItemType.SHORT_ARR,
MessageCollectionItemType.BYTE_ARR, false);
+ msg.shortArrayByteArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.SHORT_ARR), new
MessageItemType(MessageCollectionItemType.BYTE_ARR), false));
if (!reader.isLastRead())
return false;
@@ -215,7 +224,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 3:
- msg.intArrayShortArrayMap =
reader.readMap(MessageCollectionItemType.INT_ARR,
MessageCollectionItemType.SHORT_ARR, false);
+ msg.intArrayShortArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.INT_ARR), new
MessageItemType(MessageCollectionItemType.SHORT_ARR), false));
if (!reader.isLastRead())
return false;
@@ -223,7 +232,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 4:
- msg.longArrayIntArrayMap =
reader.readMap(MessageCollectionItemType.LONG_ARR,
MessageCollectionItemType.INT_ARR, false);
+ msg.longArrayIntArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.LONG_ARR), new
MessageItemType(MessageCollectionItemType.INT_ARR), false));
if (!reader.isLastRead())
return false;
@@ -231,7 +240,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 5:
- msg.charArrayLongArrayMap =
reader.readMap(MessageCollectionItemType.CHAR_ARR,
MessageCollectionItemType.LONG_ARR, false);
+ msg.charArrayLongArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.CHAR_ARR), new
MessageItemType(MessageCollectionItemType.LONG_ARR), false));
if (!reader.isLastRead())
return false;
@@ -239,7 +248,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 6:
- msg.floatArrayCharArrayMap =
reader.readMap(MessageCollectionItemType.FLOAT_ARR,
MessageCollectionItemType.CHAR_ARR, false);
+ msg.floatArrayCharArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.FLOAT_ARR), new
MessageItemType(MessageCollectionItemType.CHAR_ARR), false));
if (!reader.isLastRead())
return false;
@@ -247,7 +256,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 7:
- msg.doubleArrayFloatArrayMap =
reader.readMap(MessageCollectionItemType.DOUBLE_ARR,
MessageCollectionItemType.FLOAT_ARR, false);
+ msg.doubleArrayFloatArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.DOUBLE_ARR), new
MessageItemType(MessageCollectionItemType.FLOAT_ARR), false));
if (!reader.isLastRead())
return false;
@@ -255,7 +264,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 8:
- msg.stringDoubleArrayMap =
reader.readMap(MessageCollectionItemType.STRING,
MessageCollectionItemType.DOUBLE_ARR, false);
+ msg.stringDoubleArrayMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.STRING), new
MessageItemType(MessageCollectionItemType.DOUBLE_ARR), false));
if (!reader.isLastRead())
return false;
@@ -263,7 +272,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 9:
- msg.uuidStringMap =
reader.readMap(MessageCollectionItemType.UUID,
MessageCollectionItemType.STRING, false);
+ msg.uuidStringMap = reader.readMap(new MessageMapType(new
MessageItemType(MessageCollectionItemType.UUID), new
MessageItemType(MessageCollectionItemType.STRING), false));
if (!reader.isLastRead())
return false;
@@ -271,7 +280,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 10:
- msg.bitSetUuidMap =
reader.readMap(MessageCollectionItemType.BIT_SET,
MessageCollectionItemType.UUID, false);
+ msg.bitSetUuidMap = reader.readMap(new MessageMapType(new
MessageItemType(MessageCollectionItemType.BIT_SET), new
MessageItemType(MessageCollectionItemType.UUID), false));
if (!reader.isLastRead())
return false;
@@ -279,7 +288,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 11:
- msg.igniteUuidBitSetMap =
reader.readMap(MessageCollectionItemType.IGNITE_UUID,
MessageCollectionItemType.BIT_SET, false);
+ msg.igniteUuidBitSetMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.IGNITE_UUID), new
MessageItemType(MessageCollectionItemType.BIT_SET), false));
if (!reader.isLastRead())
return false;
@@ -287,7 +296,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 12:
- msg.affTopVersionIgniteUuidMap =
reader.readMap(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION,
MessageCollectionItemType.IGNITE_UUID, false);
+ msg.affTopVersionIgniteUuidMap = reader.readMap(new
MessageMapType(new
MessageItemType(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION), new
MessageItemType(MessageCollectionItemType.IGNITE_UUID), false));
if (!reader.isLastRead())
return false;
@@ -295,7 +304,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 13:
- msg.boxedBooleanAffTopVersionMap =
reader.readMap(MessageCollectionItemType.BOOLEAN,
MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION, false);
+ msg.boxedBooleanAffTopVersionMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.BOOLEAN), new
MessageItemType(MessageCollectionItemType.AFFINITY_TOPOLOGY_VERSION), false));
if (!reader.isLastRead())
return false;
@@ -303,7 +312,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 14:
- msg.boxedByteBoxedBooleanMap =
reader.readMap(MessageCollectionItemType.BYTE,
MessageCollectionItemType.BOOLEAN, false);
+ msg.boxedByteBoxedBooleanMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.BYTE), new
MessageItemType(MessageCollectionItemType.BOOLEAN), false));
if (!reader.isLastRead())
return false;
@@ -311,7 +320,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 15:
- msg.boxedShortBoxedByteMap =
reader.readMap(MessageCollectionItemType.SHORT, MessageCollectionItemType.BYTE,
false);
+ msg.boxedShortBoxedByteMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.SHORT), new
MessageItemType(MessageCollectionItemType.BYTE), false));
if (!reader.isLastRead())
return false;
@@ -319,7 +328,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 16:
- msg.boxedIntBoxedShortMap =
reader.readMap(MessageCollectionItemType.INT, MessageCollectionItemType.SHORT,
false);
+ msg.boxedIntBoxedShortMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.INT), new
MessageItemType(MessageCollectionItemType.SHORT), false));
if (!reader.isLastRead())
return false;
@@ -327,7 +336,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 17:
- msg.boxedLongBoxedIntMap =
reader.readMap(MessageCollectionItemType.LONG, MessageCollectionItemType.INT,
false);
+ msg.boxedLongBoxedIntMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.LONG), new
MessageItemType(MessageCollectionItemType.INT), false));
if (!reader.isLastRead())
return false;
@@ -335,7 +344,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 18:
- msg.boxedCharBoxedLongMap =
reader.readMap(MessageCollectionItemType.CHAR, MessageCollectionItemType.LONG,
false);
+ msg.boxedCharBoxedLongMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.CHAR), new
MessageItemType(MessageCollectionItemType.LONG), false));
if (!reader.isLastRead())
return false;
@@ -343,7 +352,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 19:
- msg.boxedFloatBoxedCharMap =
reader.readMap(MessageCollectionItemType.FLOAT, MessageCollectionItemType.CHAR,
false);
+ msg.boxedFloatBoxedCharMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.FLOAT), new
MessageItemType(MessageCollectionItemType.CHAR), false));
if (!reader.isLastRead())
return false;
@@ -351,7 +360,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 20:
- msg.boxedDoubleBoxedFloatMap =
reader.readMap(MessageCollectionItemType.DOUBLE,
MessageCollectionItemType.FLOAT, false);
+ msg.boxedDoubleBoxedFloatMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.DOUBLE), new
MessageItemType(MessageCollectionItemType.FLOAT), false));
if (!reader.isLastRead())
return false;
@@ -359,7 +368,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 21:
- msg.messageBoxedDoubleMap =
reader.readMap(MessageCollectionItemType.MSG, MessageCollectionItemType.DOUBLE,
false);
+ msg.messageBoxedDoubleMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.MSG), new
MessageItemType(MessageCollectionItemType.DOUBLE), false));
if (!reader.isLastRead())
return false;
@@ -367,7 +376,7 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 22:
- msg.integerGridLongListMap =
reader.readMap(MessageCollectionItemType.INT,
MessageCollectionItemType.GRID_LONG_LIST, false);
+ msg.integerGridLongListMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.INT), new
MessageItemType(MessageCollectionItemType.GRID_LONG_LIST), false));
if (!reader.isLastRead())
return false;
@@ -375,7 +384,15 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
reader.incrementState();
case 23:
- msg.gridLongListIntegerMap =
reader.readMap(MessageCollectionItemType.GRID_LONG_LIST,
MessageCollectionItemType.INT, false);
+ msg.gridLongListIntegerMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.GRID_LONG_LIST),
new MessageItemType(MessageCollectionItemType.INT), false));
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 24:
+ msg.gridlistDoubleMapUuidMap = reader.readMap(new
MessageMapType(new MessageItemType(MessageCollectionItemType.GRID_LONG_LIST),
new MessageMapType(new MessageItemType(MessageCollectionItemType.UUID), new
MessageCollectionType(new MessageItemType(MessageCollectionItemType.DOUBLE),
false), false), false));
if (!reader.isLastRead())
return false;
@@ -385,4 +402,4 @@ public class TestMapMessageSerializer implements
MessageSerializer<TestMapMessag
return true;
}
-}
+}
\ No newline at end of file
diff --git a/modules/core/src/test/resources/codegen/TestMessageSerializer.java
b/modules/core/src/test/resources/codegen/TestMessageSerializer.java
index f14de0fc03b..5ca314b8ef3 100644
--- a/modules/core/src/test/resources/codegen/TestMessageSerializer.java
+++ b/modules/core/src/test/resources/codegen/TestMessageSerializer.java
@@ -19,7 +19,9 @@ package org.apache.ignite.internal;
import org.apache.ignite.internal.TestMessage;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.plugin.extensions.communication.MessageArrayType;
import
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageItemType;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageSerializer;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -59,13 +61,13 @@ public class TestMessageSerializer implements
MessageSerializer<TestMessage> {
writer.incrementState();
case 3:
- if (!writer.writeObjectArray(msg.strArr,
MessageCollectionItemType.STRING))
+ if (!writer.writeObjectArray(msg.strArr, new
MessageArrayType(new MessageItemType(MessageCollectionItemType.STRING),
String.class)))
return false;
writer.incrementState();
case 4:
- if (!writer.writeObjectArray(msg.intMatrix,
MessageCollectionItemType.INT_ARR))
+ if (!writer.writeObjectArray(msg.intMatrix, new
MessageArrayType(new MessageItemType(MessageCollectionItemType.INT_ARR),
int[].class)))
return false;
writer.incrementState();
@@ -77,7 +79,7 @@ public class TestMessageSerializer implements
MessageSerializer<TestMessage> {
writer.incrementState();
case 6:
- if (!writer.writeObjectArray(msg.verArr,
MessageCollectionItemType.MSG))
+ if (!writer.writeObjectArray(msg.verArr, new
MessageArrayType(new MessageItemType(MessageCollectionItemType.MSG),
GridCacheVersion.class)))
return false;
writer.incrementState();
@@ -162,7 +164,7 @@ public class TestMessageSerializer implements
MessageSerializer<TestMessage> {
reader.incrementState();
case 3:
- msg.strArr =
reader.readObjectArray(MessageCollectionItemType.STRING, String.class);
+ msg.strArr = reader.readObjectArray(new MessageArrayType(new
MessageItemType(MessageCollectionItemType.STRING), String.class));
if (!reader.isLastRead())
return false;
@@ -170,7 +172,7 @@ public class TestMessageSerializer implements
MessageSerializer<TestMessage> {
reader.incrementState();
case 4:
- msg.intMatrix =
reader.readObjectArray(MessageCollectionItemType.INT, int[].class);
+ msg.intMatrix = reader.readObjectArray(new
MessageArrayType(new MessageItemType(MessageCollectionItemType.INT_ARR),
int[].class));
if (!reader.isLastRead())
return false;
@@ -186,7 +188,7 @@ public class TestMessageSerializer implements
MessageSerializer<TestMessage> {
reader.incrementState();
case 6:
- msg.verArr =
reader.readObjectArray(MessageCollectionItemType.MSG, GridCacheVersion.class);
+ msg.verArr = reader.readObjectArray(new MessageArrayType(new
MessageItemType(MessageCollectionItemType.MSG), GridCacheVersion.class));
if (!reader.isLastRead())
return false;