[ https://issues.apache.org/jira/browse/BEAM-8933?focusedWorklogId=371050&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-371050 ]
ASF GitHub Bot logged work on BEAM-8933: ---------------------------------------- Author: ASF GitHub Bot Created on: 13/Jan/20 19:50 Start Date: 13/Jan/20 19:50 Worklog Time Spent: 10m Work Description: TheNeuralBit commented on pull request #10369: [BEAM-8933] BigQueryIO Arrow for read URL: https://github.com/apache/beam/pull/10369#discussion_r365979515 ########## File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java ########## @@ -552,18 +553,26 @@ public static Read read() { .setBigQueryServices(new BigQueryServicesImpl()) .setParseFn(parseFn) .setMethod(Method.DEFAULT) + .setFormat(DataFormat.AVRO) .build(); } @VisibleForTesting - static class TableRowParser implements SerializableFunction<SchemaAndRecord, TableRow> { + public static class TableRowParser implements SerializableFunction<SchemaAndRecord, TableRow> { public static final TableRowParser INSTANCE = new TableRowParser(); @Override public TableRow apply(SchemaAndRecord schemaAndRecord) { - return BigQueryAvroUtils.convertGenericRecordToTableRow( - schemaAndRecord.getRecord(), schemaAndRecord.getTableSchema()); + if (schemaAndRecord.getRecord() != null) { + return BigQueryAvroUtils.convertGenericRecordToTableRow( + schemaAndRecord.getRecord(), schemaAndRecord.getTableSchema()); + } else if (schemaAndRecord.getRow() != null) { + return BigQueryUtils.toTableRow().apply(schemaAndRecord.getRow()); Review comment: This will copy all of the data out of each Beam Row into a `TableRow` instance. Ideally we would return the actual `Row` instances instead so that the underlying data can still be lazily accessed. I think there are two options here: 1) Address this when we add support for Arrow to `BigQueryTable` for use in SQL. It looks like we currently pass a custom conversion function to `BigQueryIO.read` [here](https://github.com/apache/beam/blob/04498d56f9bc17528d807c6ef02434eed29decf1/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java#L212). We could just make that function return the underlying row when we're reading Arrow. 2) Add a function like `BigQueryIO.readRows` that encapsulates the row-conversion logic for all the different read modes. Then we can just call that function in `BigQueryTable#buildIOReader` Either option could be addressed later, but I'd like to have a plan and a follow-up jira filed. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 371050) Time Spent: 5h 10m (was: 5h) > BigQuery IO should support read/write in Arrow format > ----------------------------------------------------- > > Key: BEAM-8933 > URL: https://issues.apache.org/jira/browse/BEAM-8933 > Project: Beam > Issue Type: Improvement > Components: io-java-gcp > Reporter: Kirill Kozlov > Assignee: Kirill Kozlov > Priority: Major > Time Spent: 5h 10m > Remaining Estimate: 0h > > As of right now BigQuery uses Avro format for reading and writing. > We should add a config to BigQueryIO to specify which format to use: Arrow or > Avro (with Avro as default). -- This message was sent by Atlassian Jira (v8.3.4#803005)