johnjcasey commented on code in PR #29550:
URL: https://github.com/apache/beam/pull/29550#discussion_r1411168052


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java:
##########
@@ -0,0 +1,805 @@
+/*
+ * 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 static 
org.apache.beam.runners.core.construction.TransformUpgrader.fromByteArray;
+import static 
org.apache.beam.runners.core.construction.TransformUpgrader.toByteArray;
+
+import com.google.api.services.bigquery.model.Clustering;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.auto.service.AutoService;
+import 
com.google.cloud.bigquery.storage.v1.AppendRowsRequest.MissingValueInterpretation;
+import com.google.cloud.bigquery.storage.v1.DataFormat;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import 
org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
+import org.apache.beam.runners.core.construction.SdkComponents;
+import 
org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead;
+import 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.FromBeamRowFunction;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.QueryPriority;
+import 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.ToBeamRowFunction;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.SchemaUpdateOption;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import 
org.apache.beam.sdk.io.gcp.bigquery.RowWriterFactory.AvroRowWriterFactory;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.NanosDuration;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes", "nullness"})
+public class BigQueryIOTranslation {
+
+  static class BigQueryIOReadTranslator implements 
TransformPayloadTranslator<TypedRead<?>> {
+
+    static Schema schema =
+        Schema.builder()
+            .addNullableStringField("json_table_ref")
+            .addNullableStringField("query")
+            .addNullableBooleanField("validate")
+            .addNullableBooleanField("flatten_results")
+            .addNullableBooleanField("use_legacy_sql")
+            .addNullableBooleanField("with_template_compatibility")
+            .addNullableByteArrayField("bigquery_services")
+            .addNullableByteArrayField("parse_fn")
+            .addNullableByteArrayField("datum_reader_factory")
+            .addNullableByteArrayField("query_priority")
+            .addNullableStringField("query_location")
+            .addNullableStringField("query_temp_dataset")
+            .addNullableByteArrayField("method")
+            .addNullableByteArrayField("format")
+            .addNullableArrayField("selected_fields", FieldType.STRING)
+            .addNullableStringField("row_restriction")
+            .addNullableByteArrayField("coder")
+            .addNullableStringField("kms_key")
+            .addNullableByteArrayField("type_descriptor")
+            .addNullableByteArrayField("to_beam_row_fn")
+            .addNullableStringField("from_beam_row_fn")
+            .addNullableBooleanField("use_avro_logical_types")
+            .addNullableBooleanField("projection_pushdown_applied")
+            .build();
+
+    public static final String BIGQUERY_READ_TRANSFORM_URN =
+        "beam:transform:org.apache.beam:bigquery_read:v1";
+
+    @Override
+    public String getUrn() {
+      return BIGQUERY_READ_TRANSFORM_URN;
+    }
+
+    @Override
+    public RunnerApi.@Nullable FunctionSpec translate(
+        AppliedPTransform<?, ?, TypedRead<?>> application, SdkComponents 
components)
+        throws IOException {
+      // Setting an empty payload since BigQuery transform payload is not 
actually used by runners
+      // currently.
+      // This can be implemented if runners started actually using the 
BigQuery transform payload.
+      return 
FunctionSpec.newBuilder().setUrn(getUrn()).setPayload(ByteString.empty()).build();
+    }
+
+    @Override
+    public Row toConfigRow(TypedRead<?> transform) {
+      Map<String, Object> fieldValues = new HashMap<>();
+
+      if (transform.getJsonTableRef() != null) {
+        fieldValues.put("json_table_ref", transform.getJsonTableRef().get());
+      }
+      if (transform.getQuery() != null) {
+        fieldValues.put("query", transform.getQuery().get());
+      }
+      fieldValues.put("validate", transform.getValidate());
+      fieldValues.put("flatten_results", transform.getFlattenResults());
+      fieldValues.put("use_legacy_sql", transform.getUseLegacySql());
+      fieldValues.put("with_template_compatibility", 
transform.getWithTemplateCompatibility());
+
+      if (transform.getBigQueryServices() != null) {
+        fieldValues.put("bigquery_services", 
toByteArray(transform.getBigQueryServices()));
+      }
+      if (transform.getParseFn() != null) {
+        fieldValues.put("parse_fn", toByteArray(transform.getParseFn()));
+      }
+      if (transform.getDatumReaderFactory() != null) {
+        fieldValues.put("datum_reader_factory", 
toByteArray(transform.getDatumReaderFactory()));
+      }
+      if (transform.getQueryPriority() != null) {
+        fieldValues.put("query_priority", 
toByteArray(transform.getQueryPriority()));
+      }
+      if (transform.getQueryLocation() != null) {
+        fieldValues.put("query_location", transform.getQueryLocation());
+      }
+      if (transform.getQueryTempDataset() != null) {
+        fieldValues.put("query_temp_dataset", transform.getQueryTempDataset());
+      }
+      if (transform.getMethod() != null) {
+        fieldValues.put("method", toByteArray(transform.getMethod()));
+      }
+      if (transform.getFormat() != null) {
+        fieldValues.put("format", toByteArray(transform.getFormat()));
+      }
+      if (transform.getSelectedFields() != null && 
!transform.getSelectedFields().get().isEmpty()) {
+        fieldValues.put("selected_fields", 
transform.getSelectedFields().get());
+      }
+      if (transform.getRowRestriction() != null) {
+        fieldValues.put("row_restriction", 
transform.getRowRestriction().get());
+      }
+      if (transform.getCoder() != null) {
+        fieldValues.put("coder", toByteArray(transform.getCoder()));
+      }
+      if (transform.getKmsKey() != null) {
+        fieldValues.put("kms_key", transform.getKmsKey());
+      }
+      if (transform.getTypeDescriptor() != null) {
+        fieldValues.put("type_descriptor", 
toByteArray(transform.getTypeDescriptor()));
+      }
+      if (transform.getToBeamRowFn() != null) {
+        fieldValues.put("to_beam_row_fn", 
toByteArray(transform.getToBeamRowFn()));
+      }
+      if (transform.getFromBeamRowFn() != null) {
+        fieldValues.put("from_beam_row_fn", 
toByteArray(transform.getFromBeamRowFn()));
+      }
+      if (transform.getUseAvroLogicalTypes() != null) {
+        fieldValues.put("use_avro_logical_types", 
transform.getUseAvroLogicalTypes());
+      }
+      fieldValues.put("projection_pushdown_applied", 
transform.getProjectionPushdownApplied());
+
+      return Row.withSchema(schema).withFieldValues(fieldValues).build();
+    }
+
+    @Override
+    public TypedRead<?> fromConfigRow(Row configRow) {
+      BigQueryIO.TypedRead.Builder builder = new 
AutoValue_BigQueryIO_TypedRead.Builder<>();
+
+      String jsonTableRef = configRow.getString("json_table_ref");
+      if (jsonTableRef != null) {
+        builder = 
builder.setJsonTableRef(StaticValueProvider.of(jsonTableRef));
+      }
+      String query = configRow.getString("query");
+      if (query != null) {
+        builder = builder.setQuery(StaticValueProvider.of(query));
+      }
+      Boolean validate = configRow.getBoolean("validate");
+      if (validate != null) {
+        builder = builder.setValidate(validate);
+      }
+      Boolean flattenResults = configRow.getBoolean("flatten_results");
+      if (flattenResults != null) {
+        builder = builder.setFlattenResults(flattenResults);
+      }
+      Boolean useLegacySQL = configRow.getBoolean("use_legacy_sql");
+      if (useLegacySQL != null) {
+        builder = builder.setUseLegacySql(useLegacySQL);
+      }
+      Boolean withTemplateCompatibility = 
configRow.getBoolean("with_template_compatibility");
+      if (withTemplateCompatibility != null) {
+        builder = 
builder.setWithTemplateCompatibility(withTemplateCompatibility);
+      }
+      byte[] bigqueryServicesBytes = configRow.getBytes("bigquery_services");
+      if (bigqueryServicesBytes != null) {
+        builder =
+            builder.setBigQueryServices((BigQueryServices) 
fromByteArray(bigqueryServicesBytes));
+      }
+      byte[] parseFnBytes = configRow.getBytes("parse_fn");
+      if (parseFnBytes != null) {
+        builder = builder.setParseFn((SerializableFunction) 
fromByteArray(parseFnBytes));
+      }
+      byte[] datumReaderFactoryBytes = 
configRow.getBytes("datum_reader_factory");
+      if (datumReaderFactoryBytes != null) {
+        builder =
+            builder.setDatumReaderFactory(
+                (SerializableFunction) fromByteArray(datumReaderFactoryBytes));
+      }
+      byte[] queryPriorityBytes = configRow.getBytes("query_priority");
+      if (queryPriorityBytes != null) {
+        builder = builder.setQueryPriority((QueryPriority) 
fromByteArray(queryPriorityBytes));
+      }
+      String queryLocation = configRow.getString("query_location");
+      if (queryLocation != null) {
+        builder = builder.setQueryLocation(queryLocation);
+      }
+      String queryTempDataset = configRow.getString("query_temp_dataset");
+      if (queryTempDataset != null) {
+        builder = builder.setQueryTempDataset(queryTempDataset);
+      }
+      byte[] methodBytes = configRow.getBytes("method");
+      if (methodBytes != null) {
+        builder = builder.setMethod((TypedRead.Method) 
fromByteArray(methodBytes));
+      }
+      byte[] formatBytes = configRow.getBytes("format");
+      if (methodBytes != null) {
+        builder = builder.setFormat((DataFormat) fromByteArray(formatBytes));
+      }
+      Collection<String> selectedFields = 
configRow.getArray("selected_fields");
+      if (selectedFields != null && !selectedFields.isEmpty()) {
+        
builder.setSelectedFields(StaticValueProvider.of(ImmutableList.of(selectedFields)));
+      }
+      String rowRestriction = configRow.getString("row_restriction");
+      if (rowRestriction != null) {
+        builder = 
builder.setRowRestriction(StaticValueProvider.of(rowRestriction));
+      }
+      byte[] coderBytes = configRow.getBytes("coder");
+      if (coderBytes != null) {
+        builder = builder.setCoder((Coder) fromByteArray(coderBytes));
+      }
+      String kmsKey = configRow.getString("kms_key");
+      if (kmsKey != null) {
+        builder = builder.setKmsKey(kmsKey);
+      }
+      byte[] typeDescriptorBytes = configRow.getBytes("type_descriptor");
+      if (typeDescriptorBytes != null) {
+        builder = builder.setTypeDescriptor((TypeDescriptor) 
fromByteArray(typeDescriptorBytes));
+      }
+      byte[] toBeamRowFnBytes = configRow.getBytes("to_beam_row_fn");
+      if (toBeamRowFnBytes != null) {
+        builder = builder.setToBeamRowFn((ToBeamRowFunction) 
fromByteArray(toBeamRowFnBytes));
+      }
+      byte[] fromBeamRowFnBytes = configRow.getBytes("from_beam_row_fn");
+      if (fromBeamRowFnBytes != null) {
+        builder = builder.setFromBeamRowFn((FromBeamRowFunction) 
fromByteArray(fromBeamRowFnBytes));
+      }
+      Boolean useAvroLogicalTypes = 
configRow.getBoolean("use_avro_logical_types");
+      if (useAvroLogicalTypes != null) {
+        builder = builder.setUseAvroLogicalTypes(useAvroLogicalTypes);
+      }
+      Boolean projectionPushdownApplied = 
configRow.getBoolean("projection_pushdown_applied");
+      if (projectionPushdownApplied != null) {
+        builder = 
builder.setProjectionPushdownApplied(projectionPushdownApplied);
+      }
+
+      return builder.build();
+    }
+  }
+
+  @AutoService(TransformPayloadTranslatorRegistrar.class)
+  public static class ReadRegistrar implements 
TransformPayloadTranslatorRegistrar {
+
+    @Override
+    @SuppressWarnings({
+      "rawtypes",
+    })
+    public Map<? extends Class<? extends PTransform>, ? extends 
TransformPayloadTranslator>
+        getTransformPayloadTranslators() {
+      return ImmutableMap.<Class<? extends PTransform>, 
TransformPayloadTranslator>builder()
+          .put(AutoValue_BigQueryIO_TypedRead.class, new 
BigQueryIOReadTranslator())
+          .build();
+    }
+  }
+
+  static class BigQueryIOWriteTranslator implements 
TransformPayloadTranslator<Write<?>> {
+
+    private static byte[] toByteArray(Object object) {

Review Comment:
   These too, just as a cleanup



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