Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
19 commits
Select commit Hold shift + click to select a range
033f47f
feat: Implement support for VARIANT type in RecordConverter with conv…
seokyun-ha-toss Feb 10, 2026
a53af92
test: implement unit test for RecordConverter for Variant
seokyun-ha-toss Feb 10, 2026
73ea656
chore: lint spotlessApply
seokyun-ha-toss Feb 10, 2026
2b3fcb1
fix: there's no already Variant value, and use `Lists.of()`
seokyun-ha-toss Feb 11, 2026
47f6fe5
fix: handle `BigDecimal` type in `numberToVariantValue()`
seokyun-ha-toss Mar 12, 2026
e8f57f1
fix: explicit error when unknown numeric type
seokyun-ha-toss Mar 12, 2026
b57e0d8
refactor: optimize field name collection in RecordConverter to use a …
seokyun-ha-toss Mar 12, 2026
0117174
fix: ensure keys in map are non-null strings before processing in Rec…
seokyun-ha-toss Mar 12, 2026
be09349
refactor: loop entrySet once
seokyun-ha-toss Mar 12, 2026
75ec7ce
test: enhance unit tests for Variant conversion with additional cases…
seokyun-ha-toss Mar 13, 2026
c6e558a
lint: apply gh actions auto-review and lints, split Cyclomatic Comple…
seokyun-ha-toss Mar 13, 2026
787663d
lint: spotlessApply
seokyun-ha-toss Mar 13, 2026
75dc8a2
feat: support `kafka Struct` type, and align if checking ordering
seokyun-ha-toss Mar 17, 2026
d9c8721
test: implt unittest for Variant for Struct
seokyun-ha-toss Mar 17, 2026
e517884
lint: gradlew spotlessApply
seokyun-ha-toss Mar 17, 2026
8bb8740
feat: add support for BigInteger conversion to BigDecimal in RecordCo…
seokyun-ha-toss Mar 27, 2026
aa2bcce
feat: support for Date conversion to Variants
seokyun-ha-toss Mar 27, 2026
5be9fa4
feat: support additional Date, Datetime, time types
seokyun-ha-toss Mar 27, 2026
179b07a
lint: spotlessApply
seokyun-ha-toss Mar 27, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,23 +22,28 @@
import java.io.IOException;
import java.io.UncheckedIOException;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.math.RoundingMode;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.time.format.DateTimeFormatterBuilder;
import java.time.format.DateTimeParseException;
import java.time.temporal.Temporal;
import java.util.Base64;
import java.util.Collection;
import java.util.Date;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.iceberg.FileFormat;
Expand All @@ -53,6 +58,7 @@
import org.apache.iceberg.mapping.NameMappingParser;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Type.PrimitiveType;
import org.apache.iceberg.types.Types.DecimalType;
Expand All @@ -64,6 +70,13 @@
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.UUIDUtil;
import org.apache.iceberg.variants.ShreddedObject;
import org.apache.iceberg.variants.ValueArray;
import org.apache.iceberg.variants.Variant;
import org.apache.iceberg.variants.VariantMetadata;
import org.apache.iceberg.variants.VariantValue;
import org.apache.iceberg.variants.Variants;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.ConnectException;

Expand Down Expand Up @@ -142,6 +155,8 @@ private Object convertValue(
return convertTimeValue(value);
case TIMESTAMP:
return convertTimestampValue(value, (TimestampType) type);
case VARIANT:
return convertVariantValue(value);
}
throw new UnsupportedOperationException("Unsupported type: " + type.typeId());
}
Expand Down Expand Up @@ -464,6 +479,171 @@ 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);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a non-shredded version of Object, any performance differences?

map.forEach(
(key, val) -> {
if (key != null && key instanceof String) {
object.put((String) key, objectToVariantValue(val, metadata));
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems like we should throw here if the key is null or the value is an instance of string?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(one way would be checking the ShreddedObject size is the same?

});
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 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));
}
if (value instanceof LocalDate) {
return Variants.ofDate(DateTimeUtil.daysFromDate((LocalDate) value));
}
if (value instanceof LocalTime) {
return Variants.ofTime(DateTimeUtil.microsFromTime((LocalTime) value));
}
if (value instanceof Date) {
int days = (int) (((Date) value).getTime() / 1000 / 60 / 60 / 24);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The java.util.Date class is poorly named, i think we should be mapping this to Timestamp TZ so we don't lose precision. https://docs.oracle.com/javase/8/docs/api/java/util/Date.html for details.

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That’s a good point. Do we need to add a check to determine the exact logical type?
I’ve opened this PR against @seokyun-ha-toss's branch. Let me know your thoughts @seokyun-ha-toss @emkornfield!

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wow, really appreciate it @brandonstanleyappfolio , Thanks! I'll take a look!!

Sadly, I don't have much time to work on in these days.... I'll resume this work as soon as possible. Thanks for guys! 🙏

return Variants.ofDate(days);
}
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 BigInteger) {
return Variants.of(new BigDecimal((BigInteger) 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);
}
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also consider Dates?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I interpreted from the pr description that these are derived from JSON which doesn't have dates? But this is a good clarifying question.

Copy link
Copy Markdown

@brandonstanleyappfolio brandonstanleyappfolio Mar 30, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That makes sense! I can provide some context around my use case if that helps.

I am loading Confluent Avro-serialized data that contains timestamp fields. When those fields are deserialized by the connector, they are converted to java.util.Date, which this PR does not currently handle. I added the following code to RecordConverter.java to resolve the issue:

if (value instanceof Date) {
  return Variants.of(((Date) value).getTime());
}

I also see that @seokyun-ha-toss added support for dates here!

throw new IllegalArgumentException(
"Cannot convert Number to variant (unknown type): " + value.getClass().getName());
}

@SuppressWarnings("JavaUtilDate")
private OffsetDateTime convertOffsetDateTime(Object value) {
if (value instanceof Number) {
Expand Down
Loading
Loading