[ 
https://issues.apache.org/jira/browse/BEAM-6392?focusedWorklogId=194836&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-194836
 ]

ASF GitHub Bot logged work on BEAM-6392:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 05/Feb/19 22:09
            Start Date: 05/Feb/19 22:09
    Worklog Time Spent: 10m 
      Work Description: kmjung commented on pull request #7441: [BEAM-6392] Add 
support for the BigQuery read API to BigQueryIO.
URL: https://github.com/apache/beam/pull/7441#discussion_r254066814
 
 

 ##########
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
 ##########
 @@ -177,27 +250,47 @@ private static Object convertRequiredField(
     // Per https://cloud.google.com/bigquery/docs/reference/v2/tables#schema, 
the type field
     // is required, so it may not be null.
     String bqType = fieldSchema.getType();
-    Type expectedAvroType = BIG_QUERY_TO_AVRO_TYPES.get(bqType);
-    verifyNotNull(expectedAvroType, "Unsupported BigQuery type: %s", bqType);
+    ImmutableCollection<Type> expectedAvroTypes = 
BIG_QUERY_TO_AVRO_TYPES.get(bqType);
+    verifyNotNull(expectedAvroTypes, "Unsupported BigQuery type: %s", bqType);
     verify(
-        avroType == expectedAvroType,
-        "Expected Avro schema type %s, not %s, for BigQuery %s field %s",
-        expectedAvroType,
-        avroType,
+        expectedAvroTypes.contains(avroType),
+        "Expected Avro schema types %s for BigQuery %s field %s, but received 
%s",
+        expectedAvroTypes,
         bqType,
-        fieldSchema.getName());
+        fieldSchema.getName(),
+        avroType);
     // For historical reasons, don't validate avroLogicalType except for with 
NUMERIC.
     // BigQuery represents NUMERIC in Avro format as BYTES with a DECIMAL 
logical type.
-    switch (fieldSchema.getType()) {
+    switch (bqType) {
       case "STRING":
-      case "DATE":
       case "DATETIME":
-      case "TIME":
       case "GEOGRAPHY":
         // Avro will use a CharSequence to represent String objects, but it 
may not always use
         // java.lang.String; for example, it may prefer 
org.apache.avro.util.Utf8.
         verify(v instanceof CharSequence, "Expected CharSequence (String), got 
%s", v.getClass());
         return v.toString();
+      case "DATE":
+        if (avroType == Type.INT) {
+          verify(v instanceof Integer, "Expected Integer, got %s", 
v.getClass());
+          verifyNotNull(avroLogicalType, "Expected Date logical type");
+          verify(avroLogicalType instanceof LogicalTypes.Date, "Expected Date 
logical type");
+          return formatDate((Integer) v);
+        } else {
+          verify(v instanceof CharSequence, "Expected CharSequence (String), 
got %s", v.getClass());
+          return v.toString();
+        }
+      case "TIME":
+        if (avroType == Type.LONG) {
+          verify(v instanceof Long, "Expected Long, got %s", v.getClass());
+          verifyNotNull(avroLogicalType, "Expected TimeMicros logical type");
 
 Review comment:
   No -- this code path (an Avro logical 'time-micros' type for a BigQuery TIME 
record) can be reached in both the export and read API cases.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 194836)
    Time Spent: 2h 10m  (was: 2h)

> Add support for new BigQuery streaming read API to BigQueryIO
> -------------------------------------------------------------
>
>                 Key: BEAM-6392
>                 URL: https://issues.apache.org/jira/browse/BEAM-6392
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-gcp
>            Reporter: Kenneth Jung
>            Assignee: Kenneth Jung
>            Priority: Major
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> BigQuery has developed a new streaming egress API which will soon reach 
> public availability. Add support for the new API in BigQueryIO.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to