RustedBones commented on code in PR #33221:
URL: https://github.com/apache/beam/pull/33221#discussion_r1875980676
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java:
##########
@@ -305,45 +339,83 @@ private static Object convertRequiredField(String name,
Schema schema, Object v)
LogicalType logicalType = schema.getLogicalType();
switch (type) {
case BOOLEAN:
- // SQL types BOOL, BOOLEAN
+ // SQL type BOOL (BOOLEAN)
return v;
case INT:
if (logicalType instanceof LogicalTypes.Date) {
- // SQL types DATE
+ // SQL type DATE
+ // ideally LocalDate but TableRowJsonCoder encodes as String
return formatDate((Integer) v);
+ } else if (logicalType instanceof LogicalTypes.TimeMillis) {
+ // Write only: SQL type TIME
+ // ideally LocalTime but TableRowJsonCoder encodes as String
+ return formatTime(((Integer) v) * 1000L);
} else {
- throw new UnsupportedOperationException(
- String.format("Unexpected Avro field schema type %s for field
named %s", type, name));
+ // Write only: SQL type INT64 (INT, SMALLINT, INTEGER, BIGINT,
TINYINT, BYTEINT)
+ // ideally Integer but keep consistency with BQ JSON export that
uses String
+ return ((Integer) v).toString();
}
case LONG:
if (logicalType instanceof LogicalTypes.TimeMicros) {
- // SQL types TIME
+ // SQL type TIME
+ // ideally LocalTime but TableRowJsonCoder encodes as String
return formatTime((Long) v);
+ } else if (logicalType instanceof LogicalTypes.TimestampMillis) {
+ // Write only: SQL type TIMESTAMP
+ // ideally Instant but TableRowJsonCoder encodes as String
+ return formatTimestamp((Long) v * 1000L);
} else if (logicalType instanceof LogicalTypes.TimestampMicros) {
- // SQL types TIMESTAMP
+ // SQL type TIMESTAMP
+ // ideally Instant but TableRowJsonCoder encodes as String
return formatTimestamp((Long) v);
+ } else if (!(VERSION_AVRO.startsWith("1.8") ||
VERSION_AVRO.startsWith("1.9"))
+ && logicalType instanceof LogicalTypes.LocalTimestampMillis) {
+ // Write only: SQL type DATETIME
+ // ideally LocalDateTime but TableRowJsonCoder encodes as String
+ return formatDatetime(((Long) v) * 1000);
+ } else if (!(VERSION_AVRO.startsWith("1.8") ||
VERSION_AVRO.startsWith("1.9"))
+ && logicalType instanceof LogicalTypes.LocalTimestampMicros) {
+ // Write only: SQL type DATETIME
+ // ideally LocalDateTime but TableRowJsonCoder encodes as String
+ return formatDatetime((Long) v);
} else {
- // SQL types INT64 (INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT)
+ // SQL type INT64 (INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT)
+ // ideally Long if in [2^53+1, 2^53-1] but keep consistency with BQ
JSON export that uses
+ // String
return ((Long) v).toString();
}
+ case FLOAT:
Review Comment:
yes
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java:
##########
@@ -501,48 +813,4 @@ public void testSchemaCollisionsInAvroConversion() {
String output = BigQueryAvroUtils.toGenericAvroSchema(schema,
false).toString();
assertThat(output.length(), greaterThan(0));
}
-
- /** Pojo class used as the record type in tests. */
- @SuppressWarnings("unused") // Used by Avro reflection.
- static class Bird {
Review Comment:
Working with this POJO was difficult due to the field names not reflecting
the underlying type.
I reworked the test suite to test all conversion in isolation.
It's also more convenient to check the `useAvroLogicalType` parameter: In
most cases, the converted schema is identical regardless of this value. we can
reused the expected schema definition in both cases.
Some extra cases have been added like parameterized numeric
--
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]