chia7712 commented on code in PR #19869:
URL: https://github.com/apache/kafka/pull/19869#discussion_r2195299248
##########
connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java:
##########
@@ -20,59 +20,55 @@
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;
import java.util.Objects;
+import java.util.stream.Collectors;
public class ConnectSchema implements Schema {
/**
* Maps {@link Schema.Type}s to a list of Java classes that can be used to
represent them.
*/
- private static final Map<Type, List<Class<?>>> SCHEMA_TYPE_CLASSES = new
EnumMap<>(Type.class);
+ private static final Map<Type, List<Class<?>>> SCHEMA_TYPE_CLASSES =
Collections.unmodifiableMap(new EnumMap<>(Map.ofEntries(
+ Map.entry(Type.INT8, List.of(Byte.class)),
+ Map.entry(Type.INT16, List.of(Short.class)),
+ Map.entry(Type.INT32, List.of(Integer.class)),
+ Map.entry(Type.INT64, List.of(Long.class)),
+ Map.entry(Type.FLOAT32, List.of(Float.class)),
+ Map.entry(Type.FLOAT64, List.of(Double.class)),
+ Map.entry(Type.BOOLEAN, List.of(Boolean.class)),
+ Map.entry(Type.STRING, List.of(String.class)),
+ // Bytes are special and have 2 representations. byte[] causes
problems because it doesn't handle equals() and
+ // hashCode() like we want objects to, so we support both byte[] and
ByteBuffer. Using plain byte[] can cause
+ // those methods to fail, so ByteBuffers are recommended
+ Map.entry(Type.BYTES, List.of(byte[].class, ByteBuffer.class)),
+ Map.entry(Type.ARRAY, List.of(List.class)),
+ Map.entry(Type.MAP, List.of(Map.class)),
+ Map.entry(Type.STRUCT, List.of(Struct.class))
+ )));
/**
* Maps known logical types to a list of Java classes that can be used to
represent them.
*/
- private static final Map<String, List<Class<?>>> LOGICAL_TYPE_CLASSES =
new HashMap<>();
+ // We don't need to put these into JAVA_CLASS_SCHEMA_TYPES since that's
only used to determine schemas for
+ // schemaless data and logical types will have ambiguous schemas (e.g.
many of them use the same Java class) so
+ // they should not be used without schemas.
+ private static final Map<String, List<Class<?>>> LOGICAL_TYPE_CLASSES =
Map.of(
+ Decimal.LOGICAL_NAME, List.of(BigDecimal.class),
+ Date.LOGICAL_NAME, List.of(java.util.Date.class),
+ Time.LOGICAL_NAME, List.of(java.util.Date.class),
+ Timestamp.LOGICAL_NAME, List.of(java.util.Date.class)
+ );
/**
* Maps the Java classes to the corresponding {@link Schema.Type}.
*/
- private static final Map<Class<?>, Type> JAVA_CLASS_SCHEMA_TYPES = new
HashMap<>();
-
- static {
- SCHEMA_TYPE_CLASSES.put(Type.INT8,
Collections.singletonList(Byte.class));
- SCHEMA_TYPE_CLASSES.put(Type.INT16,
Collections.singletonList(Short.class));
- SCHEMA_TYPE_CLASSES.put(Type.INT32,
Collections.singletonList(Integer.class));
- SCHEMA_TYPE_CLASSES.put(Type.INT64,
Collections.singletonList(Long.class));
- SCHEMA_TYPE_CLASSES.put(Type.FLOAT32,
Collections.singletonList(Float.class));
- SCHEMA_TYPE_CLASSES.put(Type.FLOAT64,
Collections.singletonList(Double.class));
- SCHEMA_TYPE_CLASSES.put(Type.BOOLEAN,
Collections.singletonList(Boolean.class));
- SCHEMA_TYPE_CLASSES.put(Type.STRING,
Collections.singletonList(String.class));
- // Bytes are special and have 2 representations. byte[] causes
problems because it doesn't handle equals() and
- // hashCode() like we want objects to, so we support both byte[] and
ByteBuffer. Using plain byte[] can cause
- // those methods to fail, so ByteBuffers are recommended
- SCHEMA_TYPE_CLASSES.put(Type.BYTES, Arrays.asList(byte[].class,
ByteBuffer.class));
- SCHEMA_TYPE_CLASSES.put(Type.ARRAY,
Collections.singletonList(List.class));
- SCHEMA_TYPE_CLASSES.put(Type.MAP,
Collections.singletonList(Map.class));
- SCHEMA_TYPE_CLASSES.put(Type.STRUCT,
Collections.singletonList(Struct.class));
-
- for (Map.Entry<Type, List<Class<?>>> schemaClasses :
SCHEMA_TYPE_CLASSES.entrySet()) {
- for (Class<?> schemaClass : schemaClasses.getValue())
- JAVA_CLASS_SCHEMA_TYPES.put(schemaClass,
schemaClasses.getKey());
- }
-
- LOGICAL_TYPE_CLASSES.put(Decimal.LOGICAL_NAME,
Collections.singletonList(BigDecimal.class));
- LOGICAL_TYPE_CLASSES.put(Date.LOGICAL_NAME,
Collections.singletonList(java.util.Date.class));
- LOGICAL_TYPE_CLASSES.put(Time.LOGICAL_NAME,
Collections.singletonList(java.util.Date.class));
- LOGICAL_TYPE_CLASSES.put(Timestamp.LOGICAL_NAME,
Collections.singletonList(java.util.Date.class));
- // We don't need to put these into JAVA_CLASS_SCHEMA_TYPES since
that's only used to determine schemas for
- // schemaless data and logical types will have ambiguous schemas (e.g.
many of them use the same Java class) so
- // they should not be used without schemas.
- }
+ private static final Map<Class<?>, Type> JAVA_CLASS_SCHEMA_TYPES = new
HashMap<>(SCHEMA_TYPE_CLASSES.entrySet()
Review Comment:
`toMap` returns the mutable map, so `new HashMap` is unnecessary.
##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java:
##########
@@ -48,14 +48,14 @@ public void
initializeInternalResources(Map<SourceAndTarget, Herder> herders) {
@Override
protected Collection<Class<?>> regularResources() {
- return Collections.singletonList(
+ return List.of(
Review Comment:
```java
return List.of(InternalMirrorResource.class);
```
##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java:
##########
@@ -215,7 +213,7 @@ public Map<String, String> workerConfig(SourceAndTarget
sourceAndTarget) {
Set<String> allConfigNames() {
Set<String> allNames = new HashSet<>();
- List<ConfigDef> connectorConfigDefs = Arrays.asList(
+ List<ConfigDef> connectorConfigDefs = List.of(
Review Comment:
Perhaps we don't need to create this temporary list.
```java
Set<String> allNames = new HashSet<>();
allNames.addAll(MirrorCheckpointConfig.CONNECTOR_CONFIG_DEF.names());
allNames.addAll(MirrorSourceConfig.CONNECTOR_CONFIG_DEF.names());
allNames.addAll(MirrorHeartbeatConfig.CONNECTOR_CONFIG_DEF.names());
return allNames;
```
or
```java
return Stream.of(
MirrorCheckpointConfig.CONNECTOR_CONFIG_DEF.names(),
MirrorSourceConfig.CONNECTOR_CONFIG_DEF.names(),
MirrorHeartbeatConfig.CONNECTOR_CONFIG_DEF.names()
)
.flatMap(Set::stream)
.collect(Collectors.toSet());
```
##########
connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java:
##########
@@ -211,7 +210,7 @@ public void testIdentityReplicationTopicSource() {
private ReplicationPolicy identityReplicationPolicy(String source) {
IdentityReplicationPolicy policy = new IdentityReplicationPolicy();
- policy.configure(Collections.singletonMap(
+ policy.configure(Map.of(
Review Comment:
```java
policy.configure(Map.of(IdentityReplicationPolicy.SOURCE_CLUSTER_ALIAS_CONFIG,
source));
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]