Skip to content

Commit

Permalink
Simplify switch case code (#514)
Browse files Browse the repository at this point in the history
* Simplify case when switch code

* Simplify case when switch code

* Simplify case when switch code
  • Loading branch information
ismailsimsek authored Feb 22, 2025
1 parent cb3f0c3 commit 7291e9e
Show file tree
Hide file tree
Showing 3 changed files with 32 additions and 51 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -187,34 +187,28 @@ private GenericRecord convert(Types.StructType tableFields, JsonNode data) {

private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) {
LOGGER.debug("Processing Field:{} Type:{}", field.name(), field.type());
final Object val;

if (node.isNull()) {
return null;
}

switch (field.type().typeId()) {
case INTEGER: // int 4 bytes
val = node.isNull() ? null : node.asInt();
break;
return node.asInt();
case LONG: // long 8 bytes
val = node.isNull() ? null : node.asLong();
break;
return node.asLong();
case FLOAT: // float is represented in 32 bits,
val = node.isNull() ? null : node.floatValue();
break;
return node.floatValue();
case DOUBLE: // double is represented in 64 bits
val = node.isNull() ? null : node.asDouble();
break;
return node.asDouble();
case BOOLEAN:
val = node.isNull() ? null : node.asBoolean();
break;
case STRING:
// if the node is not a value node (method isValueNode returns false), convert it to string.
val = node.isValueNode() ? node.asText(null) : node.toString();
break;
return node.asBoolean();
case UUID:
val = node.isValueNode() ? UUID.fromString(node.asText(null)) : UUID.fromString(node.toString());
break;
case DATE:
if (node.isNull()) {
return null;
if (node.isTextual()) {
return UUID.fromString(node.textValue());
}
throw new RuntimeException("Failed to convert date value, field: " + field.name() + " value: " + node);
case DATE:
if ((node.isInt())) {
// io.debezium.time.Date
// org.apache.kafka.connect.data.Date
Expand All @@ -228,9 +222,6 @@ private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) {
}
throw new RuntimeException("Failed to convert date value, field: " + field.name() + " value: " + node);
case TIMESTAMP:
if (node.isNull()) {
return null;
}
if (node.isNumber() && TS_MS_FIELDS.contains(field.name())) {
return timestamptzFromMillis(node.asLong());
}
Expand All @@ -241,11 +232,10 @@ private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) {
return convertLocalDateTimeValue(field, node, config.temporalPrecisionMode());
case BINARY:
try {
val = node.isNull() ? null : ByteBuffer.wrap(node.binaryValue());
return ByteBuffer.wrap(node.binaryValue());
} catch (IOException e) {
throw new RuntimeException("Failed to convert binary value to iceberg value, field: " + field.name(), e);
}
break;
case LIST:
Types.NestedField listItemsType = field.type().asListType().fields().get(0);
// recursive value mapping when list elements are nested type
Expand All @@ -254,18 +244,14 @@ private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) {
node.elements().forEachRemaining(element -> {
listVal.add(jsonValToIcebergVal(field.type().asListType().fields().get(0), element));
});
val = listVal;
break;
return listVal;
}

val = mapper.convertValue(node, ArrayList.class);
break;
return mapper.convertValue(node, ArrayList.class);
case MAP:
Type keyType = field.type().asMapType().keyType();
Type valType = field.type().asMapType().valueType();
if (keyType.isPrimitiveType() && valType.isPrimitiveType()) {
val = mapper.convertValue(node, Map.class);
break;
return mapper.convertValue(node, Map.class);
}
// convert complex/nested map value with recursion
HashMap<Object, Object> mapVal = new HashMap<>();
Expand All @@ -276,54 +262,50 @@ private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) {
mapVal.put(f.getKey(), f.getValue());
}
});
val = mapVal;
break;
return mapVal;
case STRUCT:
// create it as struct, nested type
// recursive call to get nested data/record
val = convert(field.type().asStructType(), node);
break;
return convert(field.type().asStructType(), node);
case STRING:
default:
// default to String type
// if the node is not a value node (method isValueNode returns false), convert it to string.
val = node.isValueNode() ? node.asText(null) : node.toString();
break;
return node.isValueNode() ? node.textValue() : node.toString();
}

return val;
}

public LocalDateTime convertLocalDateTimeValue(Types.NestedField field, JsonNode node, TemporalPrecisionMode temporalPrecisionMode) {

final String eexMessage = "Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode;
if (node.isNumber()) {
return switch (temporalPrecisionMode) {
case MICROSECONDS -> DateTimeUtil.timestampFromMicros(node.asLong());
case NANOSECONDS -> DateTimeUtil.timestampFromNanos(node.asLong());
case CONNECT -> timestampFromMillis(node.asLong());
default ->
throw new RuntimeException("Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode);
default -> throw new RuntimeException(eexMessage);
};
}

if (node.isTextual()) {
return switch (temporalPrecisionMode) {
case ISOSTRING -> LocalDateTime.parse(node.asText(), IsoTimestamp.FORMATTER);
default ->
throw new RuntimeException("Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode);
default -> throw new RuntimeException(eexMessage);
};
}
throw new RuntimeException("Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode);
throw new RuntimeException(eexMessage);
}

private OffsetDateTime convertOffsetDateTimeValue(Types.NestedField field, JsonNode node, TemporalPrecisionMode temporalPrecisionMode) {
final String eexMessage = "Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode;

if (node.isNumber()) {
// non Timezone
return switch (temporalPrecisionMode) {
case MICROSECONDS -> DateTimeUtil.timestamptzFromMicros(node.asLong());
case NANOSECONDS -> timestamptzFromNanos(node.asLong());
case CONNECT -> timestamptzFromMillis(node.asLong());
default ->
throw new RuntimeException("Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode);
default -> throw new RuntimeException(eexMessage);
};
}

Expand All @@ -333,7 +315,6 @@ private OffsetDateTime convertOffsetDateTimeValue(Types.NestedField field, JsonN
};
}

throw new RuntimeException("Failed to convert timestamp value, field: " + field.name() + " value: " + node + " temporalPrecisionMode: " + temporalPrecisionMode);
throw new RuntimeException(eexMessage);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,7 @@ private Type.PrimitiveType icebergPrimitiveField(String fieldName, String fieldT
return Types.BooleanType.get();
case "string":
return switch (fieldTypeName) {
case "io.debezium.data.Uuid" -> Types.UUIDType.get();
case "io.debezium.time.IsoDate" -> Types.DateType.get();
case "io.debezium.time.IsoTimestamp" -> Types.TimestampType.withoutZone();
case "io.debezium.time.ZonedTimestamp" -> Types.TimestampType.withZone();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,8 +166,7 @@ public void testNestedGeomJsonRecord() {
GenericRecord h = (GenericRecord) record.getField("h");
assertEquals("AQEAAAAAAAAAAADwPwAAAAAAAPA/", g.get(0, Types.StringType.get().typeId().javaClass()));
assertEquals(123, g.get(1, Types.IntegerType.get().typeId().javaClass()));
assertEquals("Record(null, null)", h.toString());
assertNull(h.get(0, Types.BinaryType.get().typeId().javaClass()));
assertNull(h);
}

@Test
Expand Down

0 comments on commit 7291e9e

Please sign in to comment.