brandonstanleyappfolio commented on code in PR #15283:
URL: https://github.com/apache/iceberg/pull/15283#discussion_r2992003183
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java:
##########
@@ -464,6 +476,145 @@ 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);
+ 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) {
+ if (value == null) {
+ return Variants.ofNull();
+ }
+ VariantValue primitive = primitiveToVariantValue(value);
+ if (primitive != null) {
+ return primitive;
+ }
+ if (value instanceof Collection) {
+ ValueArray array = Variants.array();
+ for (Object element : (Collection<?>) value) {
+ array.add(objectToVariantValue(element, metadata));
+ }
+ return array;
+ }
+ if (value instanceof Map) {
+ Map<?, ?> map = (Map<?, ?>) value;
+ ShreddedObject object = Variants.object(metadata);
+ map.forEach(
+ (key, val) -> {
+ if (key != null && key instanceof String) {
+ object.put((String) key, objectToVariantValue(val, metadata));
+ }
+ });
+ return object;
+ }
+ 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));
+ }
+ return object;
+ }
+ throw new IllegalArgumentException("Cannot convert to variant: " +
value.getClass().getName());
+ }
+
+ /**
+ * Converts a primitive or primitive-like value to VariantValue; returns
null if not supported.
+ */
+ private static VariantValue primitiveToVariantValue(Object value) {
+ if (value instanceof Boolean) {
+ return Variants.of((Boolean) value);
+ }
+ 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;
+ }
+
+ private static VariantValue numberToVariantValue(Number value) {
+ if (value instanceof BigDecimal) {
+ return Variants.of((BigDecimal) value);
+ }
+ if (value instanceof Integer) {
+ return Variants.of((Integer) value);
+ }
+ if (value instanceof Long) {
+ return Variants.of((Long) value);
+ }
+ if (value instanceof Float) {
+ return Variants.of((Float) value);
+ }
+ if (value instanceof Double) {
+ return Variants.of((Double) value);
+ }
+ if (value instanceof Byte) {
+ return Variants.of((Byte) value);
+ }
+ if (value instanceof Short) {
+ return Variants.of((Short) value);
+ }
Review Comment:
Should we also consider Dates?
--
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]