TheNeuralBit commented on a change in pull request #14586:
URL: https://github.com/apache/beam/pull/14586#discussion_r659086156



##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageArrowReader.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.ArrowSchema;
+import com.google.cloud.bigquery.storage.v1.ReadRowsResponse;
+import com.google.cloud.bigquery.storage.v1.ReadSession;
+import java.io.IOException;
+import java.io.InputStream;
+import javax.annotation.Nullable;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.extensions.arrow.ArrowConversion;
+import 
org.apache.beam.sdk.extensions.arrow.ArrowConversion.RecordBatchRowIterator;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.values.Row;
+
+@SuppressWarnings("nullness")
+class BigQueryStorageArrowReader implements BigQueryStorageReader {
+
+  private org.apache.beam.sdk.schemas.Schema arrowBeamSchema;
+  private @Nullable RecordBatchRowIterator recordBatchIterator;
+  private long rowCount;
+  private ArrowSchema protoSchema;
+  private @Nullable RootAllocator alloc;
+
+  BigQueryStorageArrowReader(ReadSession readSession) throws IOException {
+    protoSchema = readSession.getArrowSchema();
+    InputStream input = protoSchema.getSerializedSchema().newInput();
+    this.arrowBeamSchema =
+        ArrowConversion.ArrowSchemaTranslator.toBeamSchema(
+            ArrowConversion.arrowSchemaFromInput(input));
+    this.rowCount = 0;
+    this.alloc = null;
+  }
+
+  @Override
+  public void processReadRowsResponse(ReadRowsResponse readRowsResponse) 
throws IOException {
+    com.google.cloud.bigquery.storage.v1.ArrowRecordBatch recordBatch =
+        readRowsResponse.getArrowRecordBatch();
+    rowCount = recordBatch.getRowCount();
+    this.alloc = new RootAllocator(Long.MAX_VALUE);
+    InputStream input = protoSchema.getSerializedSchema().newInput();
+    Schema arrowSchema = ArrowConversion.arrowSchemaFromInput(input);
+    this.recordBatchIterator =
+        ArrowConversion.rowsFromSerializedRecordBatch(
+            arrowSchema, recordBatch.getSerializedRecordBatch().newInput(), 
this.alloc);
+  }
+
+  @Override
+  public long getRowCount() {
+    return rowCount;
+  }
+
+  @Override
+  public GenericRecord readSingleRecord() throws IOException {
+    if (recordBatchIterator == null) {
+      throw new IOException("Not Initialized");
+    }
+    Row row = recordBatchIterator.next();
+    return AvroUtils.toGenericRecord(row, null);

Review comment:
       Could you file a follow-up JIRA to refactor this so we don't have to 
convert to Avro first, and add a TODO here?
   ```suggestion
       // TODO(BEAM-XXX): Update this interface to expect a Row, and avoid 
converting Arrow data to GenericRecord.
       return AvroUtils.toGenericRecord(row, null);
   ```

##########
File path: 
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java
##########
@@ -1351,4 +1435,613 @@ public void testReadFromBigQueryIOWithTrimmedSchema() 
throws Exception {
 
     p.run();
   }
+
+  @Test
+  public void testReadFromBigQueryIOArrow() throws Exception {
+    LOG.info("Entering arrow test");

Review comment:
       It looks like this was a debug log message that can be removed now?

##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java
##########
@@ -163,4 +187,23 @@
   public BoundedReader<T> createReader(PipelineOptions options) throws 
IOException {
     throw new UnsupportedOperationException("BigQuery storage source must be 
split before reading");
   }
+
+  /*private static org.apache.arrow.vector.types.pojo.Schema 
convertArrowSchema(
+          ArrowSchema arrowSchema) throws IOException {
+    CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(new 
ByteArrayOutputStream());
+    return org.apache.arrow.vector.types.pojo.Schema.deserialize(bb);
+  }*/
+
+  private static ArrowSchema convertArrowSchema(
+      org.apache.arrow.vector.types.pojo.Schema arrowSchema) {
+    ByteArrayOutputStream byteOutputStream = new ByteArrayOutputStream();
+    try {
+      MessageSerializer.serialize(
+          new WriteChannel(Channels.newChannel(byteOutputStream)), 
arrowSchema);
+    } catch (IOException ex) {
+      throw new RuntimeException("Failed to serialize arrow schema.", ex);
+    }
+    ByteString byteString = 
ByteString.copyFrom(byteOutputStream.toByteArray());
+    return ArrowSchema.newBuilder().setSerializedSchema(byteString).build();
+  }

Review comment:
       Also this, I don't think it's used

##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java
##########
@@ -163,4 +187,23 @@
   public BoundedReader<T> createReader(PipelineOptions options) throws 
IOException {
     throw new UnsupportedOperationException("BigQuery storage source must be 
split before reading");
   }
+
+  /*private static org.apache.arrow.vector.types.pojo.Schema 
convertArrowSchema(
+          ArrowSchema arrowSchema) throws IOException {
+    CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(new 
ByteArrayOutputStream());
+    return org.apache.arrow.vector.types.pojo.Schema.deserialize(bb);
+  }*/

Review comment:
       Please remove this

##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java
##########
@@ -146,7 +159,18 @@
       return ImmutableList.of();
     }
 
-    Schema sessionSchema = new 
Schema.Parser().parse(readSession.getAvroSchema().getSchema());
+    Schema sessionSchema;
+    if (readSession.getDataFormat() == DataFormat.ARROW) {
+      org.apache.arrow.vector.types.pojo.Schema schema =
+          ArrowConversion.arrowSchemaFromInput(
+              readSession.getArrowSchema().getSerializedSchema().newInput());
+      org.apache.beam.sdk.schemas.Schema beamSchema =
+          ArrowConversion.ArrowSchemaTranslator.toBeamSchema(schema);
+      sessionSchema = AvroUtils.toAvroSchema(beamSchema);
+    } else {

Review comment:
       explicitly check for `DataFormat.AVRO` here. The `else` case should 
raise an`IllegalArgumentException`. 
   
   It's a little bit ugly that we have some of the format-specific logic right 
here, while the rest of it is in the `BigQueryStorage{Arrow,Avro}Reader`, but 
this is fine for now.




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