From 131b12a266334f5efb74555793085c3661e0a30f Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Wed, 17 Apr 2019 15:31:19 -0400 Subject: [PATCH 01/49] Add List serializer, desearializer, serde --- .../serialization/ListDeserializer.java | 76 +++++++++++++++++++ .../kafka/common/serialization/ListSerde.java | 54 +++++++++++++ .../common/serialization/ListSerializer.java | 64 ++++++++++++++++ 3 files changed, 194 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java new file mode 100644 index 0000000000000..6a70498f2b7f5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -0,0 +1,76 @@ +/* + * 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.kafka.common.serialization; + +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.serialization.Deserializer; + +public class ListDeserializer implements Deserializer> { + + private final Deserializer deserializer; + private final Comparator comparator; + + public ListDeserializer(Deserializer deserializer, Comparator comparator) { + this.deserializer = deserializer; + this.comparator = comparator; + } + + @Override + public void configure(Map configs, boolean isKey) { + // Do nothing + } + + @Override + public List deserialize(String topic, byte[] data) { + if (data == null || data.length == 0) { + return null; + } + @SuppressWarnings("serial") + List deserializedList = new ArrayList() { + @Override + public void sort(Comparator c) { + super.sort(comparator); + } + }; + DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data)); + try { + final int size = dis.readInt(); + for (int i = 0; i < size; i++) { + byte[] payload = new byte[dis.readInt()]; + dis.read(payload); + deserializedList.add(deserializer.deserialize(topic, payload)); + } + dis.close(); + } catch (IOException e) { + throw new RuntimeException("Unable to deserialize into a List", e); + } + return deserializedList; + } + + @Override + public void close() { + // Do nothing + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java new file mode 100644 index 0000000000000..ea2b2413e5dd3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java @@ -0,0 +1,54 @@ +/* + * 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.kafka.common.serialization; + +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +public class ListSerde implements Serde> { + + private final Serde> inner; + + public ListSerde(Serde serde, Comparator comparator) { + inner = Serdes.serdeFrom(new ListSerializer(serde.serializer()), + new ListDeserializer(serde.deserializer(), comparator)); + } + + @Override + public void configure(Map configs, boolean isKey) { + inner.serializer().configure(configs, isKey); + inner.deserializer().configure(configs, isKey); + } + + @Override + public void close() { + inner.serializer().close(); + inner.deserializer().close(); + } + + @Override + public Serializer> serializer() { + return inner.serializer(); + } + + @Override + public Deserializer> deserializer() { + return inner.deserializer(); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java new file mode 100644 index 0000000000000..4ff83a0d8133c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -0,0 +1,64 @@ +/* + * 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.kafka.common.serialization; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.serialization.Serializer; + +public class ListSerializer implements Serializer> { + + private final Serializer serializer; + + public ListSerializer(Serializer serializer) { + this.serializer = serializer; + } + + @Override + public void configure(Map configs, boolean isKey) { + // Do nothing + } + + @Override + public byte[] serialize(String topic, List data) { + final int size = data.size(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos); + try { + out.writeInt(size); + for (T entry : data) { + final byte[] bytes = serializer.serialize(topic, entry); + out.writeInt(bytes.length); + out.write(bytes); + } + out.close(); + } catch (IOException e) { + throw new RuntimeException("Failed to serialize List", e); + } + return baos.toByteArray(); + } + + @Override + public void close() { + serializer.close(); + } + +} \ No newline at end of file From 146c06a5d55f2abc8f4219ef1a415fa9292779b9 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Wed, 17 Apr 2019 16:03:15 -0400 Subject: [PATCH 02/49] Fix formatting and unused imports --- .../serialization/ListDeserializer.java | 82 +++++++++---------- .../kafka/common/serialization/ListSerde.java | 54 ++++++------ .../common/serialization/ListSerializer.java | 70 ++++++++-------- 3 files changed, 101 insertions(+), 105 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 6a70498f2b7f5..70815f0ff81a6 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -24,53 +24,51 @@ import java.util.List; import java.util.Map; -import org.apache.kafka.common.serialization.Deserializer; - public class ListDeserializer implements Deserializer> { - private final Deserializer deserializer; - private final Comparator comparator; + private final Deserializer deserializer; + private final Comparator comparator; - public ListDeserializer(Deserializer deserializer, Comparator comparator) { - this.deserializer = deserializer; - this.comparator = comparator; - } + public ListDeserializer(Deserializer deserializer, Comparator comparator) { + this.deserializer = deserializer; + this.comparator = comparator; + } - @Override - public void configure(Map configs, boolean isKey) { - // Do nothing - } + @Override + public void configure(Map configs, boolean isKey) { + // Do nothing + } - @Override - public List deserialize(String topic, byte[] data) { - if (data == null || data.length == 0) { - return null; - } - @SuppressWarnings("serial") - List deserializedList = new ArrayList() { - @Override - public void sort(Comparator c) { - super.sort(comparator); - } - }; - DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data)); - try { - final int size = dis.readInt(); - for (int i = 0; i < size; i++) { - byte[] payload = new byte[dis.readInt()]; - dis.read(payload); - deserializedList.add(deserializer.deserialize(topic, payload)); - } - dis.close(); - } catch (IOException e) { - throw new RuntimeException("Unable to deserialize into a List", e); - } - return deserializedList; - } + @Override + public List deserialize(String topic, byte[] data) { + if (data == null || data.length == 0) { + return null; + } + @SuppressWarnings("serial") + List deserializedList = new ArrayList() { + @Override + public void sort(Comparator c) { + super.sort(comparator); + } + }; + DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data)); + try { + final int size = dis.readInt(); + for (int i = 0; i < size; i++) { + byte[] payload = new byte[dis.readInt()]; + dis.read(payload); + deserializedList.add(deserializer.deserialize(topic, payload)); + } + dis.close(); + } catch (IOException e) { + throw new RuntimeException("Unable to deserialize into a List", e); + } + return deserializedList; + } - @Override - public void close() { - // Do nothing - } + @Override + public void close() { + // Do nothing + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java index ea2b2413e5dd3..7e66831785651 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java @@ -21,34 +21,34 @@ import java.util.Map; public class ListSerde implements Serde> { - - private final Serde> inner; + + private final Serde> inner; public ListSerde(Serde serde, Comparator comparator) { - inner = Serdes.serdeFrom(new ListSerializer(serde.serializer()), - new ListDeserializer(serde.deserializer(), comparator)); - } - - @Override - public void configure(Map configs, boolean isKey) { - inner.serializer().configure(configs, isKey); - inner.deserializer().configure(configs, isKey); - } - - @Override - public void close() { - inner.serializer().close(); - inner.deserializer().close(); - } - - @Override - public Serializer> serializer() { - return inner.serializer(); - } - - @Override - public Deserializer> deserializer() { - return inner.deserializer(); - } + inner = Serdes.serdeFrom(new ListSerializer(serde.serializer()), + new ListDeserializer(serde.deserializer(), comparator)); + } + + @Override + public void configure(Map configs, boolean isKey) { + inner.serializer().configure(configs, isKey); + inner.deserializer().configure(configs, isKey); + } + + @Override + public void close() { + inner.serializer().close(); + inner.deserializer().close(); + } + + @Override + public Serializer> serializer() { + return inner.serializer(); + } + + @Override + public Deserializer> deserializer() { + return inner.deserializer(); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 4ff83a0d8133c..0ae0a3a0677de 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -22,43 +22,41 @@ import java.util.List; import java.util.Map; -import org.apache.kafka.common.serialization.Serializer; - public class ListSerializer implements Serializer> { - private final Serializer serializer; - - public ListSerializer(Serializer serializer) { - this.serializer = serializer; - } - - @Override - public void configure(Map configs, boolean isKey) { - // Do nothing - } - - @Override - public byte[] serialize(String topic, List data) { - final int size = data.size(); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final DataOutputStream out = new DataOutputStream(baos); - try { - out.writeInt(size); - for (T entry : data) { - final byte[] bytes = serializer.serialize(topic, entry); - out.writeInt(bytes.length); - out.write(bytes); - } - out.close(); - } catch (IOException e) { - throw new RuntimeException("Failed to serialize List", e); - } - return baos.toByteArray(); - } - - @Override - public void close() { - serializer.close(); - } + private final Serializer serializer; + + public ListSerializer(Serializer serializer) { + this.serializer = serializer; + } + + @Override + public void configure(Map configs, boolean isKey) { + // Do nothing + } + + @Override + public byte[] serialize(String topic, List data) { + final int size = data.size(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos); + try { + out.writeInt(size); + for (T entry : data) { + final byte[] bytes = serializer.serialize(topic, entry); + out.writeInt(bytes.length); + out.write(bytes); + } + out.close(); + } catch (IOException e) { + throw new RuntimeException("Failed to serialize List", e); + } + return baos.toByteArray(); + } + + @Override + public void close() { + serializer.close(); + } } \ No newline at end of file From 6f185680f7cb8dcdb5d82cc4ee7df5ebddcf0e9b Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Mon, 6 May 2019 12:02:27 -0400 Subject: [PATCH 03/49] Delete ListSerde, and use WrapperSerde instead --- .../kafka/common/serialization/ListSerde.java | 54 ------------------- .../kafka/common/serialization/Serdes.java | 8 +++ 2 files changed, 8 insertions(+), 54 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java deleted file mode 100644 index 7e66831785651..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerde.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.kafka.common.serialization; - -import java.util.Comparator; -import java.util.List; -import java.util.Map; - -public class ListSerde implements Serde> { - - private final Serde> inner; - - public ListSerde(Serde serde, Comparator comparator) { - inner = Serdes.serdeFrom(new ListSerializer(serde.serializer()), - new ListDeserializer(serde.deserializer(), comparator)); - } - - @Override - public void configure(Map configs, boolean isKey) { - inner.serializer().configure(configs, isKey); - inner.deserializer().configure(configs, isKey); - } - - @Override - public void close() { - inner.serializer().close(); - inner.deserializer().close(); - } - - @Override - public Serializer> serializer() { - return inner.serializer(); - } - - @Override - public Deserializer> deserializer() { - return inner.deserializer(); - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 347bf8713ece8..175c90e5070b9 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.utils.Bytes; import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; import java.util.Map; import java.util.UUID; @@ -125,6 +127,12 @@ public UUIDSerde() { } } + static public final class ListSerde extends WrapperSerde> { + public ListSerde(Serde serde, Comparator comparator) { + super(new ListSerializer(serde.serializer()), new ListDeserializer<>(serde.deserializer(), comparator)); + } + } + @SuppressWarnings("unchecked") static public Serde serdeFrom(Class type) { if (String.class.isAssignableFrom(type)) { From 0fa849574cde84a4c7d4d3b3b994eb4713a72225 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Mon, 6 May 2019 12:04:51 -0400 Subject: [PATCH 04/49] Prevent possible NPE when data is null or empty --- .../org/apache/kafka/common/serialization/ListSerializer.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 0ae0a3a0677de..ff716a82b3676 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -37,6 +37,9 @@ public void configure(Map configs, boolean isKey) { @Override public byte[] serialize(String topic, List data) { + if (data == null || data.size() == 0) { + return null; + } final int size = data.size(); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream out = new DataOutputStream(baos); From dae39d7b02ca3231b2f714125c427e13b58944d3 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Mon, 6 May 2019 13:41:27 -0400 Subject: [PATCH 05/49] Use try-with-resources to utilize auto stream closing --- .../serialization/ListDeserializer.java | 19 ++++++++----------- .../common/serialization/ListSerializer.java | 6 ++---- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 70815f0ff81a6..f753b2fa808cc 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -44,26 +44,23 @@ public List deserialize(String topic, byte[] data) { if (data == null || data.length == 0) { return null; } - @SuppressWarnings("serial") - List deserializedList = new ArrayList() { - @Override - public void sort(Comparator c) { - super.sort(comparator); - } - }; - DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data)); - try { + try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { + List deserializedList = new ArrayList() { + @Override + public void sort(Comparator c) { + super.sort(comparator); + } + }; final int size = dis.readInt(); for (int i = 0; i < size; i++) { byte[] payload = new byte[dis.readInt()]; dis.read(payload); deserializedList.add(deserializer.deserialize(topic, payload)); } - dis.close(); + return deserializedList; } catch (IOException e) { throw new RuntimeException("Unable to deserialize into a List", e); } - return deserializedList; } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index ff716a82b3676..957981ba6e6fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -42,19 +42,17 @@ public byte[] serialize(String topic, List data) { } final int size = data.size(); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final DataOutputStream out = new DataOutputStream(baos); - try { + try (final DataOutputStream out = new DataOutputStream(baos)) { out.writeInt(size); for (T entry : data) { final byte[] bytes = serializer.serialize(topic, entry); out.writeInt(bytes.length); out.write(bytes); } - out.close(); + return baos.toByteArray(); } catch (IOException e) { throw new RuntimeException("Failed to serialize List", e); } - return baos.toByteArray(); } @Override From 36d206d148162da7364d9c8b7d6cb43440bbc15d Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Mon, 6 May 2019 21:54:10 -0400 Subject: [PATCH 06/49] Remove comparator parameter from ListDeserializer constructor --- .../kafka/common/serialization/ListDeserializer.java | 12 ++---------- .../apache/kafka/common/serialization/Serdes.java | 5 ++--- 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index f753b2fa808cc..7a8b89b2868d4 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -20,18 +20,15 @@ import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; import java.util.Map; public class ListDeserializer implements Deserializer> { private final Deserializer deserializer; - private final Comparator comparator; - public ListDeserializer(Deserializer deserializer, Comparator comparator) { + public ListDeserializer(Deserializer deserializer) { this.deserializer = deserializer; - this.comparator = comparator; } @Override @@ -45,12 +42,7 @@ public List deserialize(String topic, byte[] data) { return null; } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { - List deserializedList = new ArrayList() { - @Override - public void sort(Comparator c) { - super.sort(comparator); - } - }; + List deserializedList = new ArrayList<>(); final int size = dis.readInt(); for (int i = 0; i < size; i++) { byte[] payload = new byte[dis.readInt()]; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 175c90e5070b9..0e5ea753a4077 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.Bytes; import java.nio.ByteBuffer; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.UUID; @@ -128,8 +127,8 @@ public UUIDSerde() { } static public final class ListSerde extends WrapperSerde> { - public ListSerde(Serde serde, Comparator comparator) { - super(new ListSerializer(serde.serializer()), new ListDeserializer<>(serde.deserializer(), comparator)); + public ListSerde(Serde serde) { + super(new ListSerializer(serde.serializer()), new ListDeserializer<>(serde.deserializer())); } } From c93d91e419ecf183527b1cec70a9b571c2e92e63 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Mon, 13 May 2019 19:11:33 -0400 Subject: [PATCH 07/49] Implement a test case for ListSerde Since it does not fall into regular serde testing paradigm, it requires a separate test case. --- .../kafka/common/serialization/SerializationTest.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 0446bafc2fbe4..9c86ac10fb1ac 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -107,6 +107,15 @@ public void stringSerdeShouldSupportDifferentEncodings() { } @Test + public void listSerdeShouldShouldRoundtripInput() { + List testData = Arrays.asList(1, 2, 3); + Serde> listSerde = Serdes.ListSerde(Serdes.Integer()); + assertEquals("Should get the original " + List.class + + " after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test(expected = SerializationException.class) public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() { try (Serde serde = Serdes.Float()) { assertThrows(SerializationException.class, () -> serde.deserializer().deserialize(topic, new byte[0])); From aaf22ce275bba8f198e0c361803500faae5ca37a Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Fri, 24 May 2019 13:53:40 -0400 Subject: [PATCH 08/49] Close de/serializers and propagate parameters in configure methods --- .../apache/kafka/common/serialization/ListDeserializer.java | 4 ++-- .../org/apache/kafka/common/serialization/ListSerializer.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 7a8b89b2868d4..a10f1dce2e91e 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -33,7 +33,7 @@ public ListDeserializer(Deserializer deserializer) { @Override public void configure(Map configs, boolean isKey) { - // Do nothing + deserializer.configure(configs, isKey); } @Override @@ -57,7 +57,7 @@ public List deserialize(String topic, byte[] data) { @Override public void close() { - // Do nothing + deserializer.close(); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 957981ba6e6fe..2cdfeeb1caeca 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -32,7 +32,7 @@ public ListSerializer(Serializer serializer) { @Override public void configure(Map configs, boolean isKey) { - // Do nothing + serializer.configure(configs, isKey); } @Override From b0979968722669af3d34674664860583ebee70a1 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Fri, 24 May 2019 13:54:13 -0400 Subject: [PATCH 09/49] Initialize ArrayList with size for better performance --- .../org/apache/kafka/common/serialization/ListDeserializer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index a10f1dce2e91e..dc9e5279ce6e2 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -42,8 +42,8 @@ public List deserialize(String topic, byte[] data) { return null; } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { - List deserializedList = new ArrayList<>(); final int size = dis.readInt(); + List deserializedList = new ArrayList<>(size); for (int i = 0; i < size; i++) { byte[] payload = new byte[dis.readInt()]; dis.read(payload); From 89b22b5cd9776d62cf4ac5afb2786ad4244d4eae Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Mon, 10 Jun 2019 11:39:48 -0400 Subject: [PATCH 10/49] Add another test case for ListSerde After a roundtrip on an empty list, should return null --- .../kafka/common/serialization/SerializationTest.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 9c86ac10fb1ac..0b6621cc63cdb 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -107,7 +107,7 @@ public void stringSerdeShouldSupportDifferentEncodings() { } @Test - public void listSerdeShouldShouldRoundtripInput() { + public void listSerdeShouldRoundtripInput() { List testData = Arrays.asList(1, 2, 3); Serde> listSerde = Serdes.ListSerde(Serdes.Integer()); assertEquals("Should get the original " + List.class + @@ -115,6 +115,14 @@ public void listSerdeShouldShouldRoundtripInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @Test + public void listSerdeShouldReturnNull() { + List testData = Arrays.asList(); + Serde> listSerde = Serdes.ListSerde(Serdes.Integer()); + assertEquals("Should get null after serialization and deserialization on an empty list", null, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + @Test(expected = SerializationException.class) public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() { try (Serde serde = Serdes.Float()) { From 9dee55c3bda567895a66cad8217e5dfc320b08f7 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Fri, 21 Jun 2019 14:05:11 -0400 Subject: [PATCH 11/49] Add support for fixed/variable size entries encoding If a client's inner serde is a primitive (int, long, float, etc), then we don't encode size of each entry into a target byte array. Similarly in the deserializer, we check the type of the inner serde against "primitiveDeserializers" map which will return a size of an entry (if it is a primitive), and use it to deserialize a payload. --- .../common/serialization/ListDeserializer.java | 16 +++++++++++++++- .../common/serialization/ListSerializer.java | 15 ++++++++++++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index dc9e5279ce6e2..f38608091f694 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -22,13 +22,26 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class ListDeserializer implements Deserializer> { private final Deserializer deserializer; + private final Integer primitiveSize; + + private Map primitiveDeserializers = Stream.of(new Object[][]{ + {LongDeserializer.class, 8}, + {IntegerDeserializer.class, 4}, + {ShortDeserializer.class, 2}, + {FloatDeserializer.class, 4}, + {DoubleDeserializer.class, 8}, + {BytesDeserializer.class, 1} + }).collect(Collectors.toMap(e -> (Class) e[0], e -> (Integer) e[1])); public ListDeserializer(Deserializer deserializer) { this.deserializer = deserializer; + this.primitiveSize = primitiveDeserializers.get(deserializer.getClass()); } @Override @@ -45,7 +58,8 @@ public List deserialize(String topic, byte[] data) { final int size = dis.readInt(); List deserializedList = new ArrayList<>(size); for (int i = 0; i < size; i++) { - byte[] payload = new byte[dis.readInt()]; + byte[] payload; + payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; dis.read(payload); deserializedList.add(deserializer.deserialize(topic, payload)); } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 2cdfeeb1caeca..4f58931cbba11 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -19,15 +19,26 @@ import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; public class ListSerializer implements Serializer> { private final Serializer serializer; + private final Boolean isPrimitive; + + private List primitiveSerializers = Arrays.asList( + LongSerializer.class, + IntegerSerializer.class, + ShortSerializer.class, + FloatSerializer.class, + DoubleSerializer.class, + BytesSerializer.class); public ListSerializer(Serializer serializer) { this.serializer = serializer; + this.isPrimitive = primitiveSerializers.contains(serializer.getClass()); } @Override @@ -46,7 +57,9 @@ public byte[] serialize(String topic, List data) { out.writeInt(size); for (T entry : data) { final byte[] bytes = serializer.serialize(topic, entry); - out.writeInt(bytes.length); + if (!isPrimitive) { + out.writeInt(bytes.length); + } out.write(bytes); } return baos.toByteArray(); From bd0ec99eadbca42c4f6d65416952721ae7c34fa6 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Fri, 21 Jun 2019 14:10:54 -0400 Subject: [PATCH 12/49] Force user to pass list class to listSerde User must pass a list impl class to a listSerde, so that listSerde's deserializer could create an appropriate list impl object (i.e. LinkedList, ArrayList, etc). --- .../common/serialization/ListDeserializer.java | 17 ++++++++++++++--- .../kafka/common/serialization/Serdes.java | 9 +++++---- .../common/serialization/SerializationTest.java | 4 ++-- 3 files changed, 21 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index f38608091f694..34cd9997b0638 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -19,7 +19,7 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; +import java.lang.reflect.Constructor; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -27,6 +27,7 @@ public class ListDeserializer implements Deserializer> { + private final Class listClass; private final Deserializer deserializer; private final Integer primitiveSize; @@ -39,7 +40,8 @@ public class ListDeserializer implements Deserializer> { {BytesDeserializer.class, 1} }).collect(Collectors.toMap(e -> (Class) e[0], e -> (Integer) e[1])); - public ListDeserializer(Deserializer deserializer) { + public ListDeserializer(Class listClass, Deserializer deserializer) { + this.listClass = listClass; this.deserializer = deserializer; this.primitiveSize = primitiveDeserializers.get(deserializer.getClass()); } @@ -49,6 +51,15 @@ public void configure(Map configs, boolean isKey) { deserializer.configure(configs, isKey); } + private List getListInstance(int listSize) { + try { + Constructor listConstructor = listClass.getConstructor(Integer.TYPE); + return (List) listConstructor.newInstance(listSize); + } catch (Exception e) { + throw new RuntimeException("Could not construct a list instance of \"" + listClass.getCanonicalName() + "\"", e); + } + } + @Override public List deserialize(String topic, byte[] data) { if (data == null || data.length == 0) { @@ -56,7 +67,7 @@ public List deserialize(String topic, byte[] data) { } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { final int size = dis.readInt(); - List deserializedList = new ArrayList<>(size); + List deserializedList = getListInstance(size); for (int i = 0; i < size; i++) { byte[] payload; payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 0e5ea753a4077..995bf5bedc03a 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -126,10 +126,11 @@ public UUIDSerde() { } } - static public final class ListSerde extends WrapperSerde> { - public ListSerde(Serde serde) { - super(new ListSerializer(serde.serializer()), new ListDeserializer<>(serde.deserializer())); - } + /* + * A serde for nullable {@code List} type + */ + public static Serde> ListSerde(Class listClass, Serde innerSerde) { + return new ListSerde(listClass, innerSerde); } @SuppressWarnings("unchecked") diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 0b6621cc63cdb..3319e604aa89c 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -109,7 +109,7 @@ public void stringSerdeShouldSupportDifferentEncodings() { @Test public void listSerdeShouldRoundtripInput() { List testData = Arrays.asList(1, 2, 3); - Serde> listSerde = Serdes.ListSerde(Serdes.Integer()); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); assertEquals("Should get the original " + List.class + " after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); @@ -118,7 +118,7 @@ public void listSerdeShouldRoundtripInput() { @Test public void listSerdeShouldReturnNull() { List testData = Arrays.asList(); - Serde> listSerde = Serdes.ListSerde(Serdes.Integer()); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); assertEquals("Should get null after serialization and deserialization on an empty list", null, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } From 915221659986d487370fd69375ffd1198fadcda7 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Tue, 16 Jul 2019 15:01:35 -0400 Subject: [PATCH 13/49] Introduce default zero-arg constructors Default zero-arg constructors for ListSerde, List(De)serializer for implicit, reflective instantiation by Kafka --- .../serialization/ListDeserializer.java | 27 ++++++++++--------- .../common/serialization/ListSerializer.java | 6 +++-- .../kafka/common/serialization/Serdes.java | 16 ++++++++--- 3 files changed, 31 insertions(+), 18 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 34cd9997b0638..dbb988482a4af 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -20,25 +20,26 @@ import java.io.DataInputStream; import java.io.IOException; import java.lang.reflect.Constructor; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class ListDeserializer implements Deserializer> { - private final Class listClass; - private final Deserializer deserializer; - private final Integer primitiveSize; + private Deserializer inner; + private Class listClass; + private Integer primitiveSize; - private Map primitiveDeserializers = Stream.of(new Object[][]{ - {LongDeserializer.class, 8}, - {IntegerDeserializer.class, 4}, - {ShortDeserializer.class, 2}, - {FloatDeserializer.class, 4}, - {DoubleDeserializer.class, 8}, - {BytesDeserializer.class, 1} - }).collect(Collectors.toMap(e -> (Class) e[0], e -> (Integer) e[1])); + private Map primitiveDeserializers = new HashMap() {{ + put(LongDeserializer.class, 8); + put(IntegerDeserializer.class, 4); + put(ShortDeserializer.class, 2); + put(FloatDeserializer.class, 4); + put(DoubleDeserializer.class, 8); + }}; + + public ListDeserializer() { + } public ListDeserializer(Class listClass, Deserializer deserializer) { this.listClass = listClass; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 4f58931cbba11..77bce7f297606 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -33,8 +33,10 @@ public class ListSerializer implements Serializer> { IntegerSerializer.class, ShortSerializer.class, FloatSerializer.class, - DoubleSerializer.class, - BytesSerializer.class); + DoubleSerializer.class); + + public ListSerializer() { + } public ListSerializer(Serializer serializer) { this.serializer = serializer; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 995bf5bedc03a..5ab678c9aa6ea 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -126,9 +126,6 @@ public UUIDSerde() { } } - /* - * A serde for nullable {@code List} type - */ public static Serde> ListSerde(Class listClass, Serde innerSerde) { return new ListSerde(listClass, innerSerde); } @@ -273,4 +270,17 @@ static public Serde ByteArray() { static public Serde Void() { return new VoidSerde(); } + + static public final class ListSerde extends WrapperSerde> { + + public ListSerde() { + super(new ListSerializer<>(), new ListDeserializer<>()); + } + + public ListSerde(Class listClass, Serde serde) { + super(new ListSerializer(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); + } + + } + } From 204d1e75bb698d5ad8a47442e176a10fb0847b35 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Tue, 16 Jul 2019 15:03:28 -0400 Subject: [PATCH 14/49] Introduce 4 new configuration parameters for ListSerde Introducing 4 new configuration parameters for instantiating ListSerde: default.list.key.serde.type default.list.value.serde.type default.list.key.serde.inner default.list.value.serde.inner --- .../kafka/clients/CommonClientConfigs.java | 13 ++++++++++ .../serialization/ListDeserializer.java | 24 +++++++++++++++--- .../common/serialization/ListSerializer.java | 25 ++++++++++++++----- 3 files changed, 52 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 150dac1b73cb0..5b396a4d24612 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -118,6 +118,19 @@ public class CommonClientConfigs { + "elapses the client will resend the request if necessary or fail the request if " + "retries are exhausted."; + public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS = "default.list.key.serde.inner"; + + public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = "default.list.value.serde.inner"; + + public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = "default.list.key.serde.type"; + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = "default.list.value.serde.type"; + private static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = " Default class for key that implements the java.util.List interface. " + + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + private static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = " Default class for value that implements the java.util.List interface. " + + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + public static final String GROUP_ID_CONFIG = "group.id"; public static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index dbb988482a4af..e9e0988feb5b7 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -16,6 +16,10 @@ */ package org.apache.kafka.common.serialization; +import com.sun.xml.internal.ws.encoding.soap.DeserializationException; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.utils.Utils; + import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; @@ -43,13 +47,25 @@ public ListDeserializer() { public ListDeserializer(Class listClass, Deserializer deserializer) { this.listClass = listClass; - this.deserializer = deserializer; + this.inner = deserializer; this.primitiveSize = primitiveDeserializers.get(deserializer.getClass()); } @Override public void configure(Map configs, boolean isKey) { - deserializer.configure(configs, isKey); + if (inner == null) { + String listTypePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; + String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + String listType = (String) configs.get(listTypePropertyName); + String innerSerde = (String) configs.get(innerSerdePropertyName); + try { + listClass = Class.forName(listType); + this.inner = Utils.newInstance(innerSerde, Serde.class).deserializer(); + inner.configure(configs, isKey); + } catch (ClassNotFoundException e) { + throw new DeserializationException("Could not find a class for \"" + listType + "\"", e); + } + } } private List getListInstance(int listSize) { @@ -73,7 +89,7 @@ public List deserialize(String topic, byte[] data) { byte[] payload; payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; dis.read(payload); - deserializedList.add(deserializer.deserialize(topic, payload)); + deserializedList.add(inner.deserialize(topic, payload)); } return deserializedList; } catch (IOException e) { @@ -83,7 +99,7 @@ public List deserialize(String topic, byte[] data) { @Override public void close() { - deserializer.close(); + inner.close(); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 77bce7f297606..d18159e0dc4f7 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -16,6 +16,10 @@ */ package org.apache.kafka.common.serialization; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.utils.Utils; + import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -25,8 +29,8 @@ public class ListSerializer implements Serializer> { - private final Serializer serializer; - private final Boolean isPrimitive; + private Serializer inner; + private Boolean isPrimitive; private List primitiveSerializers = Arrays.asList( LongSerializer.class, @@ -39,13 +43,22 @@ public ListSerializer() { } public ListSerializer(Serializer serializer) { - this.serializer = serializer; + this.inner = serializer; this.isPrimitive = primitiveSerializers.contains(serializer.getClass()); } @Override public void configure(Map configs, boolean isKey) { - serializer.configure(configs, isKey); + if (inner == null) { + final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final String innerSerde = (String) configs.get(innerSerdePropertyName); + try { + inner = Utils.newInstance(innerSerde, Serde.class).serializer(); + inner.configure(configs, isKey); + } catch (final ClassNotFoundException e) { + throw new ConfigException(innerSerdePropertyName, innerSerde, "Serde class " + innerSerde + " could not be found."); + } + } } @Override @@ -58,7 +71,7 @@ public byte[] serialize(String topic, List data) { try (final DataOutputStream out = new DataOutputStream(baos)) { out.writeInt(size); for (T entry : data) { - final byte[] bytes = serializer.serialize(topic, entry); + final byte[] bytes = inner.serialize(topic, entry); if (!isPrimitive) { out.writeInt(bytes.length); } @@ -72,7 +85,7 @@ public byte[] serialize(String topic, List data) { @Override public void close() { - serializer.close(); + inner.close(); } } \ No newline at end of file From 2a0149a24f83f30c9d6f3a6755e7d73cb72a5c36 Mon Sep 17 00:00:00 2001 From: DANIYAR YERALIN Date: Wed, 24 Jul 2019 11:09:16 -0400 Subject: [PATCH 15/49] Update configuration strategy for ListSerde --- .../kafka/clients/CommonClientConfigs.java | 6 ++++-- .../clients/consumer/ConsumerConfig.java | 12 +++++++++++ .../clients/producer/ProducerConfig.java | 6 ++++++ .../common/serialization/ListSerializer.java | 1 + .../apache/kafka/streams/StreamsConfig.java | 21 ++++++++++++++----- 5 files changed, 39 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 5b396a4d24612..1b791e27fce90 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -1,4 +1,4 @@ -/* + * 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. @@ -123,14 +123,16 @@ public class CommonClientConfigs { public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = "default.list.value.serde.inner"; public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = "default.list.key.serde.type"; - public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = "default.list.value.serde.type"; private static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = " Default class for key that implements the java.util.List interface. " + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = "default.list.value.serde.type"; private static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = " Default class for value that implements the java.util.List interface. " + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + public static final String GROUP_ID_CONFIG = "group.id"; public static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 72c3aee1408f6..312f752de2c2a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -224,6 +224,18 @@ public class ConsumerConfig extends AbstractConfig { public static final String CHECK_CRCS_CONFIG = "check.crcs"; private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; + /** + * list.key.serializer.inner + */ + public static final String LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG = "list.key.deserializer.inner"; + public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_DOC = "Inner deserializer class for key that implements the org.apache.kafka.common.serialization.Deserializer interface."; + + /** + * list.key.serializer.type + */ + public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG = "list.key.deserializer.type"; + public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC = "Deserializer's list class for key that implements the org.apache.kafka.common.serialization.Deserializer interface."; + /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the org.apache.kafka.common.serialization.Deserializer interface."; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index dcbb6d22199b2..cbb08fbe534c2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -215,6 +215,12 @@ public class ProducerConfig extends AbstractConfig { + " prefer to leave this config unset and instead use " + DELIVERY_TIMEOUT_MS_CONFIG + " to control" + " retry behavior."; + /** + * list.key.serializer.inner + */ + public static final String LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG = "list.key.serializer.inner"; + public static final String LIST_KEY_SERIALIZER_INNER_CLASS_DOC = "Inner serializer class for key that implements the org.apache.kafka.common.serialization.Serializer interface."; + /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the org.apache.kafka.common.serialization.Serializer interface."; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index d18159e0dc4f7..079f38fc70ee8 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -50,6 +50,7 @@ public ListSerializer(Serializer serializer) { @Override public void configure(Map configs, boolean isKey) { if (inner == null) { + final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; final String innerSerde = (String) configs.get(innerSerdePropertyName); try { diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 0e9e6a149f9b6..1e3853efb3647 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -33,11 +33,8 @@ import org.apache.kafka.common.metrics.Sensor.RecordingLevel; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; -import org.apache.kafka.streams.errors.DeserializationExceptionHandler; -import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; -import org.apache.kafka.streams.errors.ProductionExceptionHandler; -import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.errors.*; +import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.StreamThread; @@ -418,6 +415,20 @@ public class StreamsConfig extends AbstractConfig { + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; + + public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS = CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS; + + public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + + public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS; + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; + private static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = " Default class for key that implements the java.util.List interface. " + + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + private static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = " Default class for value that implements the java.util.List interface. " + + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + /** {@code default.timestamp.extractor} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "default.timestamp.extractor"; From c8b9f1ae4a878801beb58cd094852e392d6977d7 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 31 Jul 2019 13:48:11 -0400 Subject: [PATCH 16/49] Update the code due to review changes --- .../kafka/clients/CommonClientConfigs.java | 10 +-- .../clients/consumer/ConsumerConfig.java | 14 ++++- .../clients/producer/ProducerConfig.java | 7 ++- .../serialization/ListDeserializer.java | 33 ++++++---- .../common/serialization/ListSerializer.java | 32 ++++++---- .../serialization/SerializationTest.java | 62 +++++++++++++++++-- .../apache/kafka/streams/StreamsConfig.java | 31 +++++++--- 7 files changed, 145 insertions(+), 44 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 1b791e27fce90..ef53cac1d8c43 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -119,18 +119,20 @@ public class CommonClientConfigs { + "retries are exhausted."; public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS = "default.list.key.serde.inner"; + public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for key that implements the org.apache.kafka.common.serialization.Serde interface."; public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = "default.list.value.serde.inner"; + public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for value that implements the org.apache.kafka.common.serialization.Serde interface."; public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = "default.list.key.serde.type"; - private static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = " Default class for key that implements the java.util.List interface. " + public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = "Default class for key that implements the java.util.List interface. " + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "'"; public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = "default.list.value.serde.type"; - private static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = " Default class for value that implements the java.util.List interface. " + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = "Default class for value that implements the java.util.List interface. " + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; + + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "'"; public static final String GROUP_ID_CONFIG = "group.id"; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 312f752de2c2a..69b56a9e8b425 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -228,13 +228,15 @@ public class ConsumerConfig extends AbstractConfig { * list.key.serializer.inner */ public static final String LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG = "list.key.deserializer.inner"; - public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_DOC = "Inner deserializer class for key that implements the org.apache.kafka.common.serialization.Deserializer interface."; + public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_DOC = "Inner deserializer class for key that implements the java.util.List interface. " + + "This configuration will be read if and only if key.deserializer configuration is set to org.apache.kafka.common.serialization.ListDeserializer"; /** * list.key.serializer.type */ public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG = "list.key.deserializer.type"; - public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC = "Deserializer's list class for key that implements the org.apache.kafka.common.serialization.Deserializer interface."; + public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC = "Deserializer's list class for key that implements the org.apache.kafka.common.serialization.Deserializer interface." + + "This configuration will be read if and only if key.deserializer configuration is set to org.apache.kafka.common.serialization.ListDeserializer"; /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; @@ -567,6 +569,14 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG, + Type.STRING, + Importance.MEDIUM, + LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC) + .define(LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG, + Type.STRING, + Importance.MEDIUM, + LIST_KEY_DESERIALIZER_INNER_CLASS_DOC) .withClientSslSupport() .withClientSaslSupport(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index cbb08fbe534c2..097fb1796a3b4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -219,7 +219,8 @@ public class ProducerConfig extends AbstractConfig { * list.key.serializer.inner */ public static final String LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG = "list.key.serializer.inner"; - public static final String LIST_KEY_SERIALIZER_INNER_CLASS_DOC = "Inner serializer class for key that implements the org.apache.kafka.common.serialization.Serializer interface."; + public static final String LIST_KEY_SERIALIZER_INNER_CLASS_DOC = "Inner serializer class for key that implements the org.apache.kafka.common.serialization.Serializer interface." + + "This configuration will be read if and only if key.serializer configuration is set to org.apache.kafka.common.serialization.ListSerializer"; /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; @@ -431,6 +432,10 @@ public class ProducerConfig extends AbstractConfig { new ConfigDef.NonEmptyString(), Importance.LOW, TRANSACTIONAL_ID_DOC) + .define(LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG, + Type.STRING, + Importance.MEDIUM, + LIST_KEY_SERIALIZER_INNER_CLASS_DOC); .defineInternal(AUTO_DOWNGRADE_TXN_COMMIT, Type.BOOLEAN, false, diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index e9e0988feb5b7..0af91e4d7bd23 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -16,8 +16,8 @@ */ package org.apache.kafka.common.serialization; -import com.sun.xml.internal.ws.encoding.soap.DeserializationException; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.Utils; import java.io.ByteArrayInputStream; @@ -34,12 +34,13 @@ public class ListDeserializer implements Deserializer> { private Class listClass; private Integer primitiveSize; - private Map primitiveDeserializers = new HashMap() {{ - put(LongDeserializer.class, 8); - put(IntegerDeserializer.class, 4); + private Map fixedLengthDeserializers = new HashMap() {{ put(ShortDeserializer.class, 2); + put(IntegerDeserializer.class, 4); put(FloatDeserializer.class, 4); + put(LongDeserializer.class, 8); put(DoubleDeserializer.class, 8); + put(UUIDDeserializer.class, 16); }}; public ListDeserializer() { @@ -48,7 +49,7 @@ public ListDeserializer() { public ListDeserializer(Class listClass, Deserializer deserializer) { this.listClass = listClass; this.inner = deserializer; - this.primitiveSize = primitiveDeserializers.get(deserializer.getClass()); + this.primitiveSize = fixedLengthDeserializers.get(deserializer.getClass()); } @Override @@ -60,18 +61,28 @@ public void configure(Map configs, boolean isKey) { String innerSerde = (String) configs.get(innerSerdePropertyName); try { listClass = Class.forName(listType); - this.inner = Utils.newInstance(innerSerde, Serde.class).deserializer(); - inner.configure(configs, isKey); } catch (ClassNotFoundException e) { - throw new DeserializationException("Could not find a class for \"" + listType + "\"", e); + throw new ConfigException(listTypePropertyName, listType, "List type class " + listType + " could not be found."); } + try { + inner = Utils.newInstance(innerSerde, Serde.class).deserializer(); + } catch (ClassNotFoundException e) { + throw new ConfigException(innerSerdePropertyName, innerSerde, "Serde class " + innerSerde + " could not be found."); + } + inner.configure(configs, isKey); } } private List getListInstance(int listSize) { try { - Constructor listConstructor = listClass.getConstructor(Integer.TYPE); - return (List) listConstructor.newInstance(listSize); + Constructor listConstructor; + try { + listConstructor = listClass.getConstructor(Integer.TYPE); + return (List) listConstructor.newInstance(listSize); + } catch (NoSuchMethodException e) { + listConstructor = listClass.getConstructor(); + return (List) listConstructor.newInstance(); + } } catch (Exception e) { throw new RuntimeException("Could not construct a list instance of \"" + listClass.getCanonicalName() + "\"", e); } @@ -79,7 +90,7 @@ private List getListInstance(int listSize) { @Override public List deserialize(String topic, byte[] data) { - if (data == null || data.length == 0) { + if (data == null) { return null; } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 079f38fc70ee8..7ea1a81b8c237 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -30,31 +30,37 @@ public class ListSerializer implements Serializer> { private Serializer inner; - private Boolean isPrimitive; + private boolean isFixedLength; - private List primitiveSerializers = Arrays.asList( - LongSerializer.class, - IntegerSerializer.class, + private List fixedLengthSerializers = Arrays.asList( ShortSerializer.class, + IntegerSerializer.class, FloatSerializer.class, - DoubleSerializer.class); + LongSerializer.class, + DoubleSerializer.class, + UUIDSerializer.class); public ListSerializer() { } public ListSerializer(Serializer serializer) { this.inner = serializer; - this.isPrimitive = primitiveSerializers.contains(serializer.getClass()); + this.isFixedLength = fixedLengthSerializers.contains(serializer.getClass()); } @Override public void configure(Map configs, boolean isKey) { if (inner == null) { - final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - final String innerSerde = (String) configs.get(innerSerdePropertyName); + final Object innerSerde = configs.get(innerSerdePropertyName); try { - inner = Utils.newInstance(innerSerde, Serde.class).serializer(); + if (innerSerde instanceof String) { + inner = Utils.newInstance((String) innerSerde, Serde.class).serializer(); + } else if (innerSerde instanceof Class) { + inner = ((Serde) Utils.newInstance((Class) innerSerde)).serializer(); + } else { + throw new ClassNotFoundException(); + } inner.configure(configs, isKey); } catch (final ClassNotFoundException e) { throw new ConfigException(innerSerdePropertyName, innerSerde, "Serde class " + innerSerde + " could not be found."); @@ -64,16 +70,16 @@ public void configure(Map configs, boolean isKey) { @Override public byte[] serialize(String topic, List data) { - if (data == null || data.size() == 0) { + if (data == null) { return null; } final int size = data.size(); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try (final DataOutputStream out = new DataOutputStream(baos)) { + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos)) { out.writeInt(size); for (T entry : data) { final byte[] bytes = inner.serialize(topic, entry); - if (!isPrimitive) { + if (!isFixedLength) { out.writeInt(bytes.length); } out.write(bytes); diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 3319e604aa89c..db7158e1995cc 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -27,7 +27,19 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Stack; +import java.util.UUID; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsNull.nullValue; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -107,22 +119,62 @@ public void stringSerdeShouldSupportDifferentEncodings() { } @Test - public void listSerdeShouldRoundtripInput() { + public void listSerdeShouldRoundtripPrimitiveInput() { List testData = Arrays.asList(1, 2, 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get the original " + List.class + - " after serialization and deserialization", testData, + assertEquals("Should get the original collection of primitive integers" + + " after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @Test - public void listSerdeShouldReturnNull() { + public void listSerdeShouldRountripNonPrimitiveInput() { + List testData = Arrays.asList("A", "B", "C"); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); + assertEquals("Should get the original collection after serialization and deserialization on an empty list", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test + public void listSerdeShouldReturnEmptyCollection() { List testData = Arrays.asList(); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get null after serialization and deserialization on an empty list", null, + assertEquals("Should get empty collection after serialization and deserialization on an empty list", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @Test + public void listSerdeShouldReturnNull() { + List testData = null; + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals("Should get null after serialization and deserialization on an empty list", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test + public void listSerdeSerializerShouldReturnByteArrayOfSize() { + List testData = Arrays.asList(1, 2, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals("Should get length of 16 bytes (integer size of the list + 3 integer entries) after serialization", 16, + listSerde.serializer().serialize(topic, testData).length); + } + + @Test + public void listSerdeShouldReturnLinkedList() { + List testData = new LinkedList<>(); + Serde> listSerde = Serdes.ListSerde(LinkedList.class, Serdes.Integer()); + assertTrue("Should return List instance of type LinkedList", + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) instanceof LinkedList); + } + + @Test + public void listSerdeShouldReturnStack() { + List testData = new Stack<>(); + Serde> listSerde = Serdes.ListSerde(Stack.class, Serdes.Integer()); + assertTrue("Should return List instance of type Stack", + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) instanceof Stack); + } + @Test(expected = SerializationException.class) public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() { try (Serde serde = Serdes.Float()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 1e3853efb3647..f7c746cb2649c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -33,7 +33,11 @@ import org.apache.kafka.common.metrics.Sensor.RecordingLevel; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.errors.*; +import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; +import org.apache.kafka.streams.errors.ProductionExceptionHandler; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.TimestampExtractor; @@ -421,13 +425,8 @@ public class StreamsConfig extends AbstractConfig { public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS; + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; - private static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = " Default class for key that implements the java.util.List interface. " - + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; - private static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = " Default class for value that implements the java.util.List interface. " - + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "' as well"; /** {@code default.timestamp.extractor} */ @SuppressWarnings("WeakerAccess") @@ -878,7 +877,23 @@ public class StreamsConfig extends AbstractConfig { Type.LONG, null, Importance.LOW, - WINDOW_SIZE_MS_DOC); + WINDOW_SIZE_MS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, + Type.STRING, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, + Type.STRING, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, + Type.STRING, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, + Type.STRING, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC); } // this is the list of configs for underlying clients From 90cc373e1ba1db0756c754e0b4fb4bf5013c9235 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 5 Aug 2019 10:05:17 -0400 Subject: [PATCH 17/49] Set all new config definitions of Type.CLASS --- .../kafka/clients/consumer/ConsumerConfig.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 69b56a9e8b425..a872133503059 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -569,14 +569,14 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) - .define(LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG, - Type.STRING, - Importance.MEDIUM, - LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC) - .define(LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG, - Type.STRING, - Importance.MEDIUM, - LIST_KEY_DESERIALIZER_INNER_CLASS_DOC) + .define(LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG, + Type.CLASS, + Importance.MEDIUM, + LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC) + .define(LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG, + Type.CLASS, + Importance.MEDIUM, + LIST_KEY_DESERIALIZER_INNER_CLASS_DOC) .withClientSslSupport() .withClientSaslSupport(); } From 6e61cd1c2fa0aba9c9546b2307618700c2bfb692 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 12 Aug 2019 14:51:50 -0400 Subject: [PATCH 18/49] Set default values for newly introduced properties to null --- .../apache/kafka/clients/CommonClientConfigs.java | 3 +-- .../kafka/clients/consumer/ConsumerConfig.java | 2 ++ .../kafka/clients/producer/ProducerConfig.java | 5 +++-- .../java/org/apache/kafka/streams/StreamsConfig.java | 12 ++++++++---- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index ef53cac1d8c43..29b627adf0a10 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -1,4 +1,4 @@ - +/* * 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. @@ -134,7 +134,6 @@ public class CommonClientConfigs { + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "'"; - public static final String GROUP_ID_CONFIG = "group.id"; public static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index a872133503059..96bb03242a8e5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -571,10 +571,12 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.SECURITY_PROTOCOL_DOC) .define(LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG, Type.CLASS, + null, Importance.MEDIUM, LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC) .define(LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG, Type.CLASS, + null, Importance.MEDIUM, LIST_KEY_DESERIALIZER_INNER_CLASS_DOC) .withClientSslSupport() diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 097fb1796a3b4..9c50d29371887 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -432,8 +432,9 @@ public class ProducerConfig extends AbstractConfig { new ConfigDef.NonEmptyString(), Importance.LOW, TRANSACTIONAL_ID_DOC) - .define(LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG, - Type.STRING, + .define(LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG, + Type.CLASS, + null, Importance.MEDIUM, LIST_KEY_SERIALIZER_INNER_CLASS_DOC); .defineInternal(AUTO_DOWNGRADE_TXN_COMMIT, diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index f7c746cb2649c..589ed2c054c62 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -879,19 +879,23 @@ public class StreamsConfig extends AbstractConfig { Importance.LOW, WINDOW_SIZE_MS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, - Type.STRING, + Type.CLASS, + null, Importance.MEDIUM, CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, - Type.STRING, + Type.CLASS, + null, Importance.MEDIUM, CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, - Type.STRING, + Type.CLASS, + null, Importance.MEDIUM, CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, - Type.STRING, + Type.CLASS, + null, Importance.MEDIUM, CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC); } From a884d8bd039ab890ecc0f9cde16018054e37e248 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 12 Aug 2019 14:52:38 -0400 Subject: [PATCH 19/49] Suppress unchecked warnings --- .../common/serialization/ListDeserializer.java | 15 ++++++++------- .../common/serialization/ListSerializer.java | 1 + .../apache/kafka/common/serialization/Serdes.java | 1 + 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 0af91e4d7bd23..56fdb88b403d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +@SuppressWarnings(value = "unchecked") public class ListDeserializer implements Deserializer> { private Deserializer inner; @@ -35,13 +36,13 @@ public class ListDeserializer implements Deserializer> { private Integer primitiveSize; private Map fixedLengthDeserializers = new HashMap() {{ - put(ShortDeserializer.class, 2); - put(IntegerDeserializer.class, 4); - put(FloatDeserializer.class, 4); - put(LongDeserializer.class, 8); - put(DoubleDeserializer.class, 8); - put(UUIDDeserializer.class, 16); - }}; + put(ShortDeserializer.class, 2); + put(IntegerDeserializer.class, 4); + put(FloatDeserializer.class, 4); + put(LongDeserializer.class, 8); + put(DoubleDeserializer.class, 8); + put(UUIDDeserializer.class, 16); + }}; public ListDeserializer() { } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 7ea1a81b8c237..9b35485cb185d 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; +@SuppressWarnings(value = "unchecked") public class ListSerializer implements Serializer> { private Serializer inner; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 5ab678c9aa6ea..2317a4d6406b5 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -277,6 +277,7 @@ public ListSerde() { super(new ListSerializer<>(), new ListDeserializer<>()); } + @SuppressWarnings(value = "unchecked") public ListSerde(Class listClass, Serde serde) { super(new ListSerializer(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); } From fe95b01608d262d0685c5b008ca1220eb4138a34 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 9 Sep 2019 10:50:07 -0400 Subject: [PATCH 20/49] Allow import of "org.apache.kafka.clients" package in serialization package --- checkstyle/import-control.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 318384fca9c0b..c98cfab787d9f 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -189,6 +189,7 @@ + From 7353b6527c1806052e7987a4f20e20aaa4b2bac1 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Fri, 13 Sep 2019 16:19:40 -0400 Subject: [PATCH 21/49] Fix spotbug warning --- .../apache/kafka/common/serialization/ListDeserializer.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 56fdb88b403d5..2e0529110c6e8 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.utils.Utils; import java.io.ByteArrayInputStream; @@ -100,7 +101,9 @@ public List deserialize(String topic, byte[] data) { for (int i = 0; i < size; i++) { byte[] payload; payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; - dis.read(payload); + if (dis.read(payload) == -1) { + throw new SerializationException("End of the stream was reached prematurely"); + } deserializedList.add(inner.deserialize(topic, payload)); } return deserializedList; From 753066922c6183f39d0c9cffcf4c624806075058 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 30 Sep 2019 14:17:06 -0400 Subject: [PATCH 22/49] Generify List class blah --- .../serialization/ListDeserializer.java | 37 ++++++++++--------- .../common/serialization/ListSerializer.java | 6 +-- .../kafka/common/serialization/Serdes.java | 13 ++++--- 3 files changed, 29 insertions(+), 27 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 2e0529110c6e8..98125f9acc46c 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -29,31 +29,32 @@ import java.util.List; import java.util.Map; -@SuppressWarnings(value = "unchecked") -public class ListDeserializer implements Deserializer> { +public class ListDeserializer, T> implements Deserializer { private Deserializer inner; - private Class listClass; + private Class listClass; private Integer primitiveSize; - private Map fixedLengthDeserializers = new HashMap() {{ - put(ShortDeserializer.class, 2); - put(IntegerDeserializer.class, 4); - put(FloatDeserializer.class, 4); - put(LongDeserializer.class, 8); - put(DoubleDeserializer.class, 8); - put(UUIDDeserializer.class, 16); + static private Map, Integer> fixedLengthDeserializers = + new HashMap, Integer>() {{ + put(ShortDeserializer.class, 2); + put(IntegerDeserializer.class, 4); + put(FloatDeserializer.class, 4); + put(LongDeserializer.class, 8); + put(DoubleDeserializer.class, 8); + put(UUIDDeserializer.class, 16); }}; public ListDeserializer() { } - public ListDeserializer(Class listClass, Deserializer deserializer) { + public ListDeserializer(Class listClass, Deserializer deserializer) { this.listClass = listClass; this.inner = deserializer; this.primitiveSize = fixedLengthDeserializers.get(deserializer.getClass()); } + @SuppressWarnings(value = "unchecked") @Override public void configure(Map configs, boolean isKey) { if (inner == null) { @@ -62,7 +63,7 @@ public void configure(Map configs, boolean isKey) { String listType = (String) configs.get(listTypePropertyName); String innerSerde = (String) configs.get(innerSerdePropertyName); try { - listClass = Class.forName(listType); + listClass = (Class) Class.forName(listType); } catch (ClassNotFoundException e) { throw new ConfigException(listTypePropertyName, listType, "List type class " + listType + " could not be found."); } @@ -75,15 +76,15 @@ public void configure(Map configs, boolean isKey) { } } - private List getListInstance(int listSize) { + private L getListInstance(int listSize) { try { - Constructor listConstructor; + Constructor listConstructor; try { listConstructor = listClass.getConstructor(Integer.TYPE); - return (List) listConstructor.newInstance(listSize); + return listConstructor.newInstance(listSize); } catch (NoSuchMethodException e) { listConstructor = listClass.getConstructor(); - return (List) listConstructor.newInstance(); + return listConstructor.newInstance(); } } catch (Exception e) { throw new RuntimeException("Could not construct a list instance of \"" + listClass.getCanonicalName() + "\"", e); @@ -91,13 +92,13 @@ private List getListInstance(int listSize) { } @Override - public List deserialize(String topic, byte[] data) { + public L deserialize(String topic, byte[] data) { if (data == null) { return null; } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { final int size = dis.readInt(); - List deserializedList = getListInstance(size); + L deserializedList = getListInstance(size); for (int i = 0; i < size; i++) { byte[] payload; payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 9b35485cb185d..201cd6fc3c2ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -28,12 +28,12 @@ import java.util.Map; @SuppressWarnings(value = "unchecked") -public class ListSerializer implements Serializer> { +public class ListSerializer, T> implements Serializer { private Serializer inner; private boolean isFixedLength; - private List fixedLengthSerializers = Arrays.asList( + static private List> fixedLengthSerializers = Arrays.asList( ShortSerializer.class, IntegerSerializer.class, FloatSerializer.class, @@ -70,7 +70,7 @@ public void configure(Map configs, boolean isKey) { } @Override - public byte[] serialize(String topic, List data) { + public byte[] serialize(String topic, L data) { if (data == null) { return null; } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 2317a4d6406b5..71d48a14c0e55 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -126,8 +126,9 @@ public UUIDSerde() { } } - public static Serde> ListSerde(Class listClass, Serde innerSerde) { - return new ListSerde(listClass, innerSerde); + @SuppressWarnings("unchecked") + public static Serde> ListSerde(Class listClass, Serde innerSerde) { + return new ListSerde<>(listClass, innerSerde); } @SuppressWarnings("unchecked") @@ -271,15 +272,15 @@ static public Serde Void() { return new VoidSerde(); } - static public final class ListSerde extends WrapperSerde> { + static public final class ListSerde, T> extends WrapperSerde> { public ListSerde() { super(new ListSerializer<>(), new ListDeserializer<>()); } - @SuppressWarnings(value = "unchecked") - public ListSerde(Class listClass, Serde serde) { - super(new ListSerializer(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); + @SuppressWarnings("unchecked") + public ListSerde(Class listClass, Serde serde) { + super(new ListSerializer<>(serde.serializer()), (Deserializer>) new ListDeserializer<>(listClass, serde.deserializer())); } } From 17cb8e7ecf7b1835ca4a9889103a6006d64dd4ad Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 1 Oct 2019 15:07:51 -0400 Subject: [PATCH 23/49] Remove deprecated import Possibly caused by bad rebase --- .../src/main/java/org/apache/kafka/streams/StreamsConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 589ed2c054c62..0b481d453f8d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -38,7 +38,6 @@ import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; import org.apache.kafka.streams.errors.ProductionExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.StreamThread; From 85a791b021b7bbb29141bbc4baec4aa30646e132 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 30 Oct 2019 15:58:29 -0400 Subject: [PATCH 24/49] Use mkMap and mkEntry to populate fixedLengthDeserializers map Use org.apache.kafka.common.utils.Utils#mkMap and org.apache.kafka.common.utils.Utils#mkEntry to populate org.apache.kafka.common.serialization.ListDeserializer#fixedLengthDeserializers map --- .../serialization/ListDeserializer.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 98125f9acc46c..a343b93cee5e0 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -25,28 +25,28 @@ import java.io.DataInputStream; import java.io.IOException; import java.lang.reflect.Constructor; -import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; + public class ListDeserializer, T> implements Deserializer { private Deserializer inner; private Class listClass; private Integer primitiveSize; - static private Map, Integer> fixedLengthDeserializers = - new HashMap, Integer>() {{ - put(ShortDeserializer.class, 2); - put(IntegerDeserializer.class, 4); - put(FloatDeserializer.class, 4); - put(LongDeserializer.class, 8); - put(DoubleDeserializer.class, 8); - put(UUIDDeserializer.class, 16); - }}; + static private Map, Integer> fixedLengthDeserializers = mkMap( + mkEntry(ShortDeserializer.class, 2), + mkEntry(IntegerDeserializer.class, 4), + mkEntry(FloatDeserializer.class, 4), + mkEntry(LongDeserializer.class, 8), + mkEntry(DoubleDeserializer.class, 8), + mkEntry(UUIDDeserializer.class, 16) + ); - public ListDeserializer() { - } + public ListDeserializer() {} public ListDeserializer(Class listClass, Deserializer deserializer) { this.listClass = listClass; From 11d09d235c62fea256ee2a9cc65fcccfa76bfcad Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 30 Oct 2019 16:00:39 -0400 Subject: [PATCH 25/49] Move SuppressWarnings statement on a method level --- .../apache/kafka/common/serialization/ListSerializer.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 201cd6fc3c2ba..a6260073a1994 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; -@SuppressWarnings(value = "unchecked") public class ListSerializer, T> implements Serializer { private Serializer inner; @@ -41,14 +40,14 @@ public class ListSerializer, T> implements Serializer { DoubleSerializer.class, UUIDSerializer.class); - public ListSerializer() { - } + public ListSerializer() {} public ListSerializer(Serializer serializer) { this.inner = serializer; this.isFixedLength = fixedLengthSerializers.contains(serializer.getClass()); } + @SuppressWarnings(value = "unchecked") @Override public void configure(Map configs, boolean isKey) { if (inner == null) { From 71bdf49cdc046cd8e469e8193f8bd3beea6e30ed Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 30 Oct 2019 17:08:56 -0400 Subject: [PATCH 26/49] Update UUID fixed size to 36 bytes --- .../org/apache/kafka/common/serialization/ListDeserializer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index a343b93cee5e0..efc4a93ca3901 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -43,7 +43,7 @@ public class ListDeserializer, T> implements Deserializer { mkEntry(FloatDeserializer.class, 4), mkEntry(LongDeserializer.class, 8), mkEntry(DoubleDeserializer.class, 8), - mkEntry(UUIDDeserializer.class, 16) + mkEntry(UUIDDeserializer.class, 36) ); public ListDeserializer() {} From 097e78084528c7c3012e6ba83937fb8d14e252f7 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 30 Oct 2019 17:09:33 -0400 Subject: [PATCH 27/49] Add more test cases to cover other types of fixed sizes More test cases for: short, float, long, double, UUID --- .../serialization/SerializationTest.java | 120 +++++++++++++++--- 1 file changed, 103 insertions(+), 17 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index db7158e1995cc..cabe5d8b6eb06 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -119,46 +119,132 @@ public void stringSerdeShouldSupportDifferentEncodings() { } @Test - public void listSerdeShouldRoundtripPrimitiveInput() { - List testData = Arrays.asList(1, 2, 3); + public void listSerdeShouldReturnEmptyCollection() { + List testData = Arrays.asList(); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get the original collection of primitive integers" - + " after serialization and deserialization", testData, + assertEquals("Should get empty collection after serialization and deserialization on an empty list", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @Test - public void listSerdeShouldRountripNonPrimitiveInput() { - List testData = Arrays.asList("A", "B", "C"); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); - assertEquals("Should get the original collection after serialization and deserialization on an empty list", testData, + public void listSerdeShouldReturnNull() { + List testData = null; + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals("Should get null after serialization and deserialization on an empty list", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @Test - public void listSerdeShouldReturnEmptyCollection() { - List testData = Arrays.asList(); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get empty collection after serialization and deserialization on an empty list", testData, + public void listSerdeShouldRoundtripShortPrimitiveInput() { + List testData = Arrays.asList((short) 1, (short) 2, (short) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); + assertEquals("Should get the original collection of short primitives" + + " after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @Test - public void listSerdeShouldReturnNull() { - List testData = null; + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitiveInput() { + List testData = Arrays.asList((short) 1, (short) 2, (short) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); + assertEquals("Should get length of 10 bytes (size of the list + 3 short entries) after serialization", 10, + listSerde.serializer().serialize(topic, testData).length); + } + + @Test + public void listSerdeShouldRoundtripIntPrimitiveInput() { + List testData = Arrays.asList(1, 2, 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get null after serialization and deserialization on an empty list", testData, + assertEquals("Should get the original collection of integer primitives" + + " after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @Test - public void listSerdeSerializerShouldReturnByteArrayOfSize() { + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveInput() { List testData = Arrays.asList(1, 2, 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get length of 16 bytes (integer size of the list + 3 integer entries) after serialization", 16, + assertEquals("Should get length of 16 bytes (size of the list + 3 integer entries) after serialization", 16, + listSerde.serializer().serialize(topic, testData).length); + } + + @Test + public void listSerdeShouldRoundtripFloatPrimitiveInput() { + List testData = Arrays.asList((float) 1, (float) 2, (float) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); + assertEquals("Should get the original collection of float primitives" + + " after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitiveInput() { + List testData = Arrays.asList((float) 1, (float) 2, (float) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); + assertEquals("Should get length of 16 bytes (size of the list + 3 float entries) after serialization", 16, listSerde.serializer().serialize(topic, testData).length); } + @Test + public void listSerdeShouldRoundtripLongPrimitiveInput() { + List testData = Arrays.asList((long) 1, (long) 2, (long) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); + assertEquals("Should get the original collection of long primitives" + + " after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveInput() { + List testData = Arrays.asList((long) 1, (long) 2, (long) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); + assertEquals("Should get length of 24 bytes (size of the list + 3 long entries) after serialization", 28, + listSerde.serializer().serialize(topic, testData).length); + } + + @Test + public void listSerdeShouldRoundtripDoublePrimitiveInput() { + List testData = Arrays.asList((double) 1, (double) 2, (double) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); + assertEquals("Should get the original collection of double primitives" + + " after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiveInput() { + List testData = Arrays.asList((double) 1, (double) 2, (double) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); + assertEquals("Should get length of 24 bytes (size of the list + 3 double entries) after serialization", 28, + listSerde.serializer().serialize(topic, testData).length); + } + + @Test + public void listSerdeShouldRoundtripUUIDInput() { + List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); + assertEquals("Should get the original collection of UUID" + + " after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { + List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); + assertEquals("Should get length of 52 bytes (size of the list + 3 UUID entries) after serialization", 112, + listSerde.serializer().serialize(topic, testData).length); + } + + @Test + public void listSerdeShouldRountripNonPrimitiveInput() { + List testData = Arrays.asList("A", "B", "C"); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); + assertEquals("Should get the original collection after serialization and deserialization on an empty list", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @Test public void listSerdeShouldReturnLinkedList() { List testData = new LinkedList<>(); From a4d4d89fa301929dc97ef469dffe95fe6a97c2ec Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 12 Nov 2019 14:07:52 -0500 Subject: [PATCH 28/49] Rearrange defines --- .../apache/kafka/streams/StreamsConfig.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 0b481d453f8d0..540bb80e66116 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -652,6 +652,26 @@ public class StreamsConfig extends AbstractConfig { Serdes.ByteArraySerde.class.getName(), Importance.MEDIUM, DEFAULT_KEY_SERDE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, DefaultProductionExceptionHandler.class.getName(), From bd9740f929b987cab8e0444220022a0e8ac20044 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 12 Nov 2019 14:08:16 -0500 Subject: [PATCH 29/49] Modify interface definitions --- .../serialization/ListDeserializer.java | 56 ++++++++----------- .../common/serialization/ListSerializer.java | 24 ++++---- .../kafka/common/serialization/Serdes.java | 29 +++++----- 3 files changed, 50 insertions(+), 59 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index efc4a93ca3901..75c78fb9f28c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.serialization; import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.utils.Utils; @@ -31,27 +30,27 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -public class ListDeserializer, T> implements Deserializer { +public class ListDeserializer implements Deserializer> { - private Deserializer inner; - private Class listClass; + private Deserializer inner; + private Class listClass; private Integer primitiveSize; static private Map, Integer> fixedLengthDeserializers = mkMap( - mkEntry(ShortDeserializer.class, 2), - mkEntry(IntegerDeserializer.class, 4), - mkEntry(FloatDeserializer.class, 4), - mkEntry(LongDeserializer.class, 8), - mkEntry(DoubleDeserializer.class, 8), - mkEntry(UUIDDeserializer.class, 36) + mkEntry(ShortDeserializer.class, 2), + mkEntry(IntegerDeserializer.class, 4), + mkEntry(FloatDeserializer.class, 4), + mkEntry(LongDeserializer.class, 8), + mkEntry(DoubleDeserializer.class, 8), + mkEntry(UUIDDeserializer.class, 36) ); public ListDeserializer() {} - public ListDeserializer(Class listClass, Deserializer deserializer) { + public ListDeserializer(Class listClass, Deserializer innerDeserializer) { this.listClass = listClass; - this.inner = deserializer; - this.primitiveSize = fixedLengthDeserializers.get(deserializer.getClass()); + this.inner = innerDeserializer; + this.primitiveSize = fixedLengthDeserializers.get(innerDeserializer.getClass()); } @SuppressWarnings(value = "unchecked") @@ -60,30 +59,22 @@ public void configure(Map configs, boolean isKey) { if (inner == null) { String listTypePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - String listType = (String) configs.get(listTypePropertyName); - String innerSerde = (String) configs.get(innerSerdePropertyName); - try { - listClass = (Class) Class.forName(listType); - } catch (ClassNotFoundException e) { - throw new ConfigException(listTypePropertyName, listType, "List type class " + listType + " could not be found."); - } - try { - inner = Utils.newInstance(innerSerde, Serde.class).deserializer(); - } catch (ClassNotFoundException e) { - throw new ConfigException(innerSerdePropertyName, innerSerde, "Serde class " + innerSerde + " could not be found."); - } + listClass = (Class>) configs.get(listTypePropertyName); + Class innerSerde = (Class) configs.get(innerSerdePropertyName); + inner = Utils.newInstance(innerSerde).deserializer(); inner.configure(configs, isKey); } } - private L getListInstance(int listSize) { + @SuppressWarnings(value = "unchecked") + private List getListInstance(int listSize) { try { - Constructor listConstructor; + Constructor> listConstructor; try { - listConstructor = listClass.getConstructor(Integer.TYPE); + listConstructor = (Constructor>) listClass.getConstructor(Integer.TYPE); return listConstructor.newInstance(listSize); } catch (NoSuchMethodException e) { - listConstructor = listClass.getConstructor(); + listConstructor = (Constructor>) listClass.getConstructor(); return listConstructor.newInstance(); } } catch (Exception e) { @@ -92,16 +83,15 @@ private L getListInstance(int listSize) { } @Override - public L deserialize(String topic, byte[] data) { + public List deserialize(String topic, byte[] data) { if (data == null) { return null; } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { final int size = dis.readInt(); - L deserializedList = getListInstance(size); + List deserializedList = getListInstance(size); for (int i = 0; i < size; i++) { - byte[] payload; - payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; + byte[] payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; if (dis.read(payload) == -1) { throw new SerializationException("End of the stream was reached prematurely"); } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index a6260073a1994..e29b2958ebd7e 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -27,22 +27,22 @@ import java.util.List; import java.util.Map; -public class ListSerializer, T> implements Serializer { +public class ListSerializer implements Serializer> { - private Serializer inner; + private Serializer inner; private boolean isFixedLength; static private List> fixedLengthSerializers = Arrays.asList( - ShortSerializer.class, - IntegerSerializer.class, - FloatSerializer.class, - LongSerializer.class, - DoubleSerializer.class, - UUIDSerializer.class); + ShortSerializer.class, + IntegerSerializer.class, + FloatSerializer.class, + LongSerializer.class, + DoubleSerializer.class, + UUIDSerializer.class); public ListSerializer() {} - public ListSerializer(Serializer serializer) { + public ListSerializer(Serializer serializer) { this.inner = serializer; this.isFixedLength = fixedLengthSerializers.contains(serializer.getClass()); } @@ -57,7 +57,7 @@ public void configure(Map configs, boolean isKey) { if (innerSerde instanceof String) { inner = Utils.newInstance((String) innerSerde, Serde.class).serializer(); } else if (innerSerde instanceof Class) { - inner = ((Serde) Utils.newInstance((Class) innerSerde)).serializer(); + inner = ((Serde) Utils.newInstance((Class) innerSerde)).serializer(); } else { throw new ClassNotFoundException(); } @@ -69,7 +69,7 @@ public void configure(Map configs, boolean isKey) { } @Override - public byte[] serialize(String topic, L data) { + public byte[] serialize(String topic, List data) { if (data == null) { return null; } @@ -77,7 +77,7 @@ public byte[] serialize(String topic, L data) { try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream out = new DataOutputStream(baos)) { out.writeInt(size); - for (T entry : data) { + for (Inner entry : data) { final byte[] bytes = inner.serialize(topic, entry); if (!isFixedLength) { out.writeInt(bytes.length); diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 71d48a14c0e55..687a993bb3901 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -126,9 +126,15 @@ public UUIDSerde() { } } - @SuppressWarnings("unchecked") - public static Serde> ListSerde(Class listClass, Serde innerSerde) { - return new ListSerde<>(listClass, innerSerde); + static public final class ListSerde extends WrapperSerde> { + + public ListSerde() { + super(new ListSerializer<>(), new ListDeserializer<>()); + } + + public ListSerde(Class listClass, Serde serde) { + super(new ListSerializer<>(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); + } } @SuppressWarnings("unchecked") @@ -272,17 +278,12 @@ static public Serde Void() { return new VoidSerde(); } - static public final class ListSerde, T> extends WrapperSerde> { - - public ListSerde() { - super(new ListSerializer<>(), new ListDeserializer<>()); - } - - @SuppressWarnings("unchecked") - public ListSerde(Class listClass, Serde serde) { - super(new ListSerializer<>(serde.serializer()), (Deserializer>) new ListDeserializer<>(listClass, serde.deserializer())); - } - + /* + * A serde for {@code List} type + */ + public static Serde> + ListSerde(Class listClass, Serde innerSerde) { + return new ListSerde<>(listClass, innerSerde); } } From dafda14b836899161bcecda6cb40565f326b915b Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 12 Nov 2019 14:17:20 -0500 Subject: [PATCH 30/49] Make more descriptive docs Fix typo Complete fixing typo --- .../org/apache/kafka/clients/CommonClientConfigs.java | 8 ++++++-- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 4 ++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 29b627adf0a10..58075d628927e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -119,18 +119,22 @@ public class CommonClientConfigs { + "retries are exhausted."; public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS = "default.list.key.serde.inner"; - public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for key that implements the org.apache.kafka.common.serialization.Serde interface."; + public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for key that implements the org.apache.kafka.common.serialization.Serde interface. " + + "This configuration will be read if and only if default.key.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde"; public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = "default.list.value.serde.inner"; - public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for value that implements the org.apache.kafka.common.serialization.Serde interface."; + public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for value that implements the org.apache.kafka.common.serialization.Serde interface. " + + "This configuration will be read if and only if default.value.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde"; public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = "default.list.key.serde.type"; public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = "Default class for key that implements the java.util.List interface. " + + "This configuration will be read if and only if default.key.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde " + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "'"; public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = "default.list.value.serde.type"; public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = "Default class for value that implements the java.util.List interface. " + + "This configuration will be read if and only if default.value.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde " + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "'"; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 96bb03242a8e5..00f3bc41b6f95 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -227,7 +227,7 @@ public class ConsumerConfig extends AbstractConfig { /** * list.key.serializer.inner */ - public static final String LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG = "list.key.deserializer.inner"; + public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_CONFIG = "list.key.deserializer.inner"; public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_DOC = "Inner deserializer class for key that implements the java.util.List interface. " + "This configuration will be read if and only if key.deserializer configuration is set to org.apache.kafka.common.serialization.ListDeserializer"; @@ -574,7 +574,7 @@ public class ConsumerConfig extends AbstractConfig { null, Importance.MEDIUM, LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC) - .define(LIST_KEY_DEERIALIZER_INNER_CLASS_CONFIG, + .define(LIST_KEY_DESERIALIZER_INNER_CLASS_CONFIG, Type.CLASS, null, Importance.MEDIUM, From bac39aab9e99edc9bb839165f10fd194431b7cbb Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 27 Jan 2020 14:13:07 -0500 Subject: [PATCH 31/49] Update the code due to review comments Commit review changes --- .../serialization/ListDeserializer.java | 39 ++++++++++++------- .../common/serialization/ListSerializer.java | 32 ++++++++------- .../kafka/common/serialization/Serdes.java | 5 ++- .../serialization/SerializationTest.java | 19 ++++++++- .../apache/kafka/streams/StreamsConfig.java | 9 ----- 5 files changed, 66 insertions(+), 38 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 75c78fb9f28c6..ee97cf43ff2cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -17,6 +17,8 @@ package org.apache.kafka.common.serialization; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.utils.Utils; @@ -24,6 +26,7 @@ import java.io.DataInputStream; import java.io.IOException; import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.List; import java.util.Map; @@ -36,7 +39,7 @@ public class ListDeserializer implements Deserializer> { private Class listClass; private Integer primitiveSize; - static private Map, Integer> fixedLengthDeserializers = mkMap( + static private Map>, Integer> fixedLengthDeserializers = mkMap( mkEntry(ShortDeserializer.class, 2), mkEntry(IntegerDeserializer.class, 4), mkEntry(FloatDeserializer.class, 4), @@ -47,26 +50,33 @@ public class ListDeserializer implements Deserializer> { public ListDeserializer() {} - public ListDeserializer(Class listClass, Deserializer innerDeserializer) { + public > ListDeserializer(Class listClass, Deserializer innerDeserializer) { this.listClass = listClass; this.inner = innerDeserializer; - this.primitiveSize = fixedLengthDeserializers.get(innerDeserializer.getClass()); + if (innerDeserializer != null) { + this.primitiveSize = fixedLengthDeserializers.get(innerDeserializer.getClass()); + } } - @SuppressWarnings(value = "unchecked") + @SuppressWarnings("unchecked") @Override public void configure(Map configs, boolean isKey) { - if (inner == null) { + if (listClass == null) { String listTypePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; - String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; listClass = (Class>) configs.get(listTypePropertyName); - Class innerSerde = (Class) configs.get(innerSerdePropertyName); - inner = Utils.newInstance(innerSerde).deserializer(); + if (listClass == null) { + throw new ConfigException("Not able to determine the list class because it was neither passed via the constructor nor set in the config"); + } + } + if (inner == null) { + String innerDeserializerPropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + Class> innerDeserializerClass = (Class>) configs.get(innerDeserializerPropertyName); + inner = Utils.newInstance(innerDeserializerClass); inner.configure(configs, isKey); } } - @SuppressWarnings(value = "unchecked") + @SuppressWarnings("unchecked") private List getListInstance(int listSize) { try { Constructor> listConstructor; @@ -77,8 +87,9 @@ private List getListInstance(int listSize) { listConstructor = (Constructor>) listClass.getConstructor(); return listConstructor.newInstance(); } - } catch (Exception e) { - throw new RuntimeException("Could not construct a list instance of \"" + listClass.getCanonicalName() + "\"", e); + } catch (InstantiationException | IllegalAccessException | NoSuchMethodException | + IllegalArgumentException | InvocationTargetException e) { + throw new KafkaException("Could not construct a list instance of \"" + listClass.getCanonicalName() + "\"", e); } } @@ -99,13 +110,15 @@ public List deserialize(String topic, byte[] data) { } return deserializedList; } catch (IOException e) { - throw new RuntimeException("Unable to deserialize into a List", e); + throw new KafkaException("Unable to deserialize into a List", e); } } @Override public void close() { - inner.close(); + if (inner != null) { + inner.close(); + } } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index e29b2958ebd7e..c84953d7a4526 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.serialization; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.Utils; @@ -32,7 +33,7 @@ public class ListSerializer implements Serializer> { private Serializer inner; private boolean isFixedLength; - static private List> fixedLengthSerializers = Arrays.asList( + static private List>> fixedLengthSerializers = Arrays.asList( ShortSerializer.class, IntegerSerializer.class, FloatSerializer.class, @@ -44,26 +45,29 @@ public ListSerializer() {} public ListSerializer(Serializer serializer) { this.inner = serializer; - this.isFixedLength = fixedLengthSerializers.contains(serializer.getClass()); + this.isFixedLength = serializer != null && fixedLengthSerializers.contains(serializer.getClass()); } - @SuppressWarnings(value = "unchecked") + @SuppressWarnings("unchecked") @Override public void configure(Map configs, boolean isKey) { if (inner == null) { - final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - final Object innerSerde = configs.get(innerSerdePropertyName); + final String innerSerializerPropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final Object innerSerializerClassOrName = configs.get(innerSerializerPropertyName); + if (innerSerializerClassOrName == null) { + throw new ConfigException("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config"); + } try { - if (innerSerde instanceof String) { - inner = Utils.newInstance((String) innerSerde, Serde.class).serializer(); - } else if (innerSerde instanceof Class) { - inner = ((Serde) Utils.newInstance((Class) innerSerde)).serializer(); + if (innerSerializerClassOrName instanceof String) { + inner = Utils.newInstance((String) innerSerializerClassOrName, Serializer.class); + } else if (innerSerializerClassOrName instanceof Class) { + inner = Utils.newInstance((Class>) innerSerializerClassOrName); } else { - throw new ClassNotFoundException(); + throw new KafkaException("Could not create a serializer class instance using \"" + innerSerializerPropertyName + "\" property."); } inner.configure(configs, isKey); } catch (final ClassNotFoundException e) { - throw new ConfigException(innerSerdePropertyName, innerSerde, "Serde class " + innerSerde + " could not be found."); + throw new ConfigException(innerSerializerPropertyName, innerSerializerClassOrName, "Serializer class " + innerSerializerClassOrName + " could not be found."); } } } @@ -86,13 +90,15 @@ public byte[] serialize(String topic, List data) { } return baos.toByteArray(); } catch (IOException e) { - throw new RuntimeException("Failed to serialize List", e); + throw new KafkaException("Failed to serialize List", e); } } @Override public void close() { - inner.close(); + if (inner != null) { + inner.close(); + } } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 687a993bb3901..b0946c274767f 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -132,7 +132,7 @@ public ListSerde() { super(new ListSerializer<>(), new ListDeserializer<>()); } - public ListSerde(Class listClass, Serde serde) { + public > ListSerde(Class listClass, Serde serde) { super(new ListSerializer<>(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); } } @@ -281,8 +281,9 @@ static public Serde Void() { /* * A serde for {@code List} type */ - public static Serde> + public static , Inner> Serde> ListSerde(Class listClass, Serde innerSerde) { + return new ListSerde<>(listClass, innerSerde); } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index cabe5d8b6eb06..acc236669bd47 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -118,6 +118,7 @@ public void stringSerdeShouldSupportDifferentEncodings() { } } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldReturnEmptyCollection() { List testData = Arrays.asList(); @@ -126,6 +127,7 @@ public void listSerdeShouldReturnEmptyCollection() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldReturnNull() { List testData = null; @@ -134,6 +136,7 @@ public void listSerdeShouldReturnNull() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripShortPrimitiveInput() { List testData = Arrays.asList((short) 1, (short) 2, (short) 3); @@ -143,6 +146,7 @@ public void listSerdeShouldRoundtripShortPrimitiveInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitiveInput() { List testData = Arrays.asList((short) 1, (short) 2, (short) 3); @@ -151,6 +155,7 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitive listSerde.serializer().serialize(topic, testData).length); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripIntPrimitiveInput() { List testData = Arrays.asList(1, 2, 3); @@ -160,6 +165,7 @@ public void listSerdeShouldRoundtripIntPrimitiveInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveInput() { List testData = Arrays.asList(1, 2, 3); @@ -168,6 +174,7 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveIn listSerde.serializer().serialize(topic, testData).length); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripFloatPrimitiveInput() { List testData = Arrays.asList((float) 1, (float) 2, (float) 3); @@ -177,6 +184,7 @@ public void listSerdeShouldRoundtripFloatPrimitiveInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitiveInput() { List testData = Arrays.asList((float) 1, (float) 2, (float) 3); @@ -185,6 +193,7 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitive listSerde.serializer().serialize(topic, testData).length); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripLongPrimitiveInput() { List testData = Arrays.asList((long) 1, (long) 2, (long) 3); @@ -194,6 +203,7 @@ public void listSerdeShouldRoundtripLongPrimitiveInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveInput() { List testData = Arrays.asList((long) 1, (long) 2, (long) 3); @@ -202,6 +212,7 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveI listSerde.serializer().serialize(topic, testData).length); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripDoublePrimitiveInput() { List testData = Arrays.asList((double) 1, (double) 2, (double) 3); @@ -211,6 +222,7 @@ public void listSerdeShouldRoundtripDoublePrimitiveInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiveInput() { List testData = Arrays.asList((double) 1, (double) 2, (double) 3); @@ -219,6 +231,7 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiv listSerde.serializer().serialize(topic, testData).length); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripUUIDInput() { List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); @@ -228,6 +241,7 @@ public void listSerdeShouldRoundtripUUIDInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") @Test public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); @@ -236,8 +250,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { listSerde.serializer().serialize(topic, testData).length); } + @SuppressWarnings("unchecked") @Test - public void listSerdeShouldRountripNonPrimitiveInput() { + public void listSerdeShouldRoundtripNonPrimitiveInput() { List testData = Arrays.asList("A", "B", "C"); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); assertEquals("Should get the original collection after serialization and deserialization on an empty list", testData, @@ -245,6 +260,7 @@ public void listSerdeShouldRountripNonPrimitiveInput() { } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldReturnLinkedList() { List testData = new LinkedList<>(); @@ -253,6 +269,7 @@ public void listSerdeShouldReturnLinkedList() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) instanceof LinkedList); } + @SuppressWarnings("unchecked") @Test public void listSerdeShouldReturnStack() { List testData = new Stack<>(); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 540bb80e66116..708b3a7925ca4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -418,15 +418,6 @@ public class StreamsConfig extends AbstractConfig { + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; - - public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS = CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS; - - public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - - public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS; - - public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; - /** {@code default.timestamp.extractor} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "default.timestamp.extractor"; From 8edaacc083d53daf05601c817cff5a8a5d44cc6c Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Thu, 13 Feb 2020 15:27:59 -0500 Subject: [PATCH 32/49] Refactor configure methods for list (de)serializers --- .../serialization/ListDeserializer.java | 56 ++++++++++++++++--- .../common/serialization/ListSerializer.java | 25 +++++---- 2 files changed, 62 insertions(+), 19 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index ee97cf43ff2cb..ecc534b448466 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -58,24 +58,57 @@ public > ListDeserializer(Class listClass, Deserializer } } - @SuppressWarnings("unchecked") @Override public void configure(Map configs, boolean isKey) { if (listClass == null) { - String listTypePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; - listClass = (Class>) configs.get(listTypePropertyName); - if (listClass == null) { - throw new ConfigException("Not able to determine the list class because it was neither passed via the constructor nor set in the config"); - } + configureListClass(configs, isKey); } if (inner == null) { - String innerDeserializerPropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - Class> innerDeserializerClass = (Class>) configs.get(innerDeserializerPropertyName); - inner = Utils.newInstance(innerDeserializerClass); + configureInnerSerde(configs, isKey); + } + } + + private void configureListClass(Map configs, boolean isKey) { + String listTypePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; + final Object listClassOrName = configs.get(listTypePropertyName); + if (listClassOrName == null) { + throw new ConfigException("Not able to determine the list class because it was neither passed via the constructor nor set in the config."); + } + try { + if (listClassOrName instanceof String) { + listClass = Utils.loadClass((String) listClassOrName, Object.class); + } else if (listClassOrName instanceof Class) { + listClass = (Class) listClassOrName; + } else { + throw new KafkaException("Could not determine the list class instance using \"" + listTypePropertyName + "\" property."); + } + } catch (final ClassNotFoundException e) { + throw new ConfigException(listTypePropertyName, listClassOrName, "Deserializer's list class \"" + listClassOrName + "\" could not be found."); + } + } + + @SuppressWarnings("unchecked") + private void configureInnerSerde(Map configs, boolean isKey) { + String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final Object innerSerdeClassOrName = configs.get(innerSerdePropertyName); + if (innerSerdeClassOrName == null) { + throw new ConfigException("Not able to determine the inner serde class because it was neither passed via the constructor nor set in the config."); + } + try { + if (innerSerdeClassOrName instanceof String) { + inner = Utils.newInstance((String) innerSerdeClassOrName, Serde.class).deserializer(); + } else if (innerSerdeClassOrName instanceof Class) { + inner = (Deserializer) ((Serde) Utils.newInstance((Class) innerSerdeClassOrName)).deserializer(); + } else { + throw new KafkaException("Could not determine the inner serde class instance using \"" + innerSerdePropertyName + "\" property."); + } inner.configure(configs, isKey); + } catch (final ClassNotFoundException e) { + throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Deserializer's inner serde class \"" + innerSerdeClassOrName + "\" could not be found."); } } + @SuppressWarnings("unchecked") private List getListInstance(int listSize) { try { @@ -121,4 +154,9 @@ public void close() { } } + // Only for testing + Deserializer innerDeserializer() { + return inner; + } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index c84953d7a4526..d3210d1479fb0 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -52,22 +52,22 @@ public ListSerializer(Serializer serializer) { @Override public void configure(Map configs, boolean isKey) { if (inner == null) { - final String innerSerializerPropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - final Object innerSerializerClassOrName = configs.get(innerSerializerPropertyName); - if (innerSerializerClassOrName == null) { - throw new ConfigException("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config"); + final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final Object innerSerdeClassOrName = configs.get(innerSerdePropertyName); + if (innerSerdeClassOrName == null) { + throw new ConfigException("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); } try { - if (innerSerializerClassOrName instanceof String) { - inner = Utils.newInstance((String) innerSerializerClassOrName, Serializer.class); - } else if (innerSerializerClassOrName instanceof Class) { - inner = Utils.newInstance((Class>) innerSerializerClassOrName); + if (innerSerdeClassOrName instanceof String) { + inner = Utils.newInstance((String) innerSerdeClassOrName, Serde.class).serializer(); + } else if (innerSerdeClassOrName instanceof Class) { + inner = (Serializer) ((Serde) Utils.newInstance((Class) innerSerdeClassOrName)).serializer(); } else { - throw new KafkaException("Could not create a serializer class instance using \"" + innerSerializerPropertyName + "\" property."); + throw new KafkaException("Could not create a serializer class instance using \"" + innerSerdePropertyName + "\" property."); } inner.configure(configs, isKey); } catch (final ClassNotFoundException e) { - throw new ConfigException(innerSerializerPropertyName, innerSerializerClassOrName, "Serializer class " + innerSerializerClassOrName + " could not be found."); + throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Serializer class " + innerSerdeClassOrName + " could not be found."); } } } @@ -101,4 +101,9 @@ public void close() { } } + // Only for testing + Serializer innerSerializer() { + return inner; + } + } \ No newline at end of file From f6326f1744b31fea1f1a68fb424b6f2b2c36bec8 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Thu, 13 Feb 2020 15:28:36 -0500 Subject: [PATCH 33/49] Introduce list (de)serializers configuration tests Merge with trunk --- .../clients/producer/ProducerConfig.java | 2 +- .../serialization/ListDeserializerTest.java | 169 ++++++++++++++++++ .../serialization/ListSerializerTest.java | 132 ++++++++++++++ 3 files changed, 302 insertions(+), 1 deletion(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 9c50d29371887..4fd1bf09c4571 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -436,7 +436,7 @@ public class ProducerConfig extends AbstractConfig { Type.CLASS, null, Importance.MEDIUM, - LIST_KEY_SERIALIZER_INNER_CLASS_DOC); + LIST_KEY_SERIALIZER_INNER_CLASS_DOC) .defineInternal(AUTO_DOWNGRADE_TXN_COMMIT, Type.BOOLEAN, false, diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java new file mode 100644 index 0000000000000..e749843ee4200 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -0,0 +1,169 @@ +/* + * 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.kafka.common.serialization; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class ListDeserializerTest { + private final ListDeserializer listDeserializer = new ListDeserializer<>(); + private final Map props = new HashMap<>(); + private final String nonExistingClass = "non.existing.class"; + private static class FakeObject { + } + + @Test + public void testListKeyDeserializerNoArgConstructorsWithClassNames() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class.getName()); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); + listDeserializer.configure(props, true); + final Deserializer inner = listDeserializer.innerDeserializer(); + assertNotNull("Inner deserializer should be not null", inner); + assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); + } + + @Test + public void testListValueDeserializerNoArgConstructorsWithClassNames() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class.getName()); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.IntegerSerde.class.getName()); + listDeserializer.configure(props, false); + final Deserializer inner = listDeserializer.innerDeserializer(); + assertNotNull("Inner deserializer should be not null", inner); + assertTrue("Inner deserializer type should be IntegerDeserializer", inner instanceof IntegerDeserializer); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsWithClassObjects() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + listDeserializer.configure(props, true); + final Deserializer inner = listDeserializer.innerDeserializer(); + assertNotNull("Inner deserializer should be not null", inner); + assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); + } + + @Test + public void testListValueDeserializerNoArgConstructorsWithClassObjects() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + listDeserializer.configure(props, false); + final Deserializer inner = listDeserializer.innerDeserializer(); + assertNotNull("Inner deserializer should be not null", inner); + assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); + } + + + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingInnerClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Not able to determine the inner serde class because it was neither passed via the constructor nor set in the config.")); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingTypeClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Not able to determine the list class because it was neither passed via the constructor nor set in the config.")); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidTypeClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, new FakeObject()); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Could not determine the list class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + "\" property.")); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidInnerClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Could not determine the inner serde class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.")); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidTypeClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, new FakeObject()); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, false) + ); + assertThat(exception.getMessage(), is("Could not determine the list class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + "\" property.")); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidInnerClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, false) + ); + assertThat(exception.getMessage(), is("Could not determine the inner serde class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.")); + } + + @Test + public void testListDeserializerNoArgConstructorsShouldThrowConfigExceptionDueListClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, nonExistingClass); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Invalid value " + nonExistingClass + " for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + ": Deserializer's list class \"" + nonExistingClass + "\" could not be found.")); + } + + @Test + public void testListDeserializerNoArgConstructorsShouldThrowConfigExceptionDueInnerSerdeClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, nonExistingClass); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Invalid value " + nonExistingClass + " for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Deserializer's inner serde class \"" + nonExistingClass + "\" could not be found.")); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java new file mode 100644 index 0000000000000..a1b4a86256233 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.serialization; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class ListSerializerTest { + private final ListSerializer listSerializer = new ListSerializer<>(); + private final Map props = new HashMap<>(); + private final String nonExistingClass = "non.existing.class"; + private static class FakeObject { + } + + @Test + public void testListKeySerializerNoArgConstructorsWithClassName() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); + listSerializer.configure(props, true); + final Serializer inner = listSerializer.innerSerializer(); + assertNotNull("Inner serializer should be not null", inner); + assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + } + + @Test + public void testListValueSerializerNoArgConstructorsWithClassName() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); + listSerializer.configure(props, false); + final Serializer inner = listSerializer.innerSerializer(); + assertNotNull("Inner serializer should be not null", inner); + assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + } + + @Test + public void testListKeySerializerNoArgConstructorsWithClassObject() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + listSerializer.configure(props, true); + final Serializer inner = listSerializer.innerSerializer(); + assertNotNull("Inner serializer should be not null", inner); + assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + } + + @Test + public void testListValueSerializerNoArgConstructorsWithClassObject() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + listSerializer.configure(props, false); + final Serializer inner = listSerializer.innerSerializer(); + assertNotNull("Inner serializer should be not null", inner); + assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + } + + @Test + public void testListSerializerNoArgConstructorsShouldThrowConfigExceptionDueMissingProp() { + ConfigException exception = assertThrows( + ConfigException.class, + () -> listSerializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.")); + + exception = assertThrows( + ConfigException.class, + () -> listSerializer.configure(props, false) + ); + assertThat(exception.getMessage(), is("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.")); + + } + + @Test + public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.")); + } + + @Test + public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, false) + ); + assertThat(exception.getMessage(), is("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.")); + } + + @Test + public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, nonExistingClass); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, true) + ); + assertThat(exception.getMessage(), is("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.")); + } + + @Test + public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDueClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, nonExistingClass); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, false) + ); + assertThat(exception.getMessage(), is("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.")); + } + +} From dea18d03369acbe74881d4222e0e9bc6fd032fdc Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 13 Jul 2020 10:11:24 -0400 Subject: [PATCH 34/49] Refactor getters Add public key --- .../kafka/common/serialization/ListDeserializer.java | 10 +++++----- .../kafka/common/serialization/ListSerializer.java | 10 +++++----- .../common/serialization/ListDeserializerTest.java | 8 ++++---- .../kafka/common/serialization/ListSerializerTest.java | 8 ++++---- 4 files changed, 18 insertions(+), 18 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index ecc534b448466..622a204ee5b02 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -58,6 +58,11 @@ public > ListDeserializer(Class listClass, Deserializer } } + Deserializer getInnerDeserializer() { + public Deserializer getInnerDeserializer() { + return inner; + } + @Override public void configure(Map configs, boolean isKey) { if (listClass == null) { @@ -154,9 +159,4 @@ public void close() { } } - // Only for testing - Deserializer innerDeserializer() { - return inner; - } - } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index d3210d1479fb0..acad8b36cf6e1 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -48,6 +48,11 @@ public ListSerializer(Serializer serializer) { this.isFixedLength = serializer != null && fixedLengthSerializers.contains(serializer.getClass()); } + Serializer getInnerSerializer() { + public Serializer getInnerSerializer() { + return inner; + } + @SuppressWarnings("unchecked") @Override public void configure(Map configs, boolean isKey) { @@ -101,9 +106,4 @@ public void close() { } } - // Only for testing - Serializer innerSerializer() { - return inner; - } - } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java index e749843ee4200..d145af5fcb15e 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -43,7 +43,7 @@ public void testListKeyDeserializerNoArgConstructorsWithClassNames() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class.getName()); props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listDeserializer.configure(props, true); - final Deserializer inner = listDeserializer.innerDeserializer(); + final Deserializer inner = listDeserializer.getInnerDeserializer(); assertNotNull("Inner deserializer should be not null", inner); assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); } @@ -53,7 +53,7 @@ public void testListValueDeserializerNoArgConstructorsWithClassNames() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class.getName()); props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.IntegerSerde.class.getName()); listDeserializer.configure(props, false); - final Deserializer inner = listDeserializer.innerDeserializer(); + final Deserializer inner = listDeserializer.getInnerDeserializer(); assertNotNull("Inner deserializer should be not null", inner); assertTrue("Inner deserializer type should be IntegerDeserializer", inner instanceof IntegerDeserializer); } @@ -63,7 +63,7 @@ public void testListKeyDeserializerNoArgConstructorsWithClassObjects() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); listDeserializer.configure(props, true); - final Deserializer inner = listDeserializer.innerDeserializer(); + final Deserializer inner = listDeserializer.getInnerDeserializer(); assertNotNull("Inner deserializer should be not null", inner); assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); } @@ -73,7 +73,7 @@ public void testListValueDeserializerNoArgConstructorsWithClassObjects() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); listDeserializer.configure(props, false); - final Deserializer inner = listDeserializer.innerDeserializer(); + final Deserializer inner = listDeserializer.getInnerDeserializer(); assertNotNull("Inner deserializer should be not null", inner); assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java index a1b4a86256233..2f913dce2669a 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java @@ -41,7 +41,7 @@ private static class FakeObject { public void testListKeySerializerNoArgConstructorsWithClassName() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listSerializer.configure(props, true); - final Serializer inner = listSerializer.innerSerializer(); + final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull("Inner serializer should be not null", inner); assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); } @@ -50,7 +50,7 @@ public void testListKeySerializerNoArgConstructorsWithClassName() { public void testListValueSerializerNoArgConstructorsWithClassName() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listSerializer.configure(props, false); - final Serializer inner = listSerializer.innerSerializer(); + final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull("Inner serializer should be not null", inner); assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); } @@ -59,7 +59,7 @@ public void testListValueSerializerNoArgConstructorsWithClassName() { public void testListKeySerializerNoArgConstructorsWithClassObject() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); listSerializer.configure(props, true); - final Serializer inner = listSerializer.innerSerializer(); + final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull("Inner serializer should be not null", inner); assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); } @@ -68,7 +68,7 @@ public void testListKeySerializerNoArgConstructorsWithClassObject() { public void testListValueSerializerNoArgConstructorsWithClassObject() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); listSerializer.configure(props, false); - final Serializer inner = listSerializer.innerSerializer(); + final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull("Inner serializer should be not null", inner); assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); } From eb0db3ac6d7b3d5062b09febfd1c521b479fc73f Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 13 Jul 2020 15:45:26 -0400 Subject: [PATCH 35/49] Add null-index-list and negative-size serialization strategies functionality --- .../serialization/ListDeserializer.java | 46 ++++++++++++++++--- .../common/serialization/ListSerializer.java | 42 ++++++++++++++--- .../kafka/common/serialization/Serdes.java | 23 ++++++++++ 3 files changed, 98 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 622a204ee5b02..c077826c1715f 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -27,9 +27,11 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -58,7 +60,6 @@ public > ListDeserializer(Class listClass, Deserializer } } - Deserializer getInnerDeserializer() { public Deserializer getInnerDeserializer() { return inner; } @@ -131,20 +132,53 @@ private List getListInstance(int listSize) { } } + private SerializationStrategy parseSerializationStrategyFlag(final int serializationStrategyFlag) throws IOException { + if (serializationStrategyFlag < 0 || serializationStrategyFlag >= SerializationStrategy.VALUES.length) { + throw new SerializationException("Invalid serialization strategy flag value"); + } + return SerializationStrategy.VALUES[serializationStrategyFlag]; + } + + private List deserializeNullIndexList(final DataInputStream dis) throws IOException { + int nullIndexListSize = dis.readInt(); + List nullIndexList = new ArrayList<>(nullIndexListSize); + while (nullIndexListSize != 0) { + nullIndexList.add(dis.readInt()); + nullIndexListSize--; + } + return nullIndexList; + } + @Override public List deserialize(String topic, byte[] data) { if (data == null) { return null; } try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { + SerializationStrategy serStrategy = parseSerializationStrategyFlag(dis.readByte()); + List nullIndexList = null; + if (serStrategy == SerializationStrategy.NULL_INDEX_LIST) { + nullIndexList = deserializeNullIndexList(dis); + } final int size = dis.readInt(); List deserializedList = getListInstance(size); for (int i = 0; i < size; i++) { - byte[] payload = new byte[primitiveSize == null ? dis.readInt() : primitiveSize]; - if (dis.read(payload) == -1) { - throw new SerializationException("End of the stream was reached prematurely"); + if (serStrategy == SerializationStrategy.NULL_INDEX_LIST + && nullIndexList.contains(i)) { + deserializedList.add(null); + continue; + } + int entrySize = primitiveSize == null || serStrategy == SerializationStrategy.NEGATIVE_SIZE ? dis.readInt() : primitiveSize; + if (serStrategy == SerializationStrategy.NEGATIVE_SIZE && + entrySize == Serdes.ListSerde.NEGATIVE_SIZE_VALUE) { + deserializedList.add(null); + } else { + byte[] payload = new byte[entrySize]; + if (dis.read(payload) == -1) { + throw new SerializationException("End of the stream was reached prematurely"); + } + deserializedList.add(inner.deserialize(topic, payload)); } - deserializedList.add(inner.deserialize(topic, payload)); } return deserializedList; } catch (IOException e) { @@ -159,4 +193,4 @@ public void close() { } } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index acad8b36cf6e1..b24178cd74635 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -27,10 +27,15 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; public class ListSerializer implements Serializer> { private Serializer inner; + private SerializationStrategy serStrategy; private boolean isFixedLength; static private List>> fixedLengthSerializers = Arrays.asList( @@ -46,9 +51,14 @@ public ListSerializer() {} public ListSerializer(Serializer serializer) { this.inner = serializer; this.isFixedLength = serializer != null && fixedLengthSerializers.contains(serializer.getClass()); + this.serStrategy = this.isFixedLength ? SerializationStrategy.NULL_INDEX_LIST : SerializationStrategy.NEGATIVE_SIZE; + } + + public ListSerializer(Serializer serializer, SerializationStrategy serStrategy) { + this(serializer); + this.serStrategy = serStrategy; } - Serializer getInnerSerializer() { public Serializer getInnerSerializer() { return inner; } @@ -77,21 +87,39 @@ public void configure(Map configs, boolean isKey) { } } + private void serializeNullIndexList(final DataOutputStream out, List data) throws IOException { + List nullIndexList = IntStream.range(0, data.size()) + .filter(i -> data.get(i) == null) + .boxed().collect(Collectors.toList()); + out.writeInt(nullIndexList.size()); + for (int i : nullIndexList) out.writeInt(i); + } + @Override public byte[] serialize(String topic, List data) { if (data == null) { return null; } - final int size = data.size(); try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream out = new DataOutputStream(baos)) { + out.writeByte(serStrategy.ordinal()); // write serialization strategy flag + if (serStrategy == SerializationStrategy.NULL_INDEX_LIST) { + serializeNullIndexList(out, data); + } + final int size = data.size(); out.writeInt(size); for (Inner entry : data) { - final byte[] bytes = inner.serialize(topic, entry); - if (!isFixedLength) { - out.writeInt(bytes.length); + if (entry == null) { + if (serStrategy == SerializationStrategy.NEGATIVE_SIZE) { + out.writeInt(Serdes.ListSerde.NEGATIVE_SIZE_VALUE); + } + } else { + final byte[] bytes = inner.serialize(topic, entry); + if (!isFixedLength || serStrategy == SerializationStrategy.NEGATIVE_SIZE) { + out.writeInt(bytes.length); + } + out.write(bytes); } - out.write(bytes); } return baos.toByteArray(); } catch (IOException e) { @@ -106,4 +134,4 @@ public void close() { } } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index b0946c274767f..953452d718cbc 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -128,6 +128,15 @@ public UUIDSerde() { static public final class ListSerde extends WrapperSerde> { + final static int NEGATIVE_SIZE_VALUE = -1; + + enum SerializationStrategy { + NULL_INDEX_LIST, + NEGATIVE_SIZE; + + public static final SerializationStrategy[] VALUES = SerializationStrategy.values(); + } + public ListSerde() { super(new ListSerializer<>(), new ListDeserializer<>()); } @@ -135,6 +144,11 @@ public ListSerde() { public > ListSerde(Class listClass, Serde serde) { super(new ListSerializer<>(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); } + + public > ListSerde(Class listClass, Serde serde, SerializationStrategy serStrategy) { + super(new ListSerializer<>(serde.serializer(), serStrategy), new ListDeserializer<>(listClass, serde.deserializer())); + } + } @SuppressWarnings("unchecked") @@ -287,4 +301,13 @@ static public Serde Void() { return new ListSerde<>(listClass, innerSerde); } + /* + * A serde for {@code List} type + */ + public static , Inner> Serde> + ListSerde(Class listClass, Serde innerSerde, ListSerde.SerializationStrategy serStrategy) { + + return new ListSerde<>(listClass, innerSerde, serStrategy); + } + } From 44494eb975e4208f7cfbbc0854255ef085b3ac75 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 13 Jul 2020 15:45:52 -0400 Subject: [PATCH 36/49] Add test coverage for serialization strategies functionality --- .../serialization/SerializationTest.java | 47 +++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index acc236669bd47..51d383707d519 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -259,6 +259,53 @@ public void listSerdeShouldRoundtripNonPrimitiveInput() { listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripPrimitiveInputWithNullEntries() { + List testData = Arrays.asList(1, null, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals("Should get the original collection of integer primitives with null entries" + + " after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripNonPrimitiveInputWithNullEntries() { + List testData = Arrays.asList("A", null, "C"); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); + assertEquals("Should get the original collection of strings list with null entries " + + "after serialization and deserialization", testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripInputWithNullIndexListSerializationStrategy() { + List testData = Arrays.asList(1, null, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NULL_INDEX_LIST); + byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); + assertEquals("Should get length of 21 bytes (serialization flag + size of the null index list + 1 null index entry " + + "+ size of the input list + 2 integer entries) after serialization", 21, + listSerde.serializer().serialize(topic, testData).length); + assertEquals("Should get the original collection of integer primitives with null entries " + + "after serialization and deserialization with null index list serialization strategy", testData, + listSerde.deserializer().deserialize(topic, serializedPayload)); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripInputWithNegativeSizeSerializationStrategy() { + List testData = Arrays.asList(1, null, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NEGATIVE_SIZE); + byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); + assertEquals("Should get length of 25 bytes (serialization flag + size of the input list + 2 integer entries " + + "+ 1 negative size entry) after serialization", 25, + listSerde.serializer().serialize(topic, testData).length); + assertEquals("Should get the original collection of integer primitives with null entries " + + "after serialization and deserialization with negative size serialization strategy", testData, + listSerde.deserializer().deserialize(topic, serializedPayload)); + } @SuppressWarnings("unchecked") @Test From 569969e2f26ddb7bd7748efc37d380aea0ff916d Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 13 Jul 2020 15:46:05 -0400 Subject: [PATCH 37/49] Update existing test cases --- .../serialization/SerializationTest.java | 44 ++++++++++--------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 51d383707d519..29a1ee00cc56b 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -36,6 +36,7 @@ import java.util.Stack; import java.util.UUID; +import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsNull.nullValue; @@ -138,39 +139,39 @@ public void listSerdeShouldReturnNull() { @SuppressWarnings("unchecked") @Test - public void listSerdeShouldRoundtripShortPrimitiveInput() { - List testData = Arrays.asList((short) 1, (short) 2, (short) 3); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); - assertEquals("Should get the original collection of short primitives" + public void listSerdeShouldRoundtripIntPrimitiveInput() { + List testData = Arrays.asList(1, 2, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals("Should get the original collection of integer primitives" + " after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @SuppressWarnings("unchecked") @Test - public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitiveInput() { - List testData = Arrays.asList((short) 1, (short) 2, (short) 3); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); - assertEquals("Should get length of 10 bytes (size of the list + 3 short entries) after serialization", 10, + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveInput() { + List testData = Arrays.asList(1, 2, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals("Should get length of 21 bytes after serialization", 21, listSerde.serializer().serialize(topic, testData).length); } @SuppressWarnings("unchecked") @Test - public void listSerdeShouldRoundtripIntPrimitiveInput() { - List testData = Arrays.asList(1, 2, 3); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get the original collection of integer primitives" + public void listSerdeShouldRoundtripShortPrimitiveInput() { + List testData = Arrays.asList((short) 1, (short) 2, (short) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); + assertEquals("Should get the original collection of short primitives" + " after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } @SuppressWarnings("unchecked") @Test - public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveInput() { - List testData = Arrays.asList(1, 2, 3); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get length of 16 bytes (size of the list + 3 integer entries) after serialization", 16, + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitiveInput() { + List testData = Arrays.asList((short) 1, (short) 2, (short) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); + assertEquals("Should get length of 15 bytes after serialization", 15, listSerde.serializer().serialize(topic, testData).length); } @@ -189,7 +190,7 @@ public void listSerdeShouldRoundtripFloatPrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitiveInput() { List testData = Arrays.asList((float) 1, (float) 2, (float) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); - assertEquals("Should get length of 16 bytes (size of the list + 3 float entries) after serialization", 16, + assertEquals("Should get length of 21 bytes after serialization", 21, listSerde.serializer().serialize(topic, testData).length); } @@ -208,7 +209,7 @@ public void listSerdeShouldRoundtripLongPrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveInput() { List testData = Arrays.asList((long) 1, (long) 2, (long) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); - assertEquals("Should get length of 24 bytes (size of the list + 3 long entries) after serialization", 28, + assertEquals("Should get length of 33 bytes after serialization", 33, listSerde.serializer().serialize(topic, testData).length); } @@ -227,7 +228,7 @@ public void listSerdeShouldRoundtripDoublePrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiveInput() { List testData = Arrays.asList((double) 1, (double) 2, (double) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); - assertEquals("Should get length of 24 bytes (size of the list + 3 double entries) after serialization", 28, + assertEquals("Should get length of 33 bytes after serialization", 33, listSerde.serializer().serialize(topic, testData).length); } @@ -246,7 +247,7 @@ public void listSerdeShouldRoundtripUUIDInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); - assertEquals("Should get length of 52 bytes (size of the list + 3 UUID entries) after serialization", 112, + assertEquals("Should get length of 117 bytes after serialization", 117, listSerde.serializer().serialize(topic, testData).length); } @@ -255,7 +256,8 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { public void listSerdeShouldRoundtripNonPrimitiveInput() { List testData = Arrays.asList("A", "B", "C"); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); - assertEquals("Should get the original collection after serialization and deserialization on an empty list", testData, + assertEquals("Should get the original collection of strings list " + + "after serialization and deserialization", testData, listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); } From da88bb7292b1f358a39c847286dd43a70cdfadc3 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 20 Jan 2021 16:02:00 -0500 Subject: [PATCH 38/49] Migrate to Junit 5 Jupiter --- .../serialization/ListDeserializerTest.java | 54 ++++--- .../serialization/ListSerializerTest.java | 40 +++--- .../serialization/SerializationTest.java | 134 +++++++++--------- 3 files changed, 121 insertions(+), 107 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java index d145af5fcb15e..d008c9c062d0d 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -16,20 +16,20 @@ */ package org.apache.kafka.common.serialization; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigException; -import org.junit.Test; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; +import org.junit.jupiter.api.Test; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; public class ListDeserializerTest { private final ListDeserializer listDeserializer = new ListDeserializer<>(); @@ -44,8 +44,8 @@ public void testListKeyDeserializerNoArgConstructorsWithClassNames() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listDeserializer.configure(props, true); final Deserializer inner = listDeserializer.getInnerDeserializer(); - assertNotNull("Inner deserializer should be not null", inner); - assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); + assertNotNull(inner, "Inner deserializer should be not null"); + assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } @Test @@ -54,8 +54,8 @@ public void testListValueDeserializerNoArgConstructorsWithClassNames() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.IntegerSerde.class.getName()); listDeserializer.configure(props, false); final Deserializer inner = listDeserializer.getInnerDeserializer(); - assertNotNull("Inner deserializer should be not null", inner); - assertTrue("Inner deserializer type should be IntegerDeserializer", inner instanceof IntegerDeserializer); + assertNotNull(inner, "Inner deserializer should be not null"); + assertTrue(inner instanceof IntegerDeserializer, "Inner deserializer type should be IntegerDeserializer"); } @Test @@ -64,8 +64,8 @@ public void testListKeyDeserializerNoArgConstructorsWithClassObjects() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); listDeserializer.configure(props, true); final Deserializer inner = listDeserializer.getInnerDeserializer(); - assertNotNull("Inner deserializer should be not null", inner); - assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); + assertNotNull(inner, "Inner deserializer should be not null"); + assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } @Test @@ -74,8 +74,8 @@ public void testListValueDeserializerNoArgConstructorsWithClassObjects() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); listDeserializer.configure(props, false); final Deserializer inner = listDeserializer.getInnerDeserializer(); - assertNotNull("Inner deserializer should be not null", inner); - assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer); + assertNotNull(inner, "Inner deserializer should be not null"); + assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } @@ -87,7 +87,8 @@ public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDu ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Not able to determine the inner serde class because it was neither passed via the constructor nor set in the config.")); + assertEquals(exception.getMessage(), "Not able to determine the inner serde class because " + + "it was neither passed via the constructor nor set in the config."); } @Test @@ -97,7 +98,8 @@ public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDu ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Not able to determine the list class because it was neither passed via the constructor nor set in the config.")); + assertEquals(exception.getMessage(), "Not able to determine the list class because " + + "it was neither passed via the constructor nor set in the config."); } @Test @@ -108,7 +110,8 @@ public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listDeserializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Could not determine the list class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + "\" property.")); + assertEquals(exception.getMessage(), "Could not determine the list class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + "\" property."); } @Test @@ -119,7 +122,8 @@ public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listDeserializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Could not determine the inner serde class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.")); + assertEquals(exception.getMessage(), "Could not determine the inner serde class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property."); } @Test @@ -130,7 +134,8 @@ public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionD KafkaException.class, () -> listDeserializer.configure(props, false) ); - assertThat(exception.getMessage(), is("Could not determine the list class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + "\" property.")); + assertEquals(exception.getMessage(), "Could not determine the list class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + "\" property."); } @Test @@ -141,7 +146,8 @@ public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionD KafkaException.class, () -> listDeserializer.configure(props, false) ); - assertThat(exception.getMessage(), is("Could not determine the inner serde class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.")); + assertEquals(exception.getMessage(), "Could not determine the inner serde class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property."); } @Test @@ -152,7 +158,9 @@ public void testListDeserializerNoArgConstructorsShouldThrowConfigExceptionDueLi ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Invalid value " + nonExistingClass + " for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + ": Deserializer's list class \"" + nonExistingClass + "\" could not be found.")); + assertEquals(exception.getMessage(), "Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + ": Deserializer's list class " + + "\"" + nonExistingClass + "\" could not be found."); } @Test @@ -163,7 +171,9 @@ public void testListDeserializerNoArgConstructorsShouldThrowConfigExceptionDueIn ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Invalid value " + nonExistingClass + " for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Deserializer's inner serde class \"" + nonExistingClass + "\" could not be found.")); + assertEquals(exception.getMessage(), "Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Deserializer's inner serde class " + + "\"" + nonExistingClass + "\" could not be found."); } } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java index 2f913dce2669a..94aa7d2943c04 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java @@ -16,19 +16,19 @@ */ package org.apache.kafka.common.serialization; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; public class ListSerializerTest { private final ListSerializer listSerializer = new ListSerializer<>(); @@ -42,8 +42,8 @@ public void testListKeySerializerNoArgConstructorsWithClassName() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listSerializer.configure(props, true); final Serializer inner = listSerializer.getInnerSerializer(); - assertNotNull("Inner serializer should be not null", inner); - assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + assertNotNull(inner, "Inner serializer should be not null"); + assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); } @Test @@ -51,8 +51,8 @@ public void testListValueSerializerNoArgConstructorsWithClassName() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listSerializer.configure(props, false); final Serializer inner = listSerializer.getInnerSerializer(); - assertNotNull("Inner serializer should be not null", inner); - assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + assertNotNull(inner, "Inner serializer should be not null"); + assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); } @Test @@ -60,8 +60,8 @@ public void testListKeySerializerNoArgConstructorsWithClassObject() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); listSerializer.configure(props, true); final Serializer inner = listSerializer.getInnerSerializer(); - assertNotNull("Inner serializer should be not null", inner); - assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + assertNotNull(inner, "Inner serializer should be not null"); + assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); } @Test @@ -69,8 +69,8 @@ public void testListValueSerializerNoArgConstructorsWithClassObject() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); listSerializer.configure(props, false); final Serializer inner = listSerializer.getInnerSerializer(); - assertNotNull("Inner serializer should be not null", inner); - assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer); + assertNotNull(inner, "Inner serializer should be not null"); + assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); } @Test @@ -79,13 +79,13 @@ public void testListSerializerNoArgConstructorsShouldThrowConfigExceptionDueMiss ConfigException.class, () -> listSerializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.")); + assertEquals(exception.getMessage(), "Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); exception = assertThrows( ConfigException.class, () -> listSerializer.configure(props, false) ); - assertThat(exception.getMessage(), is("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.")); + assertEquals(exception.getMessage(), "Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); } @@ -96,7 +96,7 @@ public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueIn KafkaException.class, () -> listSerializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.")); + assertEquals(exception.getMessage(), "Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property."); } @Test @@ -106,7 +106,7 @@ public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listSerializer.configure(props, false) ); - assertThat(exception.getMessage(), is("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.")); + assertEquals(exception.getMessage(), "Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property."); } @Test @@ -116,7 +116,7 @@ public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueCl KafkaException.class, () -> listSerializer.configure(props, true) ); - assertThat(exception.getMessage(), is("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.")); + assertEquals(exception.getMessage(), "Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found."); } @Test @@ -126,7 +126,7 @@ public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listSerializer.configure(props, false) ); - assertThat(exception.getMessage(), is("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.")); + assertEquals(exception.getMessage(), "Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found."); } } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 29a1ee00cc56b..4f874e0545f8a 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -37,9 +37,6 @@ import java.util.UUID; import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.IsNull.nullValue; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -124,8 +121,9 @@ public void stringSerdeShouldSupportDifferentEncodings() { public void listSerdeShouldReturnEmptyCollection() { List testData = Arrays.asList(); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get empty collection after serialization and deserialization on an empty list", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get empty collection after serialization and deserialization on an empty list"); } @SuppressWarnings("unchecked") @@ -133,8 +131,9 @@ public void listSerdeShouldReturnEmptyCollection() { public void listSerdeShouldReturnNull() { List testData = null; Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get null after serialization and deserialization on an empty list", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get null after serialization and deserialization on an empty list"); } @SuppressWarnings("unchecked") @@ -142,9 +141,9 @@ public void listSerdeShouldReturnNull() { public void listSerdeShouldRoundtripIntPrimitiveInput() { List testData = Arrays.asList(1, 2, 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get the original collection of integer primitives" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of integer primitives after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -152,8 +151,8 @@ public void listSerdeShouldRoundtripIntPrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveInput() { List testData = Arrays.asList(1, 2, 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get length of 21 bytes after serialization", 21, - listSerde.serializer().serialize(topic, testData).length); + assertEquals(21, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 21 bytes after serialization"); } @SuppressWarnings("unchecked") @@ -161,9 +160,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveIn public void listSerdeShouldRoundtripShortPrimitiveInput() { List testData = Arrays.asList((short) 1, (short) 2, (short) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); - assertEquals("Should get the original collection of short primitives" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of short primitives after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -171,8 +170,8 @@ public void listSerdeShouldRoundtripShortPrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitiveInput() { List testData = Arrays.asList((short) 1, (short) 2, (short) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); - assertEquals("Should get length of 15 bytes after serialization", 15, - listSerde.serializer().serialize(topic, testData).length); + assertEquals(15, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 15 bytes after serialization"); } @SuppressWarnings("unchecked") @@ -180,9 +179,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitive public void listSerdeShouldRoundtripFloatPrimitiveInput() { List testData = Arrays.asList((float) 1, (float) 2, (float) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); - assertEquals("Should get the original collection of float primitives" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of float primitives after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -190,8 +189,8 @@ public void listSerdeShouldRoundtripFloatPrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitiveInput() { List testData = Arrays.asList((float) 1, (float) 2, (float) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); - assertEquals("Should get length of 21 bytes after serialization", 21, - listSerde.serializer().serialize(topic, testData).length); + assertEquals(21, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 21 bytes after serialization"); } @SuppressWarnings("unchecked") @@ -199,9 +198,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitive public void listSerdeShouldRoundtripLongPrimitiveInput() { List testData = Arrays.asList((long) 1, (long) 2, (long) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); - assertEquals("Should get the original collection of long primitives" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of long primitives after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -209,8 +208,8 @@ public void listSerdeShouldRoundtripLongPrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveInput() { List testData = Arrays.asList((long) 1, (long) 2, (long) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); - assertEquals("Should get length of 33 bytes after serialization", 33, - listSerde.serializer().serialize(topic, testData).length); + assertEquals(33, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 33 bytes after serialization"); } @SuppressWarnings("unchecked") @@ -218,9 +217,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveI public void listSerdeShouldRoundtripDoublePrimitiveInput() { List testData = Arrays.asList((double) 1, (double) 2, (double) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); - assertEquals("Should get the original collection of double primitives" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of double primitives after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -228,8 +227,8 @@ public void listSerdeShouldRoundtripDoublePrimitiveInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiveInput() { List testData = Arrays.asList((double) 1, (double) 2, (double) 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); - assertEquals("Should get length of 33 bytes after serialization", 33, - listSerde.serializer().serialize(topic, testData).length); + assertEquals(33, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 33 bytes after serialization"); } @SuppressWarnings("unchecked") @@ -237,9 +236,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiv public void listSerdeShouldRoundtripUUIDInput() { List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); - assertEquals("Should get the original collection of UUID" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of UUID after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -247,8 +246,8 @@ public void listSerdeShouldRoundtripUUIDInput() { public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); - assertEquals("Should get length of 117 bytes after serialization", 117, - listSerde.serializer().serialize(topic, testData).length); + assertEquals(117, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 117 bytes after serialization"); } @SuppressWarnings("unchecked") @@ -256,9 +255,9 @@ public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { public void listSerdeShouldRoundtripNonPrimitiveInput() { List testData = Arrays.asList("A", "B", "C"); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); - assertEquals("Should get the original collection of strings list " - + "after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of strings list after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -266,9 +265,10 @@ public void listSerdeShouldRoundtripNonPrimitiveInput() { public void listSerdeShouldRoundtripPrimitiveInputWithNullEntries() { List testData = Arrays.asList(1, null, 3); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); - assertEquals("Should get the original collection of integer primitives with null entries" - + " after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of integer primitives with null entries " + + "after serialization and deserialization"); } @SuppressWarnings("unchecked") @@ -276,37 +276,41 @@ public void listSerdeShouldRoundtripPrimitiveInputWithNullEntries() { public void listSerdeShouldRoundtripNonPrimitiveInputWithNullEntries() { List testData = Arrays.asList("A", null, "C"); Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); - assertEquals("Should get the original collection of strings list with null entries " - + "after serialization and deserialization", testData, - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData))); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of strings list with null entries " + + "after serialization and deserialization"); } @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripInputWithNullIndexListSerializationStrategy() { List testData = Arrays.asList(1, null, 3); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NULL_INDEX_LIST); + Serde> listSerde = Serdes + .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NULL_INDEX_LIST); byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); - assertEquals("Should get length of 21 bytes (serialization flag + size of the null index list + 1 null index entry " - + "+ size of the input list + 2 integer entries) after serialization", 21, - listSerde.serializer().serialize(topic, testData).length); - assertEquals("Should get the original collection of integer primitives with null entries " - + "after serialization and deserialization with null index list serialization strategy", testData, - listSerde.deserializer().deserialize(topic, serializedPayload)); + assertEquals(21, + listSerde.serializer().serialize(topic, testData).length, + "Should get length of 21 bytes (serialization flag + size of the null index list " + + "+ 1 null index entry + size of the input list + 2 integer entries) after serialization"); + assertEquals(testData, listSerde.deserializer().deserialize(topic, serializedPayload), + "Should get the original collection of integer primitives with null entries " + + "after serialization and deserialization with null index list serialization strategy"); } @SuppressWarnings("unchecked") @Test public void listSerdeShouldRoundtripInputWithNegativeSizeSerializationStrategy() { List testData = Arrays.asList(1, null, 3); - Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NEGATIVE_SIZE); + Serde> listSerde = Serdes + .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NEGATIVE_SIZE); byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); - assertEquals("Should get length of 25 bytes (serialization flag + size of the input list + 2 integer entries " - + "+ 1 negative size entry) after serialization", 25, - listSerde.serializer().serialize(topic, testData).length); - assertEquals("Should get the original collection of integer primitives with null entries " - + "after serialization and deserialization with negative size serialization strategy", testData, - listSerde.deserializer().deserialize(topic, serializedPayload)); + assertEquals(25, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 25 bytes (serialization flag + size of the input list + 2 integer entries " + + "+ 1 negative size entry) after serialization"); + assertEquals(testData, listSerde.deserializer().deserialize(topic, serializedPayload), + "Should get the original collection of integer primitives with null entries " + + "after serialization and deserialization with negative size serialization strategy"); } @SuppressWarnings("unchecked") @@ -314,8 +318,8 @@ public void listSerdeShouldRoundtripInputWithNegativeSizeSerializationStrategy() public void listSerdeShouldReturnLinkedList() { List testData = new LinkedList<>(); Serde> listSerde = Serdes.ListSerde(LinkedList.class, Serdes.Integer()); - assertTrue("Should return List instance of type LinkedList", - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) instanceof LinkedList); + assertTrue(listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) + instanceof LinkedList, "Should return List instance of type LinkedList"); } @SuppressWarnings("unchecked") @@ -323,11 +327,11 @@ public void listSerdeShouldReturnLinkedList() { public void listSerdeShouldReturnStack() { List testData = new Stack<>(); Serde> listSerde = Serdes.ListSerde(Stack.class, Serdes.Integer()); - assertTrue("Should return List instance of type Stack", - listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) instanceof Stack); + assertTrue(listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) + instanceof Stack, "Should return List instance of type Stack"); } - @Test(expected = SerializationException.class) + @Test public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() { try (Serde serde = Serdes.Float()) { assertThrows(SerializationException.class, () -> serde.deserializer().deserialize(topic, new byte[0])); From 614aa89ff712c2caa59c7baf30cc747c4219b371 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Fri, 12 Feb 2021 17:19:46 -0500 Subject: [PATCH 39/49] Fix merge conflict --- .../serialization/SerializationTest.java | 5 ----- .../apache/kafka/streams/StreamsConfig.java | 22 +------------------ 2 files changed, 1 insertion(+), 26 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 4f874e0545f8a..5ea3a879dade5 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -28,13 +28,8 @@ import java.util.Map; import java.util.UUID; import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; import java.util.LinkedList; -import java.util.List; -import java.util.Map; import java.util.Stack; -import java.util.UUID; import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 708b3a7925ca4..274322ee218d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -887,27 +887,7 @@ public class StreamsConfig extends AbstractConfig { Type.LONG, null, Importance.LOW, - WINDOW_SIZE_MS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC); + WINDOW_SIZE_MS_DOC); } // this is the list of configs for underlying clients From b982f369844b863e5cb72d312c15d56cf2027259 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 26 Apr 2021 12:34:13 -0400 Subject: [PATCH 40/49] Remove extraneous configurations --- .../clients/consumer/ConsumerConfig.java | 24 ------------------- .../clients/producer/ProducerConfig.java | 12 ---------- 2 files changed, 36 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 00f3bc41b6f95..72c3aee1408f6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -224,20 +224,6 @@ public class ConsumerConfig extends AbstractConfig { public static final String CHECK_CRCS_CONFIG = "check.crcs"; private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; - /** - * list.key.serializer.inner - */ - public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_CONFIG = "list.key.deserializer.inner"; - public static final String LIST_KEY_DESERIALIZER_INNER_CLASS_DOC = "Inner deserializer class for key that implements the java.util.List interface. " - + "This configuration will be read if and only if key.deserializer configuration is set to org.apache.kafka.common.serialization.ListDeserializer"; - - /** - * list.key.serializer.type - */ - public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG = "list.key.deserializer.type"; - public static final String LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC = "Deserializer's list class for key that implements the org.apache.kafka.common.serialization.Deserializer interface." - + "This configuration will be read if and only if key.deserializer configuration is set to org.apache.kafka.common.serialization.ListDeserializer"; - /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the org.apache.kafka.common.serialization.Deserializer interface."; @@ -569,16 +555,6 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) - .define(LIST_KEY_DESERIALIZER_TYPE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - LIST_KEY_DESERIALIZER_TYPE_CLASS_DOC) - .define(LIST_KEY_DESERIALIZER_INNER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - LIST_KEY_DESERIALIZER_INNER_CLASS_DOC) .withClientSslSupport() .withClientSaslSupport(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 4fd1bf09c4571..dcbb6d22199b2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -215,13 +215,6 @@ public class ProducerConfig extends AbstractConfig { + " prefer to leave this config unset and instead use " + DELIVERY_TIMEOUT_MS_CONFIG + " to control" + " retry behavior."; - /** - * list.key.serializer.inner - */ - public static final String LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG = "list.key.serializer.inner"; - public static final String LIST_KEY_SERIALIZER_INNER_CLASS_DOC = "Inner serializer class for key that implements the org.apache.kafka.common.serialization.Serializer interface." - + "This configuration will be read if and only if key.serializer configuration is set to org.apache.kafka.common.serialization.ListSerializer"; - /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the org.apache.kafka.common.serialization.Serializer interface."; @@ -432,11 +425,6 @@ public class ProducerConfig extends AbstractConfig { new ConfigDef.NonEmptyString(), Importance.LOW, TRANSACTIONAL_ID_DOC) - .define(LIST_KEY_SERIALIZER_INNER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - LIST_KEY_SERIALIZER_INNER_CLASS_DOC) .defineInternal(AUTO_DOWNGRADE_TXN_COMMIT, Type.BOOLEAN, false, From 9e02a9036fcc961fd3c355d188db021c04d8b796 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 26 Apr 2021 12:34:47 -0400 Subject: [PATCH 41/49] Rename constants --- .../kafka/common/serialization/ListDeserializer.java | 10 +++++----- .../kafka/common/serialization/ListSerializer.java | 10 +++++----- .../org/apache/kafka/common/serialization/Serdes.java | 6 +++--- .../kafka/common/serialization/SerializationTest.java | 4 ++-- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index c077826c1715f..d3b14365289c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -157,20 +157,20 @@ public List deserialize(String topic, byte[] data) { try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { SerializationStrategy serStrategy = parseSerializationStrategyFlag(dis.readByte()); List nullIndexList = null; - if (serStrategy == SerializationStrategy.NULL_INDEX_LIST) { + if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { nullIndexList = deserializeNullIndexList(dis); } final int size = dis.readInt(); List deserializedList = getListInstance(size); for (int i = 0; i < size; i++) { - if (serStrategy == SerializationStrategy.NULL_INDEX_LIST + if (serStrategy == SerializationStrategy.CONSTANT_SIZE && nullIndexList.contains(i)) { deserializedList.add(null); continue; } - int entrySize = primitiveSize == null || serStrategy == SerializationStrategy.NEGATIVE_SIZE ? dis.readInt() : primitiveSize; - if (serStrategy == SerializationStrategy.NEGATIVE_SIZE && - entrySize == Serdes.ListSerde.NEGATIVE_SIZE_VALUE) { + int entrySize = (primitiveSize == null || serStrategy == SerializationStrategy.VARIABLE_SIZE) ? dis.readInt() : primitiveSize; + if (serStrategy == SerializationStrategy.VARIABLE_SIZE && + entrySize == Serdes.ListSerde.NULL_ENTRY_VALUE) { deserializedList.add(null); } else { byte[] payload = new byte[entrySize]; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index b24178cd74635..bfff513b2b9cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -51,7 +51,7 @@ public ListSerializer() {} public ListSerializer(Serializer serializer) { this.inner = serializer; this.isFixedLength = serializer != null && fixedLengthSerializers.contains(serializer.getClass()); - this.serStrategy = this.isFixedLength ? SerializationStrategy.NULL_INDEX_LIST : SerializationStrategy.NEGATIVE_SIZE; + this.serStrategy = this.isFixedLength ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; } public ListSerializer(Serializer serializer, SerializationStrategy serStrategy) { @@ -103,19 +103,19 @@ public byte[] serialize(String topic, List data) { try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream out = new DataOutputStream(baos)) { out.writeByte(serStrategy.ordinal()); // write serialization strategy flag - if (serStrategy == SerializationStrategy.NULL_INDEX_LIST) { + if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { serializeNullIndexList(out, data); } final int size = data.size(); out.writeInt(size); for (Inner entry : data) { if (entry == null) { - if (serStrategy == SerializationStrategy.NEGATIVE_SIZE) { - out.writeInt(Serdes.ListSerde.NEGATIVE_SIZE_VALUE); + if (serStrategy == SerializationStrategy.VARIABLE_SIZE) { + out.writeInt(Serdes.ListSerde.NULL_ENTRY_VALUE); } } else { final byte[] bytes = inner.serialize(topic, entry); - if (!isFixedLength || serStrategy == SerializationStrategy.NEGATIVE_SIZE) { + if (!isFixedLength || serStrategy == SerializationStrategy.VARIABLE_SIZE) { out.writeInt(bytes.length); } out.write(bytes); diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 953452d718cbc..dc246cf046b04 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -128,11 +128,11 @@ public UUIDSerde() { static public final class ListSerde extends WrapperSerde> { - final static int NEGATIVE_SIZE_VALUE = -1; + final static int NULL_ENTRY_VALUE = -1; enum SerializationStrategy { - NULL_INDEX_LIST, - NEGATIVE_SIZE; + CONSTANT_SIZE, + VARIABLE_SIZE; public static final SerializationStrategy[] VALUES = SerializationStrategy.values(); } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 5ea3a879dade5..67adfc910d7e3 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -282,7 +282,7 @@ public void listSerdeShouldRoundtripNonPrimitiveInputWithNullEntries() { public void listSerdeShouldRoundtripInputWithNullIndexListSerializationStrategy() { List testData = Arrays.asList(1, null, 3); Serde> listSerde = Serdes - .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NULL_INDEX_LIST); + .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.CONSTANT_SIZE); byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); assertEquals(21, listSerde.serializer().serialize(topic, testData).length, @@ -298,7 +298,7 @@ public void listSerdeShouldRoundtripInputWithNullIndexListSerializationStrategy( public void listSerdeShouldRoundtripInputWithNegativeSizeSerializationStrategy() { List testData = Arrays.asList(1, null, 3); Serde> listSerde = Serdes - .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.NEGATIVE_SIZE); + .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.VARIABLE_SIZE); byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); assertEquals(25, listSerde.serializer().serialize(topic, testData).length, "Should get length of 25 bytes (serialization flag + size of the input list + 2 integer entries " From 15eb0d81ea0ce1c45ff6b70f5c57629302d3b16c Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 26 Apr 2021 12:35:03 -0400 Subject: [PATCH 42/49] Use primitives' BYTES constants --- .../kafka/common/serialization/ListDeserializer.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index d3b14365289c6..0aa6c085e7f9c 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -41,12 +41,12 @@ public class ListDeserializer implements Deserializer> { private Class listClass; private Integer primitiveSize; - static private Map>, Integer> fixedLengthDeserializers = mkMap( - mkEntry(ShortDeserializer.class, 2), - mkEntry(IntegerDeserializer.class, 4), - mkEntry(FloatDeserializer.class, 4), - mkEntry(LongDeserializer.class, 8), - mkEntry(DoubleDeserializer.class, 8), + private static Map>, Integer> fixedLengthDeserializers = mkMap( + mkEntry(ShortDeserializer.class, Short.BYTES), + mkEntry(IntegerDeserializer.class, Integer.BYTES), + mkEntry(FloatDeserializer.class, Float.BYTES), + mkEntry(LongDeserializer.class, Long.BYTES), + mkEntry(DoubleDeserializer.class, Double.BYTES), mkEntry(UUIDDeserializer.class, 36) ); From cdbbbd5675773a5f0053a80cc377e49df28cacde Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Mon, 26 Apr 2021 13:27:31 -0400 Subject: [PATCH 43/49] Validate passed constructor parameters --- .../kafka/common/serialization/ListDeserializer.java | 9 +++++---- .../kafka/common/serialization/ListSerializer.java | 5 ++++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 0aa6c085e7f9c..5d6bb7cd816d1 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -53,11 +53,12 @@ public class ListDeserializer implements Deserializer> { public ListDeserializer() {} public > ListDeserializer(Class listClass, Deserializer innerDeserializer) { + if (listClass == null || innerDeserializer == null) { + throw new IllegalArgumentException("ListDeserializer requires both \"listClass\" and \"innerDeserializer\" parameters to be provided during initialization"); + } this.listClass = listClass; this.inner = innerDeserializer; - if (innerDeserializer != null) { - this.primitiveSize = fixedLengthDeserializers.get(innerDeserializer.getClass()); - } + this.primitiveSize = fixedLengthDeserializers.get(innerDeserializer.getClass()); } public Deserializer getInnerDeserializer() { @@ -89,7 +90,7 @@ private void configureListClass(Map configs, boolean isKey) { throw new KafkaException("Could not determine the list class instance using \"" + listTypePropertyName + "\" property."); } } catch (final ClassNotFoundException e) { - throw new ConfigException(listTypePropertyName, listClassOrName, "Deserializer's list class \"" + listClassOrName + "\" could not be found."); + throw new ConfigException(listTypePropertyName, listClassOrName, "Deserializer's list class \"" + listClassOrName + "\" was not a valid Serde/Deserializer."); } } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index bfff513b2b9cb..4ea4321ab7def 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -49,8 +49,11 @@ public class ListSerializer implements Serializer> { public ListSerializer() {} public ListSerializer(Serializer serializer) { + if (serializer == null) { + throw new IllegalArgumentException("ListSerializer requires \"serializer\" parameter to be provided during initialization"); + } this.inner = serializer; - this.isFixedLength = serializer != null && fixedLengthSerializers.contains(serializer.getClass()); + this.isFixedLength = fixedLengthSerializers.contains(serializer.getClass()); this.serStrategy = this.isFixedLength ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; } From e8a5e5967d9ce727b3e74a6a1e4803bf8a67906f Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 27 Apr 2021 13:11:09 -0400 Subject: [PATCH 44/49] Throw when trying to configure initialized list Throw an exception if a user tries to configure already initialized list (de)serializer --- .../serialization/ListDeserializer.java | 11 +++--- .../common/serialization/ListSerializer.java | 35 ++++++++++--------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 5d6bb7cd816d1..74a4babb39c75 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -67,12 +67,11 @@ public Deserializer getInnerDeserializer() { @Override public void configure(Map configs, boolean isKey) { - if (listClass == null) { - configureListClass(configs, isKey); - } - if (inner == null) { - configureInnerSerde(configs, isKey); + if (listClass != null || inner != null) { + throw new ConfigException("List deserializer was already initialized using a non-default constructor"); } + configureListClass(configs, isKey); + configureInnerSerde(configs, isKey); } private void configureListClass(Map configs, boolean isKey) { @@ -90,7 +89,7 @@ private void configureListClass(Map configs, boolean isKey) { throw new KafkaException("Could not determine the list class instance using \"" + listTypePropertyName + "\" property."); } } catch (final ClassNotFoundException e) { - throw new ConfigException(listTypePropertyName, listClassOrName, "Deserializer's list class \"" + listClassOrName + "\" was not a valid Serde/Deserializer."); + throw new ConfigException(listTypePropertyName, listClassOrName, "Deserializer's list class \"" + listClassOrName + "\" could not be found."); } } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 4ea4321ab7def..1c30b0b5efd4e 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -69,24 +69,25 @@ public Serializer getInnerSerializer() { @SuppressWarnings("unchecked") @Override public void configure(Map configs, boolean isKey) { - if (inner == null) { - final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; - final Object innerSerdeClassOrName = configs.get(innerSerdePropertyName); - if (innerSerdeClassOrName == null) { - throw new ConfigException("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); - } - try { - if (innerSerdeClassOrName instanceof String) { - inner = Utils.newInstance((String) innerSerdeClassOrName, Serde.class).serializer(); - } else if (innerSerdeClassOrName instanceof Class) { - inner = (Serializer) ((Serde) Utils.newInstance((Class) innerSerdeClassOrName)).serializer(); - } else { - throw new KafkaException("Could not create a serializer class instance using \"" + innerSerdePropertyName + "\" property."); - } - inner.configure(configs, isKey); - } catch (final ClassNotFoundException e) { - throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Serializer class " + innerSerdeClassOrName + " could not be found."); + if (inner != null) { + throw new ConfigException("List serializer was already initialized using a non-default constructor"); + } + final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final Object innerSerdeClassOrName = configs.get(innerSerdePropertyName); + if (innerSerdeClassOrName == null) { + throw new ConfigException("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); + } + try { + if (innerSerdeClassOrName instanceof String) { + inner = Utils.newInstance((String) innerSerdeClassOrName, Serde.class).serializer(); + } else if (innerSerdeClassOrName instanceof Class) { + inner = (Serializer) ((Serde) Utils.newInstance((Class) innerSerdeClassOrName)).serializer(); + } else { + throw new KafkaException("Could not create a serializer class instance using \"" + innerSerdePropertyName + "\" property."); } + inner.configure(configs, isKey); + } catch (final ClassNotFoundException e) { + throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Serializer class " + innerSerdeClassOrName + " could not be found."); } } From 819e186d52ffa39de327c14949217f5b72822b51 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 27 Apr 2021 13:15:29 -0400 Subject: [PATCH 45/49] Refactor list (de)serializer tests --- .../serialization/ListDeserializerTest.java | 132 ++++++++++++++---- .../serialization/ListSerializerTest.java | 35 ++++- 2 files changed, 130 insertions(+), 37 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java index d008c9c062d0d..8556718c4faa4 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -30,7 +30,7 @@ import java.util.Map; import org.junit.jupiter.api.Test; - +@SuppressWarnings("unchecked") public class ListDeserializerTest { private final ListDeserializer listDeserializer = new ListDeserializer<>(); private final Map props = new HashMap<>(); @@ -78,8 +78,6 @@ public void testListValueDeserializerNoArgConstructorsWithClassObjects() { assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } - - @Test public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingInnerClassProp() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); @@ -87,19 +85,41 @@ public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDu ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Not able to determine the inner serde class because " - + "it was neither passed via the constructor nor set in the config."); + assertEquals("Not able to determine the inner serde class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingInnerClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Not able to determine the inner serde class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); } @Test public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingTypeClassProp() { - props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); final ConfigException exception = assertThrows( ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Not able to determine the list class because " - + "it was neither passed via the constructor nor set in the config."); + assertEquals("Not able to determine the list class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingTypeClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Not able to determine the list class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); } @Test @@ -110,32 +130,32 @@ public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listDeserializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Could not determine the list class instance using " - + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + "\" property."); + assertEquals("Could not determine the list class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + "\" property.", exception.getMessage()); } @Test - public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidInnerClass() { - props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); - props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, new FakeObject()); + public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidTypeClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, new FakeObject()); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); final KafkaException exception = assertThrows( KafkaException.class, - () -> listDeserializer.configure(props, true) + () -> listDeserializer.configure(props, false) ); - assertEquals(exception.getMessage(), "Could not determine the inner serde class instance using " - + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property."); + assertEquals("Could not determine the list class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + "\" property.", exception.getMessage()); } @Test - public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidTypeClass() { - props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, new FakeObject()); - props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidInnerClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, new FakeObject()); final KafkaException exception = assertThrows( KafkaException.class, - () -> listDeserializer.configure(props, false) + () -> listDeserializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Could not determine the list class instance using " - + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + "\" property."); + assertEquals("Could not determine the inner serde class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); } @Test @@ -146,34 +166,86 @@ public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionD KafkaException.class, () -> listDeserializer.configure(props, false) ); - assertEquals(exception.getMessage(), "Could not determine the inner serde class instance using " - + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property."); + assertEquals("Could not determine the inner serde class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); } @Test - public void testListDeserializerNoArgConstructorsShouldThrowConfigExceptionDueListClassNotFound() { + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueListClassNotFound() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, nonExistingClass); props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); final ConfigException exception = assertThrows( ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Invalid value " + nonExistingClass + " for configuration " + assertEquals("Invalid value " + nonExistingClass + " for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + ": Deserializer's list class " - + "\"" + nonExistingClass + "\" could not be found."); + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); } @Test - public void testListDeserializerNoArgConstructorsShouldThrowConfigExceptionDueInnerSerdeClassNotFound() { + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueListClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, nonExistingClass); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + ": Deserializer's list class " + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueInnerSerdeClassNotFound() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, nonExistingClass); final ConfigException exception = assertThrows( ConfigException.class, () -> listDeserializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Invalid value " + nonExistingClass + " for configuration " + assertEquals("Invalid value " + nonExistingClass + " for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Deserializer's inner serde class " - + "\"" + nonExistingClass + "\" could not be found."); + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueInnerSerdeClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, nonExistingClass); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Deserializer's inner serde class " + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListDeserializer initializedListDeserializer = new ListDeserializer<>(ArrayList.class, + Serdes.Integer().deserializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListDeserializer.configure(props, true) + ); + assertEquals("List deserializer was already initialized using a non-default constructor", exception.getMessage()); + } + + @Test + public void testListValueDeserializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListDeserializer initializedListDeserializer = new ListDeserializer<>(ArrayList.class, + Serdes.Integer().deserializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListDeserializer.configure(props, true) + ); + assertEquals("List deserializer was already initialized using a non-default constructor", exception.getMessage()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java index 94aa7d2943c04..a8ab191cad203 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java @@ -79,14 +79,13 @@ public void testListSerializerNoArgConstructorsShouldThrowConfigExceptionDueMiss ConfigException.class, () -> listSerializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); + assertEquals("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.", exception.getMessage()); exception = assertThrows( ConfigException.class, () -> listSerializer.configure(props, false) ); - assertEquals(exception.getMessage(), "Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); - + assertEquals("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.", exception.getMessage()); } @Test @@ -96,7 +95,7 @@ public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueIn KafkaException.class, () -> listSerializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property."); + assertEquals("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); } @Test @@ -106,7 +105,7 @@ public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listSerializer.configure(props, false) ); - assertEquals(exception.getMessage(), "Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property."); + assertEquals("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); } @Test @@ -116,7 +115,7 @@ public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueCl KafkaException.class, () -> listSerializer.configure(props, true) ); - assertEquals(exception.getMessage(), "Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found."); + assertEquals("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.", exception.getMessage()); } @Test @@ -126,7 +125,29 @@ public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDue KafkaException.class, () -> listSerializer.configure(props, false) ); - assertEquals(exception.getMessage(), "Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found."); + assertEquals("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.", exception.getMessage()); + } + + @Test + public void testListKeySerializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListSerializer initializedListSerializer = new ListSerializer<>(Serdes.Integer().serializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListSerializer.configure(props, true) + ); + assertEquals("List serializer was already initialized using a non-default constructor", exception.getMessage()); + } + + @Test + public void testListValueSerializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListSerializer initializedListSerializer = new ListSerializer<>(Serdes.Integer().serializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListSerializer.configure(props, false) + ); + assertEquals("List serializer was already initialized using a non-default constructor", exception.getMessage()); } } From f58bfce7342d7e41c89c6a29cdf1f94693b002da Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 4 May 2021 16:46:49 -0400 Subject: [PATCH 46/49] Review refactoring --- .../serialization/ListDeserializer.java | 19 +++++------ .../common/serialization/ListSerializer.java | 25 +++++++-------- .../kafka/common/serialization/Serdes.java | 15 +-------- .../serialization/SerializationTest.java | 32 ------------------- 4 files changed, 22 insertions(+), 69 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 74a4babb39c75..1102da62a11c7 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -37,11 +37,7 @@ public class ListDeserializer implements Deserializer> { - private Deserializer inner; - private Class listClass; - private Integer primitiveSize; - - private static Map>, Integer> fixedLengthDeserializers = mkMap( + private static final Map>, Integer> FIXED_LENGTH_DESERIALIZERS = mkMap( mkEntry(ShortDeserializer.class, Short.BYTES), mkEntry(IntegerDeserializer.class, Integer.BYTES), mkEntry(FloatDeserializer.class, Float.BYTES), @@ -50,15 +46,19 @@ public class ListDeserializer implements Deserializer> { mkEntry(UUIDDeserializer.class, 36) ); + private Deserializer inner; + private Class listClass; + private Integer primitiveSize; + public ListDeserializer() {} - public > ListDeserializer(Class listClass, Deserializer innerDeserializer) { - if (listClass == null || innerDeserializer == null) { + public > ListDeserializer(Class listClass, Deserializer inner) { + if (listClass == null || inner == null) { throw new IllegalArgumentException("ListDeserializer requires both \"listClass\" and \"innerDeserializer\" parameters to be provided during initialization"); } this.listClass = listClass; - this.inner = innerDeserializer; - this.primitiveSize = fixedLengthDeserializers.get(innerDeserializer.getClass()); + this.inner = inner; + this.primitiveSize = FIXED_LENGTH_DESERIALIZERS.get(inner.getClass()); } public Deserializer getInnerDeserializer() { @@ -109,6 +109,7 @@ private void configureInnerSerde(Map configs, boolean isKey) { throw new KafkaException("Could not determine the inner serde class instance using \"" + innerSerdePropertyName + "\" property."); } inner.configure(configs, isKey); + primitiveSize = FIXED_LENGTH_DESERIALIZERS.get(inner.getClass()); } catch (final ClassNotFoundException e) { throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Deserializer's inner serde class \"" + innerSerdeClassOrName + "\" could not be found."); } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index 1c30b0b5efd4e..a01c85212bb52 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -34,11 +34,7 @@ public class ListSerializer implements Serializer> { - private Serializer inner; - private SerializationStrategy serStrategy; - private boolean isFixedLength; - - static private List>> fixedLengthSerializers = Arrays.asList( + private static final List>> FIXED_LENGTH_SERIALIZERS = Arrays.asList( ShortSerializer.class, IntegerSerializer.class, FloatSerializer.class, @@ -46,22 +42,21 @@ public class ListSerializer implements Serializer> { DoubleSerializer.class, UUIDSerializer.class); + private Serializer inner; + private SerializationStrategy serStrategy; + private boolean isFixedLength; + public ListSerializer() {} - public ListSerializer(Serializer serializer) { - if (serializer == null) { + public ListSerializer(Serializer inner) { + if (inner == null) { throw new IllegalArgumentException("ListSerializer requires \"serializer\" parameter to be provided during initialization"); } - this.inner = serializer; - this.isFixedLength = fixedLengthSerializers.contains(serializer.getClass()); + this.inner = inner; + this.isFixedLength = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()); this.serStrategy = this.isFixedLength ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; } - public ListSerializer(Serializer serializer, SerializationStrategy serStrategy) { - this(serializer); - this.serStrategy = serStrategy; - } - public Serializer getInnerSerializer() { return inner; } @@ -86,6 +81,8 @@ public void configure(Map configs, boolean isKey) { throw new KafkaException("Could not create a serializer class instance using \"" + innerSerdePropertyName + "\" property."); } inner.configure(configs, isKey); + isFixedLength = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()); + serStrategy = this.isFixedLength ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; } catch (final ClassNotFoundException e) { throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Serializer class " + innerSerdeClassOrName + " could not be found."); } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index dc246cf046b04..84a7d2f547e9e 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -145,10 +145,6 @@ public > ListSerde(Class listClass, Serde serde) super(new ListSerializer<>(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); } - public > ListSerde(Class listClass, Serde serde, SerializationStrategy serStrategy) { - super(new ListSerializer<>(serde.serializer(), serStrategy), new ListDeserializer<>(listClass, serde.deserializer())); - } - } @SuppressWarnings("unchecked") @@ -295,19 +291,10 @@ static public Serde Void() { /* * A serde for {@code List} type */ - public static , Inner> Serde> + static public , Inner> Serde> ListSerde(Class listClass, Serde innerSerde) { return new ListSerde<>(listClass, innerSerde); } - /* - * A serde for {@code List} type - */ - public static , Inner> Serde> - ListSerde(Class listClass, Serde innerSerde, ListSerde.SerializationStrategy serStrategy) { - - return new ListSerde<>(listClass, innerSerde, serStrategy); - } - } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 67adfc910d7e3..85c09dd17ae09 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -31,7 +31,6 @@ import java.util.LinkedList; import java.util.Stack; -import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -277,37 +276,6 @@ public void listSerdeShouldRoundtripNonPrimitiveInputWithNullEntries() { + "after serialization and deserialization"); } - @SuppressWarnings("unchecked") - @Test - public void listSerdeShouldRoundtripInputWithNullIndexListSerializationStrategy() { - List testData = Arrays.asList(1, null, 3); - Serde> listSerde = Serdes - .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.CONSTANT_SIZE); - byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); - assertEquals(21, - listSerde.serializer().serialize(topic, testData).length, - "Should get length of 21 bytes (serialization flag + size of the null index list " - + "+ 1 null index entry + size of the input list + 2 integer entries) after serialization"); - assertEquals(testData, listSerde.deserializer().deserialize(topic, serializedPayload), - "Should get the original collection of integer primitives with null entries " - + "after serialization and deserialization with null index list serialization strategy"); - } - - @SuppressWarnings("unchecked") - @Test - public void listSerdeShouldRoundtripInputWithNegativeSizeSerializationStrategy() { - List testData = Arrays.asList(1, null, 3); - Serde> listSerde = Serdes - .ListSerde(ArrayList.class, Serdes.Integer(), SerializationStrategy.VARIABLE_SIZE); - byte[] serializedPayload = listSerde.serializer().serialize(topic, testData); - assertEquals(25, listSerde.serializer().serialize(topic, testData).length, - "Should get length of 25 bytes (serialization flag + size of the input list + 2 integer entries " - + "+ 1 negative size entry) after serialization"); - assertEquals(testData, listSerde.deserializer().deserialize(topic, serializedPayload), - "Should get the original collection of integer primitives with null entries " - + "after serialization and deserialization with negative size serialization strategy"); - } - @SuppressWarnings("unchecked") @Test public void listSerdeShouldReturnLinkedList() { From b4989fe2075406973e4015d665613cfd479edfd5 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Tue, 11 May 2021 17:18:51 -0400 Subject: [PATCH 47/49] Review changes --- .../serialization/ListDeserializer.java | 54 ++++++++++--------- .../common/serialization/ListSerializer.java | 28 +++++----- .../kafka/common/serialization/Serdes.java | 4 +- .../serialization/ListDeserializerTest.java | 8 +-- 4 files changed, 49 insertions(+), 45 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java index 1102da62a11c7..38bd111d9496c 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.common.serialization; -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.utils.Utils; +import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import java.io.ByteArrayInputStream; import java.io.DataInputStream; @@ -30,10 +28,12 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; - -import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serdes.ListSerde; +import org.apache.kafka.common.utils.Utils; public class ListDeserializer implements Deserializer> { @@ -61,7 +61,7 @@ public > ListDeserializer(Class listClass, Deserializer this.primitiveSize = FIXED_LENGTH_DESERIALIZERS.get(inner.getClass()); } - public Deserializer getInnerDeserializer() { + public Deserializer innerDeserializer() { return inner; } @@ -117,7 +117,7 @@ private void configureInnerSerde(Map configs, boolean isKey) { @SuppressWarnings("unchecked") - private List getListInstance(int listSize) { + private List createListInstance(int listSize) { try { Constructor> listConstructor; try { @@ -159,27 +159,29 @@ public List deserialize(String topic, byte[] data) { SerializationStrategy serStrategy = parseSerializationStrategyFlag(dis.readByte()); List nullIndexList = null; if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { + // In CONSTANT_SIZE strategy, indexes of null entries are decoded from a null index list nullIndexList = deserializeNullIndexList(dis); } final int size = dis.readInt(); - List deserializedList = getListInstance(size); + List deserializedList = createListInstance(size); for (int i = 0; i < size; i++) { - if (serStrategy == SerializationStrategy.CONSTANT_SIZE - && nullIndexList.contains(i)) { - deserializedList.add(null); - continue; - } - int entrySize = (primitiveSize == null || serStrategy == SerializationStrategy.VARIABLE_SIZE) ? dis.readInt() : primitiveSize; - if (serStrategy == SerializationStrategy.VARIABLE_SIZE && - entrySize == Serdes.ListSerde.NULL_ENTRY_VALUE) { - deserializedList.add(null); - } else { - byte[] payload = new byte[entrySize]; - if (dis.read(payload) == -1) { - throw new SerializationException("End of the stream was reached prematurely"); + int entrySize = -1; + if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { + if (nullIndexList.contains(i)) { + deserializedList.add(null); } - deserializedList.add(inner.deserialize(topic, payload)); + entrySize = primitiveSize; + } else if (serStrategy == SerializationStrategy.VARIABLE_SIZE) { + entrySize = dis.readInt(); + if (entrySize == ListSerde.NULL_ENTRY_VALUE) { + deserializedList.add(null); + } + } + byte[] payload = new byte[entrySize]; + if (dis.read(payload) == -1) { + throw new SerializationException("End of the stream was reached prematurely"); } + deserializedList.add(inner.deserialize(topic, payload)); } return deserializedList; } catch (IOException e) { diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java index a01c85212bb52..6274c9d3f7b53 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.serialization; +import java.util.ArrayList; +import java.util.Iterator; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigException; @@ -27,8 +29,6 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; @@ -44,7 +44,6 @@ public class ListSerializer implements Serializer> { private Serializer inner; private SerializationStrategy serStrategy; - private boolean isFixedLength; public ListSerializer() {} @@ -53,8 +52,7 @@ public ListSerializer(Serializer inner) { throw new IllegalArgumentException("ListSerializer requires \"serializer\" parameter to be provided during initialization"); } this.inner = inner; - this.isFixedLength = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()); - this.serStrategy = this.isFixedLength ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; + this.serStrategy = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()) ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; } public Serializer getInnerSerializer() { @@ -81,19 +79,24 @@ public void configure(Map configs, boolean isKey) { throw new KafkaException("Could not create a serializer class instance using \"" + innerSerdePropertyName + "\" property."); } inner.configure(configs, isKey); - isFixedLength = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()); - serStrategy = this.isFixedLength ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; + serStrategy = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()) ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; } catch (final ClassNotFoundException e) { throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Serializer class " + innerSerdeClassOrName + " could not be found."); } } private void serializeNullIndexList(final DataOutputStream out, List data) throws IOException { - List nullIndexList = IntStream.range(0, data.size()) - .filter(i -> data.get(i) == null) - .boxed().collect(Collectors.toList()); + int i = 0; + List nullIndexList = new ArrayList<>(); + for (Iterator it = data.listIterator(); it.hasNext(); i++) { + if (it.next() == null) { + nullIndexList.add(i); + } + } out.writeInt(nullIndexList.size()); - for (int i : nullIndexList) out.writeInt(i); + for (int nullIndex : nullIndexList) { + out.writeInt(nullIndex); + } } @Override @@ -105,6 +108,7 @@ public byte[] serialize(String topic, List data) { final DataOutputStream out = new DataOutputStream(baos)) { out.writeByte(serStrategy.ordinal()); // write serialization strategy flag if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { + // In CONSTANT_SIZE strategy, indexes of null entries are encoded in a null index list serializeNullIndexList(out, data); } final int size = data.size(); @@ -116,7 +120,7 @@ public byte[] serialize(String topic, List data) { } } else { final byte[] bytes = inner.serialize(topic, entry); - if (!isFixedLength || serStrategy == SerializationStrategy.VARIABLE_SIZE) { + if (serStrategy == SerializationStrategy.VARIABLE_SIZE) { out.writeInt(bytes.length); } out.write(bytes); diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 84a7d2f547e9e..4a150e0c022ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -291,9 +291,7 @@ static public Serde Void() { /* * A serde for {@code List} type */ - static public , Inner> Serde> - ListSerde(Class listClass, Serde innerSerde) { - + static public , Inner> Serde> ListSerde(Class listClass, Serde innerSerde) { return new ListSerde<>(listClass, innerSerde); } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java index 8556718c4faa4..aff01e3fe8a89 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -43,7 +43,7 @@ public void testListKeyDeserializerNoArgConstructorsWithClassNames() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class.getName()); props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); listDeserializer.configure(props, true); - final Deserializer inner = listDeserializer.getInnerDeserializer(); + final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } @@ -53,7 +53,7 @@ public void testListValueDeserializerNoArgConstructorsWithClassNames() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class.getName()); props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.IntegerSerde.class.getName()); listDeserializer.configure(props, false); - final Deserializer inner = listDeserializer.getInnerDeserializer(); + final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); assertTrue(inner instanceof IntegerDeserializer, "Inner deserializer type should be IntegerDeserializer"); } @@ -63,7 +63,7 @@ public void testListKeyDeserializerNoArgConstructorsWithClassObjects() { props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); listDeserializer.configure(props, true); - final Deserializer inner = listDeserializer.getInnerDeserializer(); + final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } @@ -73,7 +73,7 @@ public void testListValueDeserializerNoArgConstructorsWithClassObjects() { props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); listDeserializer.configure(props, false); - final Deserializer inner = listDeserializer.getInnerDeserializer(); + final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); } From e7c7789a1498a2041b807fa5560e225d7e615769 Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Wed, 12 May 2021 14:48:10 -0400 Subject: [PATCH 48/49] Add documentation changes --- docs/streams/developer-guide/datatypes.html | 4 +++- docs/streams/upgrade-guide.html | 6 ++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html index e6930bc31ce83..2201b5b69d35a 100644 --- a/docs/streams/developer-guide/datatypes.html +++ b/docs/streams/developer-guide/datatypes.html @@ -140,10 +140,12 @@

Primitive and basic typesUUID Serdes.UUID() - Void Serdes.Void() + List + Serdes.ListSerde() +
diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index de747bb862f33..57389bdc615d0 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -1104,6 +1104,12 @@

  • JoinWindows has no default size anymore: JoinWindows.of("name").within(1000) changes to JoinWindows.of(1000)
  • +

    New serde type ListSerde:

    +
      +
    • added class ListSerde to (de)serialize List-based objects
    • +
    • introduced ListSerializer and ListDeserializer to power the new functionality
    • +
    +