Abacn opened a new issue, #26408:
URL: https://github.com/apache/beam/issues/26408

   ### What happened?
   
   When implementing BigQuery STORAGE_WRITE_API performance test, it is found 
that withAvroFormatFunction needs to be set differently:
   
   For FILE_LOAD:
   ```
   BigQueryIO.<byte[]>write()
       .withAvroFormatFunction(
               writeRequest -> {
                 byte[] data = writeRequest.getElement();
                 GenericRecord record = new 
GenericData.Record(writeRequest.getSchema());
                 record.put("data", ByteBuffer.wrap(data));
                 return record;
               })
   ```
   See 
https://github.com/apache/beam/blob/44a17cbe429699160e689bd88935c32c6c2de6b2/sdks/java/io/bigquery-io-perf-tests/src/test/java/org/apache/beam/sdk/bigqueryioperftests/BigQueryIOIT.java#L169
   
   
   For STORAGE_WRITE_API, however, it needs to be
   ```
   BigQueryIO.<byte[]>write()
       .withAvroFormatFunction(
               writeRequest -> {
                 byte[] data = writeRequest.getElement();
                 GenericRecord record = new 
GenericData.Record(writeRequest.getSchema());
                 record.put("data", data);
                 return record;
               })
   ```
   
   Using `ByteBuffer.wrap(data)` will throw this Exception at run time:
   ```
   Caused by: java.lang.ClassCastException: class java.nio.HeapByteBuffer 
cannot be cast to class [B (java.nio.HeapByteBuffer and [B are in module 
java.base of loader 'bootstrap')
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroGenericRecordToStorageApiProto.lambda$static$4(AvroGenericRecordToStorageApiProto.java:90)
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroGenericRecordToStorageApiProto.scalarToProtoValue(AvroGenericRecordToStorageApiProto.java:375)
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroGenericRecordToStorageApiProto.toProtoValue(AvroGenericRecordToStorageApiProto.java:332)
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroGenericRecordToStorageApiProto.toProtoValue(AvroGenericRecordToStorageApiProto.java:318)
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroGenericRecordToStorageApiProto.messageValueFromGenericRecordValue(AvroGenericRecordToStorageApiProto.java:296)
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroGenericRecordToStorageApiProto.messageFromGenericRecord(AvroGenericRecordToStorageApiProto.java:183)
           at 
org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinationsGenericRecord$GenericRecordConverter.toMessage(StorageApiDynamicDestinationsGenericRecord.java:73)
           at 
org.apache.beam.sdk.io.gcp.bigquery.StorageApiConvertMessages$ConvertMessagesDoFn.processElement(StorageApiConvertMessages.java:139)
   ```
   at here:
   
   
https://github.com/apache/beam/blob/44a17cbe429699160e689bd88935c32c6c2de6b2/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java#L90
   
   In contrast, if directly feed a byte[] for FILE_LOAD, it gives the following 
error:
   
   ```
   Caused by: org.apache.avro.file.DataFileWriter$AppendWriteException: 
org.apache.avro.AvroRuntimeException: Unknown datum type [B: [B@6f10f22c
           at 
org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:317)
           at 
org.apache.beam.sdk.io.gcp.bigquery.AvroRowWriter.write(AvroRowWriter.java:52)
           at 
org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.processElement(WriteBundlesToFiles.java:236)
   Caused by: org.apache.avro.AvroRuntimeException: Unknown datum type [B: 
[B@6f10f22c
           at 
org.apache.avro.generic.GenericData.getSchemaName(GenericData.java:933)
           at 
org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:892)
           at 
org.apache.avro.generic.GenericDatumWriter.resolveUnion(GenericDatumWriter.java:307)
   ```
   
   Ideally we should support ByteBuffer in AvroGenericRecordToStorageApiProto.
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [X] 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