RustedBones commented on code in PR #32360: URL: https://github.com/apache/beam/pull/32360#discussion_r1743371777
########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java: ########## @@ -64,35 +58,103 @@ */ class BigQueryAvroUtils { + static class DateTimeLogicalType extends LogicalType { + public DateTimeLogicalType() { + super("datetime"); + } + } + + static final DateTimeLogicalType DATETIME_LOGICAL_TYPE = new DateTimeLogicalType(); + /** * Defines the valid mapping between BigQuery types and native Avro types. * - * <p>Some BigQuery types are duplicated here since slightly different Avro records are produced - * when exporting data in Avro format and when reading data directly using the read API. + * @see <a href=https://cloud.google.com/bigquery/docs/exporting-data#avro_export_details>BQ avro + * export</a> + * @see <a href=https://cloud.google.com/bigquery/docs/reference/storage#avro_schema_details>BQ + * avro storage</a> */ - static final ImmutableMultimap<String, Type> BIG_QUERY_TO_AVRO_TYPES = - ImmutableMultimap.<String, Type>builder() - .put("STRING", Type.STRING) - .put("GEOGRAPHY", Type.STRING) - .put("BYTES", Type.BYTES) - .put("INTEGER", Type.LONG) - .put("INT64", Type.LONG) - .put("FLOAT", Type.DOUBLE) - .put("FLOAT64", Type.DOUBLE) - .put("NUMERIC", Type.BYTES) - .put("BIGNUMERIC", Type.BYTES) - .put("BOOLEAN", Type.BOOLEAN) - .put("BOOL", Type.BOOLEAN) - .put("TIMESTAMP", Type.LONG) - .put("RECORD", Type.RECORD) - .put("STRUCT", Type.RECORD) - .put("DATE", Type.STRING) Review Comment: This was probably the cause of https://github.com/apache/beam/issues/20677. Schema conversion was only taking the 1st occurence. When writing a `date` we want a `int` with `date` logical type. We only want the `string` representation when reading an exported table without logical-types enabled. -- 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org