TheNeuralBit commented on a change in pull request #14586: URL: https://github.com/apache/beam/pull/14586#discussion_r641853161
########## File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageReader.java ########## @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import java.io.IOException; + +interface BigQueryStorageReader extends AutoCloseable { + + void processReadRowsResponse(ReadRowsResponse readRowsResponse) throws IOException; + + long getRowCount(); + + Object readSingleRecord() throws IOException; Review comment: ```suggestion // TODO(BEAM-XXX): BigQueryStorageReader should produce Rows, rather than GenericRecords GenericRecord readSingleRecord() throws IOException; ``` Let's make this return a specific type (`GenericRecord`). Please also file a follow-up jira to change this interface to produce Rows (or I can if you'd prefer), and we can put it in a TODO here. ########## File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaAndRecord.java ########## @@ -33,6 +35,11 @@ public SchemaAndRecord(GenericRecord record, TableSchema tableSchema) { this.tableSchema = tableSchema; } + public SchemaAndRecord(Row record, TableSchema tableSchema) { Review comment: nit: let's not do this conversion in the `SchemaAndRecord` constructor. Instead we could change `BigQueryStorageStreamSource.readSingleRecord` to always return a `GenericRecord`, and the Arrow implementation can do the conversion there. ########## File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaAndRecord.java ########## @@ -18,23 +18,32 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableSchema; +import javax.annotation.Nullable; import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.values.Row; /** * A wrapper for a {@link GenericRecord} and the {@link TableSchema} representing the schema of the * table (or query) it was generated from. */ public class SchemaAndRecord { - private final GenericRecord record; + private final Object record; private final TableSchema tableSchema; - public SchemaAndRecord(GenericRecord record, TableSchema tableSchema) { + public SchemaAndRecord(Object record, TableSchema tableSchema) { this.record = record; this.tableSchema = tableSchema; } public GenericRecord getRecord() { - return record; + if (!(record instanceof GenericRecord)) { + throw new IllegalStateException("Object is not GenericRecord"); + } + return (GenericRecord) record; + } Review comment: Ok, sounds good, thank you ########## File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java ########## @@ -1351,4 +1353,20 @@ public void testReadFromBigQueryIOWithTrimmedSchema() throws Exception { p.run(); } + + private static org.apache.arrow.vector.types.pojo.Field field( Review comment: Should there be an test with the Arrow format in this file? It looks like it's still only testing Avro, and these new methods aren't used. ########## File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java ########## @@ -1351,4 +1353,20 @@ public void testReadFromBigQueryIOWithTrimmedSchema() throws Exception { p.run(); } + + private static org.apache.arrow.vector.types.pojo.Field field( Review comment: Ah it looks like these methods are only used in `BigQueryStorageReaderTest`. That file doesn't have any end-to-end tests of reading Arrow either though. Could you add a test like `testReadFromBigQueryIO` that uses the Arrow format? -- 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: [email protected]
