jonathan-lemos opened a new issue, #26870:
URL: https://github.com/apache/beam/issues/26870

   ### What happened?
   
   On 2.48.0-SNAPSHOT, reading from a BigQuery table with a column of type 
`ARRAY<STRING NOT NULL> NOT NULL` causes the following exception:
   ```
   org.apache.beam.sdk.Pipeline$PipelineExecutionException: 
java.lang.ClassCastException: class java.lang.String cannot be cast to class 
java.util.Map (java.lang.String and java.util.Map are in module java.base of 
loader 'bootstrap')
   
        at 
org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:374)
        at 
org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:342)
        at 
com.google.cloud.syndeo.perf.SyndeoTestingPairsIT.run(SyndeoTestingPairsIT.java:216)
        at 
com.google.cloud.syndeo.perf.SyndeoTestingPairsIT.testSyndeoTemplateDataflow(SyndeoTestingPairsIT.java:175)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:566)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at 
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
        at 
org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
        at 
org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
        at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
        at 
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
        at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
        at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
        at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
        at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
        at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
        at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
        at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
        at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
        at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
        at org.junit.runners.Suite.runChild(Suite.java:128)
        at org.junit.runners.Suite.runChild(Suite.java:27)
        at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
        at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
        at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
        at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
        at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
        at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
        at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
        at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
        at 
com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
        at 
com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
        at 
com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
        at 
com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
        at 
com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
        at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
   Caused by: java.lang.ClassCastException: class java.lang.String cannot be 
cast to class java.util.Map (java.lang.String and java.util.Map are in module 
java.base of loader 'bootstrap')
        at 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamValue$12(BigQueryUtils.java:697)
        at 
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
        at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)
        at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
        at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
        at 
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
        at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at 
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
        at 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamValue(BigQueryUtils.java:699)
        at 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRowFieldValue(BigQueryUtils.java:643)
        at 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.lambda$toBeamRow$6(BigQueryUtils.java:629)
        at 
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
        at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)
        at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
        at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
        at 
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
        at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at 
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
        at 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toBeamRow(BigQueryUtils.java:630)
        at 
org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryStorageWriteApiSchemaTransformProvider$BigQueryStorageWriteApiPCollectionRowTupleTransform.lambda$expand$96381ff0$1(BigQueryStorageWriteApiSchemaTransformProvider.java:348)
        at 
org.apache.beam.sdk.transforms.MapElements$2.processElement(MapElements.java:151)
   ```
   
   Looking at `BigQueryUtils.java:697`, it looks like `toBeamValue` expects a 
List to have inner type `Map<String, Object>`, when the passed in value is 
`List<String>`.
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [ ] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [X] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


-- 
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]

Reply via email to