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

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

                Author: ASF GitHub Bot
            Created on: 06/Jul/18 21:53
            Start Date: 06/Jul/18 21:53
    Worklog Time Spent: 10m 
      Work Description: lukecwik closed pull request #5892: [BEAM-4738] [SQL] 
Remove decimal support in BigQuery SQL read.
URL: https://github.com/apache/beam/pull/5892
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
index c39da40939e..8b3b4a64769 100644
--- 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
+++ 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
@@ -19,7 +19,6 @@
 
 import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.BYTE;
-import static org.apache.beam.sdk.schemas.Schema.FieldType.DECIMAL;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.DOUBLE;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.FLOAT;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.INT16;
@@ -32,7 +31,6 @@
 
 import com.google.common.collect.ImmutableMap;
 import java.io.Serializable;
-import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.Pipeline;
@@ -77,7 +75,6 @@
           .addNullableField("c_integer", INT32)
           .addNullableField("c_float", FLOAT)
           .addNullableField("c_double", DOUBLE)
-          .addNullableField("c_decimal", DECIMAL)
           .addNullableField("c_boolean", BOOLEAN)
           .addNullableField("c_timestamp", 
FieldType.DATETIME.withMetadata("TS"))
           .addNullableField("c_varchar", STRING)
@@ -102,7 +99,6 @@ public void testSQLRead() {
             + "   c_integer INTEGER, \n"
             + "   c_float FLOAT, \n"
             + "   c_double DOUBLE, \n"
-            + "   c_decimal DECIMAL, \n"
             + "   c_boolean BOOLEAN, \n"
             + "   c_timestamp TIMESTAMP, \n"
             + "   c_varchar VARCHAR, \n "
@@ -123,7 +119,6 @@ public void testSQLRead() {
             + "2147483647, "
             + "1.0, "
             + "1.0, "
-            + "123.45, "
             + "TRUE, "
             + "TIMESTAMP '2018-05-28 20:17:40.123', "
             + "'varchar', "
@@ -149,7 +144,6 @@ public void testSQLRead() {
                 2147483647,
                 (float) 1.0,
                 1.0,
-                BigDecimal.valueOf(123.45),
                 true,
                 new DateTime(2018, 05, 28, 20, 17, 40, 123, 
ISOChronology.getInstanceUTC()),
                 "varchar",
@@ -171,7 +165,6 @@ public void testSQLTypes() {
             + "   c_integer INTEGER, \n"
             + "   c_float FLOAT, \n"
             + "   c_double DOUBLE, \n"
-            + "   c_decimal DECIMAL, \n"
             + "   c_boolean BOOLEAN, \n"
             + "   c_timestamp TIMESTAMP, \n"
             + "   c_varchar VARCHAR, \n "
@@ -192,7 +185,6 @@ public void testSQLTypes() {
             + "2147483647, "
             + "1.0, "
             + "1.0, "
-            + "123.45, "
             + "TRUE, "
             + "TIMESTAMP '2018-05-28 20:17:40.123', "
             + "'varchar', "
@@ -214,7 +206,6 @@ public void testSQLTypes() {
                 2147483647,
                 (float) 1.0,
                 1.0,
-                BigDecimal.valueOf(123.45),
                 true,
                 new DateTime(2018, 05, 28, 20, 17, 40, 123, 
ISOChronology.getInstanceUTC()),
                 "varchar",
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroUtils.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroUtils.java
index c9d9c9fe17b..e36700d0726 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroUtils.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroUtils.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.sdk.io.gcp.bigquery;
 
-import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.sdk.schemas.Schema.Field;
@@ -35,7 +34,6 @@ public static Object convertAvroFormat(Field beamField, 
Object value) throws Run
       case INT64:
       case FLOAT:
       case DOUBLE:
-      case DECIMAL:
       case BYTE:
       case BOOLEAN:
         ret = convertAvroPrimitiveTypes(beamFieldTypeName, value);
@@ -50,6 +48,8 @@ public static Object convertAvroFormat(Field beamField, 
Object value) throws Run
       case ARRAY:
         ret = convertAvroArray(beamField, value);
         break;
+      case DECIMAL:
+        throw new RuntimeException("Does not support converting DECIMAL type 
value");
       case MAP:
         throw new RuntimeException("Does not support converting MAP type 
value");
       default:
@@ -99,7 +99,7 @@ private static Object convertAvroPrimitiveTypes(TypeName 
beamType, Object value)
       case BOOLEAN:
         return (Boolean) value;
       case DECIMAL:
-        return BigDecimal.valueOf((double) value);
+        throw new RuntimeException("Does not support converting DECIMAL type 
value");
       case STRING:
         return convertAvroString(value);
       default:


 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 120088)
    Time Spent: 40m  (was: 0.5h)

> org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryReadWriteIT.testSQL
>  java.nio.HeapByteBuffer cannot be cast to java.lang.Double
> ------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-4738
>                 URL: https://issues.apache.org/jira/browse/BEAM-4738
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>            Reporter: Luke Cwik
>            Assignee: Rui Wang
>            Priority: Major
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryReadWriteIT.testSQLRead
>  fails a lot:
> [https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/1022/testReport/]
> [https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/1023/testReport/]
> [https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/1027/testReport/]
> {code:java}
> java.lang.ClassCastException: java.nio.HeapByteBuffer cannot be cast to 
> java.lang.Double at 
> org.apache.beam.sdk.io.gcp.bigquery.AvroUtils.convertAvroPrimitiveTypes(AvroUtils.java:102)
>  at 
> org.apache.beam.sdk.io.gcp.bigquery.AvroUtils.convertAvroFormat(AvroUtils.java:41)
>  at 
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRow(BigQueryUtils.java:207)
>  at 
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils$ToBeamRow.apply(BigQueryUtils.java:199)
>  at 
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils$ToBeamRow.apply(BigQueryUtils.java:186)
>  at 
> org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase$1.apply(BigQuerySourceBase.java:222)
>  at 
> org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase$1.apply(BigQuerySourceBase.java:215)
>  at 
> org.apache.beam.sdk.io.AvroSource$AvroBlock.readNextRecord(AvroSource.java:567)
>  at 
> org.apache.beam.sdk.io.BlockBasedSource$BlockBasedReader.readNextRecord(BlockBasedSource.java:209)
>  at 
> org.apache.beam.sdk.io.FileBasedSource$FileBasedReader.advanceImpl(FileBasedSource.java:484)
>  at 
> org.apache.beam.sdk.io.FileBasedSource$FileBasedReader.startImpl(FileBasedSource.java:479)
>  at 
> org.apache.beam.sdk.io.OffsetBasedSource$OffsetBasedReader.start(OffsetBasedSource.java:249)
>  at 
> org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$BoundedReadEvaluator.processElement(BoundedReadEvaluatorFactory.java:147)
>  at 
> org.apache.beam.runners.direct.DirectTransformExecutor.processElements(DirectTransformExecutor.java:160)
>  at 
> org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:124)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748){code}



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

Reply via email to