seokyun-ha-toss commented on code in PR #15283:
URL: https://github.com/apache/iceberg/pull/15283#discussion_r3055379581
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java:
##########
@@ -464,6 +479,218 @@ protected Temporal convertTimestampValue(Object value,
TimestampType type) {
return convertLocalDateTime(value);
}
+ protected Variant convertVariantValue(Object value) {
+ if (value instanceof ByteBuffer) {
+ return Variant.from((ByteBuffer) value);
+ }
+
+ Set<String> fieldNames = Sets.newHashSet();
+ collectFieldNames(value, fieldNames);
+ List<String> allFieldNames =
fieldNames.stream().sorted().collect(Collectors.toList());
+ VariantMetadata metadata = Variants.metadata(allFieldNames);
+ VariantValue variantValue = objectToVariantValue(value, metadata, null);
+ return Variant.of(metadata, variantValue);
+ }
+
+ /**
+ * Collects all field names (map keys) from the entire object tree into the
given set. Used to
+ * build a single VariantMetadata for the whole Variant (required for nested
maps).
+ */
+ private static void collectFieldNames(Object value, Set<String> names) {
+ if (value == null) {
+ return;
+ }
+ if (value instanceof Collection) {
+ for (Object element : (Collection<?>) value) {
+ collectFieldNames(element, names);
+ }
+ return;
+ }
+ if (value instanceof Map) {
+ Map<?, ?> map = (Map<?, ?>) value;
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ Object key = entry.getKey();
+ if (key != null && key instanceof String) {
+ names.add((String) key);
+ collectFieldNames(entry.getValue(), names);
+ }
+ }
+ return;
+ }
+ if (value instanceof Struct) {
+ Struct struct = (Struct) value;
+ for (Field field : struct.schema().fields()) {
+ names.add(field.name());
+ collectFieldNames(struct.get(field), names);
+ }
+ }
+ }
+
+ /**
+ * Recursively converts a Java object to a VariantValue using the given
shared metadata for all
+ * nested maps. Handles primitives, List (array), and Map (object); map keys
become field names.
+ */
+ private static VariantValue objectToVariantValue(
+ Object value, VariantMetadata metadata,
org.apache.kafka.connect.data.Schema schema) {
+ if (value == null) {
+ return Variants.ofNull();
+ }
+ VariantValue primitive = primitiveToVariantValue(value, schema);
+ if (primitive != null) {
+ return primitive;
+ }
+ if (value instanceof Collection) {
+ ValueArray array = Variants.array();
+ org.apache.kafka.connect.data.Schema elementSchema =
+ schema != null ? schema.valueSchema() : null;
+ for (Object element : (Collection<?>) value) {
+ array.add(objectToVariantValue(element, metadata, elementSchema));
+ }
+ return array;
+ }
+ if (value instanceof Map) {
+ return mapToVariantValue(value, metadata, schema);
+ }
+ if (value instanceof Struct) {
+ Struct struct = (Struct) value;
+ ShreddedObject object = Variants.object(metadata);
+ for (Field field : struct.schema().fields()) {
+ object.put(field.name(), objectToVariantValue(struct.get(field),
metadata, field.schema()));
+ }
+ return object;
+ }
+ throw new IllegalArgumentException("Cannot convert to variant: " +
value.getClass().getName());
+ }
+
+ private static VariantValue mapToVariantValue(
+ Object value, VariantMetadata metadata,
org.apache.kafka.connect.data.Schema schema) {
+ Map<?, ?> map = (Map<?, ?>) value;
+ ShreddedObject object = Variants.object(metadata);
+ org.apache.kafka.connect.data.Schema mapValueSchema =
+ schema != null ? schema.valueSchema() : null;
+ map.forEach(
+ (key, val) -> {
+ if (key != null && key instanceof String) {
+ object.put((String) key, objectToVariantValue(val, metadata,
mapValueSchema));
+ } else {
+ throw new IllegalArgumentException(
+ "Cannot convert map to variant: keys must be non-null strings,
was: "
+ + (key == null ? "null" : key.getClass().getName()));
+ }
+ });
+ return object;
+ }
+
+ /**
+ * Converts a primitive or primitive-like value to VariantValue; returns
null if not supported.
+ * The optional schema is used to disambiguate java.util.Date which Kafka
Connect uses for Date,
+ * Time, and Timestamp logical types.
+ */
+ private static VariantValue primitiveToVariantValue(
+ Object value, org.apache.kafka.connect.data.Schema schema) {
+ if (value instanceof Boolean) {
+ return Variants.of((Boolean) value);
+ }
+ VariantValue temporal = temporalObjectToVariantValue(value, schema);
+ if (temporal != null) {
+ return temporal;
+ }
+ if (value instanceof Number) {
+ return numberToVariantValue((Number) value);
+ }
+ if (value instanceof String) {
+ return Variants.of((String) value);
+ }
+ if (value instanceof ByteBuffer) {
+ return Variants.of((ByteBuffer) value);
+ }
+ if (value instanceof byte[]) {
+ return Variants.of(ByteBuffer.wrap((byte[]) value));
+ }
+ if (value instanceof UUID) {
+ return Variants.ofUUID((UUID) value);
+ }
+ return null;
+ }
+
+ /**
+ * Converts java.time values and java.util.Date (with Connect logical type
from the optional
+ * schema) to VariantValue; returns null if the value is not a supported
temporal representation.
+ */
+ private static VariantValue temporalObjectToVariantValue(
+ Object value, org.apache.kafka.connect.data.Schema schema) {
+ if (value instanceof Instant) {
+ return Variants.ofTimestamptz(DateTimeUtil.microsFromInstant((Instant)
value));
+ }
+ if (value instanceof OffsetDateTime) {
+ return
Variants.ofTimestamptz(DateTimeUtil.microsFromTimestamptz((OffsetDateTime)
value));
+ }
+ if (value instanceof ZonedDateTime) {
+ return Variants.ofTimestamptz(
+ DateTimeUtil.microsFromTimestamptz(((ZonedDateTime)
value).toOffsetDateTime()));
+ }
+ if (value instanceof LocalDateTime) {
+ return
Variants.ofTimestampntz(DateTimeUtil.microsFromTimestamp((LocalDateTime)
value));
Review Comment:
correct! @brandonstanleyappfolio, LocalDateTime is converted into
microseconds across the codebase.
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java:
##########
@@ -464,6 +479,218 @@ protected Temporal convertTimestampValue(Object value,
TimestampType type) {
return convertLocalDateTime(value);
}
+ protected Variant convertVariantValue(Object value) {
+ if (value instanceof ByteBuffer) {
+ return Variant.from((ByteBuffer) value);
+ }
+
+ Set<String> fieldNames = Sets.newHashSet();
+ collectFieldNames(value, fieldNames);
+ List<String> allFieldNames =
fieldNames.stream().sorted().collect(Collectors.toList());
+ VariantMetadata metadata = Variants.metadata(allFieldNames);
+ VariantValue variantValue = objectToVariantValue(value, metadata, null);
+ return Variant.of(metadata, variantValue);
+ }
+
+ /**
+ * Collects all field names (map keys) from the entire object tree into the
given set. Used to
+ * build a single VariantMetadata for the whole Variant (required for nested
maps).
+ */
+ private static void collectFieldNames(Object value, Set<String> names) {
+ if (value == null) {
+ return;
+ }
+ if (value instanceof Collection) {
+ for (Object element : (Collection<?>) value) {
+ collectFieldNames(element, names);
+ }
+ return;
+ }
+ if (value instanceof Map) {
+ Map<?, ?> map = (Map<?, ?>) value;
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ Object key = entry.getKey();
+ if (key != null && key instanceof String) {
+ names.add((String) key);
+ collectFieldNames(entry.getValue(), names);
+ }
+ }
+ return;
+ }
+ if (value instanceof Struct) {
+ Struct struct = (Struct) value;
+ for (Field field : struct.schema().fields()) {
+ names.add(field.name());
+ collectFieldNames(struct.get(field), names);
+ }
+ }
+ }
+
+ /**
+ * Recursively converts a Java object to a VariantValue using the given
shared metadata for all
+ * nested maps. Handles primitives, List (array), and Map (object); map keys
become field names.
+ */
+ private static VariantValue objectToVariantValue(
+ Object value, VariantMetadata metadata,
org.apache.kafka.connect.data.Schema schema) {
+ if (value == null) {
+ return Variants.ofNull();
+ }
+ VariantValue primitive = primitiveToVariantValue(value, schema);
+ if (primitive != null) {
+ return primitive;
+ }
+ if (value instanceof Collection) {
+ ValueArray array = Variants.array();
+ org.apache.kafka.connect.data.Schema elementSchema =
+ schema != null ? schema.valueSchema() : null;
+ for (Object element : (Collection<?>) value) {
+ array.add(objectToVariantValue(element, metadata, elementSchema));
+ }
+ return array;
+ }
+ if (value instanceof Map) {
+ return mapToVariantValue(value, metadata, schema);
+ }
+ if (value instanceof Struct) {
+ Struct struct = (Struct) value;
+ ShreddedObject object = Variants.object(metadata);
+ for (Field field : struct.schema().fields()) {
+ object.put(field.name(), objectToVariantValue(struct.get(field),
metadata, field.schema()));
+ }
+ return object;
+ }
+ throw new IllegalArgumentException("Cannot convert to variant: " +
value.getClass().getName());
+ }
+
+ private static VariantValue mapToVariantValue(
+ Object value, VariantMetadata metadata,
org.apache.kafka.connect.data.Schema schema) {
+ Map<?, ?> map = (Map<?, ?>) value;
+ ShreddedObject object = Variants.object(metadata);
+ org.apache.kafka.connect.data.Schema mapValueSchema =
+ schema != null ? schema.valueSchema() : null;
+ map.forEach(
+ (key, val) -> {
+ if (key != null && key instanceof String) {
+ object.put((String) key, objectToVariantValue(val, metadata,
mapValueSchema));
+ } else {
+ throw new IllegalArgumentException(
+ "Cannot convert map to variant: keys must be non-null strings,
was: "
+ + (key == null ? "null" : key.getClass().getName()));
+ }
+ });
+ return object;
+ }
+
+ /**
+ * Converts a primitive or primitive-like value to VariantValue; returns
null if not supported.
+ * The optional schema is used to disambiguate java.util.Date which Kafka
Connect uses for Date,
+ * Time, and Timestamp logical types.
+ */
+ private static VariantValue primitiveToVariantValue(
+ Object value, org.apache.kafka.connect.data.Schema schema) {
+ if (value instanceof Boolean) {
+ return Variants.of((Boolean) value);
+ }
+ VariantValue temporal = temporalObjectToVariantValue(value, schema);
+ if (temporal != null) {
+ return temporal;
+ }
+ if (value instanceof Number) {
+ return numberToVariantValue((Number) value);
+ }
+ if (value instanceof String) {
+ return Variants.of((String) value);
+ }
+ if (value instanceof ByteBuffer) {
+ return Variants.of((ByteBuffer) value);
+ }
+ if (value instanceof byte[]) {
+ return Variants.of(ByteBuffer.wrap((byte[]) value));
+ }
+ if (value instanceof UUID) {
+ return Variants.ofUUID((UUID) value);
+ }
+ return null;
+ }
+
+ /**
+ * Converts java.time values and java.util.Date (with Connect logical type
from the optional
+ * schema) to VariantValue; returns null if the value is not a supported
temporal representation.
+ */
+ private static VariantValue temporalObjectToVariantValue(
+ Object value, org.apache.kafka.connect.data.Schema schema) {
+ if (value instanceof Instant) {
+ return Variants.ofTimestamptz(DateTimeUtil.microsFromInstant((Instant)
value));
+ }
+ if (value instanceof OffsetDateTime) {
+ return
Variants.ofTimestamptz(DateTimeUtil.microsFromTimestamptz((OffsetDateTime)
value));
+ }
+ if (value instanceof ZonedDateTime) {
+ return Variants.ofTimestamptz(
+ DateTimeUtil.microsFromTimestamptz(((ZonedDateTime)
value).toOffsetDateTime()));
+ }
+ if (value instanceof LocalDateTime) {
+ return
Variants.ofTimestampntz(DateTimeUtil.microsFromTimestamp((LocalDateTime)
value));
Review Comment:
correct! @brandonstanleyappfolio, `LocalDateTime` is converted into
microseconds across the codebase.
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]