ahmedabu98 commented on code in PR #32688:
URL: https://github.com/apache/beam/pull/32688#discussion_r1794167592
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java:
##########
@@ -282,12 +306,50 @@ private static void copyFieldIntoRecord(Record rec,
Types.NestedField field, Row
Optional.ofNullable(value.getDouble(name)).ifPresent(v ->
rec.setField(name, v));
break;
case DATE:
- throw new UnsupportedOperationException("Date fields not yet
supported");
+ Optional.ofNullable(value.getLogicalTypeValue(name, LocalDate.class))
+ .ifPresent(v -> rec.setField(name, v));
+ break;
case TIME:
- throw new UnsupportedOperationException("Time fields not yet
supported");
+ Optional.ofNullable(value.getLogicalTypeValue(name, LocalTime.class))
+ .ifPresent(v -> rec.setField(name, v));
+ break;
case TIMESTAMP:
- Optional.ofNullable(value.getDateTime(name))
- .ifPresent(v -> rec.setField(name, v.getMillis()));
+ Object val = value.getValue(name);
+ if (val == null) {
+ break;
+ }
+ Types.TimestampType ts = (Types.TimestampType)
field.type().asPrimitiveType();
+ // timestamp with timezone
+ if (ts.shouldAdjustToUTC()) {
+ // currently only string is supported because other types
+ // do not maintain timezone information
+ if (val instanceof String) {
+ // e.g. 2007-12-03T10:15:30+01:00
+ rec.setField(name, OffsetDateTime.parse((String) val));
+ break;
+ } else {
+ throw new UnsupportedOperationException(
Review Comment:
I refactored the conversion logic so that now, any of these types can write
to both `timestamp` and `timestamptz`:
- Schema.FieldType.INT64 --> long
- Schema.FieldType.INT64. --> String
- Schema.FieldType.DATETIME --> org.joda.time.DateTime
- SqlTypes.DATETIME --> java.time.LocalDateTime
When writing to `timestamptz`, we will resolve to UTC first before writing.
Lmk how things look now. I updated the java doc as well with more details
--
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]