Skip to content
Open
Changes from 4 commits
Commits
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 @@ -56,6 +56,8 @@
import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
import org.apache.beam.sdk.schemas.logicaltypes.FixedString;
import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant;
import org.apache.beam.sdk.schemas.logicaltypes.NanosInstant;
import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes;
Expand Down Expand Up @@ -416,6 +418,10 @@ static Object toBeamObject(Object value, FieldType fieldType, boolean verifyValu
String identifier = logicalType.getIdentifier();
if (TimeWithLocalTzType.IDENTIFIER.equals(identifier)) {
return Instant.ofEpochMilli(((Number) value).longValue());
} else if (MicrosInstant.IDENTIFIER.equals(identifier)
|| NanosInstant.IDENTIFIER.equals(identifier)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

There is a loss of accuracy here, as micros instant and nanos instant are sub-millis. Also I'd need to confirm this would actually work when input Row has a schema that contains a MicrosInstant/NanoInstant logical type

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes nice catch but I think this is because calcite itself is limited to milliseconds (reference) and the sub-millis are lost in the Beam -> Calcite -> Beam round-trip. Can u confirm if there is a different fix for this?

// Portable instant logical types: treat calcite numeric as epoch milliseconds.
return Instant.ofEpochMilli(((Number) value).longValue());
} else if (SqlTypes.DATE.getIdentifier().equals(identifier)) {
if (value instanceof Date) {
value = SqlFunctions.toInt((Date) value);
Expand Down Expand Up @@ -574,6 +580,17 @@ private static Expression getBeamField(
value = Expressions.call(expression, "getBytes", fieldName);
} else if (TimeWithLocalTzType.IDENTIFIER.equals(identifier)) {
value = Expressions.call(expression, "getDateTime", fieldName);
} else if (MicrosInstant.IDENTIFIER.equals(identifier)
|| NanosInstant.IDENTIFIER.equals(identifier)) {
// Instant-like logical types: retrieve as Instant
value =
Expressions.convert_(
Expressions.call(
expression,
"getLogicalTypeValue",
fieldName,
Expressions.constant(java.time.Instant.class)),
java.time.Instant.class);
} else if (SqlTypes.DATE.getIdentifier().equals(identifier)) {
value =
Expressions.convert_(
Expand Down Expand Up @@ -601,6 +618,59 @@ private static Expression getBeamField(
fieldName,
Expressions.constant(LocalDateTime.class)),
LocalDateTime.class);
} else if (fieldType.getLogicalType() instanceof PassThroughLogicalType) {
// For pass-through logical types, read the underlying base type using the
// corresponding Row getter.
FieldType baseType = fieldType.getLogicalType().getBaseType();
switch (baseType.getTypeName()) {
case BYTE:
value = Expressions.call(expression, "getByte", fieldName);
break;
case INT16:
value = Expressions.call(expression, "getInt16", fieldName);
break;
case INT32:
value = Expressions.call(expression, "getInt32", fieldName);
break;
case INT64:
value = Expressions.call(expression, "getInt64", fieldName);
break;
case DECIMAL:
value = Expressions.call(expression, "getDecimal", fieldName);
break;
case FLOAT:
value = Expressions.call(expression, "getFloat", fieldName);
break;
case DOUBLE:
value = Expressions.call(expression, "getDouble", fieldName);
break;
case STRING:
value = Expressions.call(expression, "getString", fieldName);
break;
case DATETIME:
value = Expressions.call(expression, "getDateTime", fieldName);
break;
case BOOLEAN:
value = Expressions.call(expression, "getBoolean", fieldName);
break;
case BYTES:
value = Expressions.call(expression, "getBytes", fieldName);
break;
case ARRAY:
value = Expressions.call(expression, "getArray", fieldName);
break;
case MAP:
value = Expressions.call(expression, "getMap", fieldName);
break;
case ROW:
value = Expressions.call(expression, "getRow", fieldName);
break;
case ITERABLE:
value = Expressions.call(expression, "getIterable", fieldName);
break;
default:
throw new UnsupportedOperationException("Unable to get logical type " + identifier);
}
} else if (FixedPrecisionNumeric.IDENTIFIER.equals(identifier)) {
value = Expressions.call(expression, "getDecimal", fieldName);
} else {
Expand Down Expand Up @@ -661,6 +731,13 @@ private static Expression toCalciteValue(Expression value, FieldType fieldType)
} else if (TimeWithLocalTzType.IDENTIFIER.equals(identifier)) {
return nullOr(
value, Expressions.call(Expressions.convert_(value, DateTime.class), "getMillis"));
} else if (MicrosInstant.IDENTIFIER.equals(identifier)
|| NanosInstant.IDENTIFIER.equals(identifier)) {
// Convert java.time.Instant to epoch milliseconds for Calcite
return nullOr(
value,
Expressions.call(
Expressions.convert_(value, java.time.Instant.class), "toEpochMilli"));
} else if (SqlTypes.DATE.getIdentifier().equals(identifier)) {
return nullOr(
value,
Expand Down Expand Up @@ -692,6 +769,47 @@ private static Expression toCalciteValue(Expression value, FieldType fieldType)
return nullOr(value, returnValue);
} else if (FixedPrecisionNumeric.IDENTIFIER.equals(identifier)) {
return Expressions.convert_(value, BigDecimal.class);
} else if (fieldType.getLogicalType() instanceof PassThroughLogicalType) {
// For pass-through logical types, convert underlying base type to Calcite value
FieldType baseType = fieldType.getLogicalType().getBaseType();
switch (baseType.getTypeName()) {
case BYTE:
return Expressions.convert_(value, Byte.class);
case INT16:
return Expressions.convert_(value, Short.class);
case INT32:
return Expressions.convert_(value, Integer.class);
case INT64:
return Expressions.convert_(value, Long.class);
case DECIMAL:
return Expressions.convert_(value, BigDecimal.class);
case FLOAT:
return Expressions.convert_(value, Float.class);
case DOUBLE:
return Expressions.convert_(value, Double.class);
case STRING:
return Expressions.convert_(value, String.class);
case BOOLEAN:
return Expressions.convert_(value, Boolean.class);
case DATETIME:
return nullOr(
value,
Expressions.call(
Expressions.convert_(value, AbstractInstant.class), "getMillis"));
case BYTES:
return nullOr(
value,
Expressions.new_(ByteString.class, Expressions.convert_(value, byte[].class)));
case ARRAY:
return nullOr(value, toCalciteList(value, baseType.getCollectionElementType()));
case MAP:
return nullOr(value, toCalciteMap(value, baseType.getMapValueType()));
case ROW:
return nullOr(value, toCalciteRow(value, baseType.getRowSchema()));
default:
throw new UnsupportedOperationException(
"Unable to convert logical type " + identifier);
}
} else {
throw new UnsupportedOperationException("Unable to convert logical type " + identifier);
}
Expand Down
Loading