From 2bc780b959cad59b51d26562bd2f801432e05e2a Mon Sep 17 00:00:00 2001 From: Koen De Groote Date: Fri, 29 Dec 2017 13:39:18 +0100 Subject: [PATCH] MINOR: Use EnumMap/EnumSet if possible (#3919) They are more efficient than HashMap/HashSet. Reviewers: Ismael Juma --- .../org/apache/kafka/connect/data/ConnectSchema.java | 3 ++- .../org/apache/kafka/connect/json/JsonConverter.java | 3 ++- .../java/org/apache/kafka/connect/transforms/Cast.java | 9 ++++----- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index 5b0579ee30c..f90e6357455 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -22,6 +22,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; +import java.util.EnumMap; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -31,7 +32,7 @@ public class ConnectSchema implements Schema { /** * Maps Schema.Types to a list of Java classes that can be used to represent them. */ - private static final Map> SCHEMA_TYPE_CLASSES = new HashMap<>(); + private static final Map> SCHEMA_TYPE_CLASSES = new EnumMap<>(Type.class); /** * Maps known logical types to a list of Java classes that can be used to represent them. */ diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index a7b37f11832..01b0abb0209 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -42,6 +42,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; +import java.util.EnumMap; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -55,7 +56,7 @@ public class JsonConverter implements Converter { private static final String SCHEMAS_CACHE_SIZE_CONFIG = "schemas.cache.size"; private static final int SCHEMAS_CACHE_SIZE_DEFAULT = 1000; - private static final HashMap TO_CONNECT_CONVERTERS = new HashMap<>(); + private static final Map TO_CONNECT_CONVERTERS = new EnumMap<>(Schema.Type.class); static { TO_CONNECT_CONVERTERS.put(Schema.Type.BOOLEAN, new JsonToConnectTypeConverter() { diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java index 17be48ce174..d94f8f648b3 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -32,9 +32,8 @@ import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.transforms.util.SchemaUtil; import org.apache.kafka.connect.transforms.util.SimpleConfig; -import java.util.Arrays; +import java.util.EnumSet; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; @@ -79,9 +78,9 @@ public abstract class Cast> implements Transformation private static final String PURPOSE = "cast types"; - private static final Set SUPPORTED_CAST_TYPES = new HashSet<>( - Arrays.asList(Schema.Type.INT8, Schema.Type.INT16, Schema.Type.INT32, Schema.Type.INT64, - Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN, Schema.Type.STRING) + private static final Set SUPPORTED_CAST_TYPES = EnumSet.of( + Schema.Type.INT8, Schema.Type.INT16, Schema.Type.INT32, Schema.Type.INT64, + Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN, Schema.Type.STRING ); // As a special case for casting the entire value (e.g. the incoming key is a int64 but you know it could be an