chamikaramj commented on code in PR #31486:
URL: https://github.com/apache/beam/pull/31486#discussion_r1831489396


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java:
##########
@@ -100,7 +98,7 @@ protected SchemaTransform from(BigQueryWriteConfiguration 
configuration) {
 
   @Override
   public String identifier() {
-    return 
getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_STORAGE_WRITE);
+    return "beam:schematransform:org.apache.beam:bigquery_storage_write:v2";

Review Comment:
   Ditto.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryFileLoadsWriteSchemaTransformProvider.java:
##########
@@ -61,7 +60,7 @@ protected SchemaTransform from(BigQueryWriteConfiguration 
configuration) {
 
   @Override
   public String identifier() {
-    return 
getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_FILE_LOADS);
+    return "beam:schematransform:org.apache.beam:bigquery_fileloads:v1";

Review Comment:
   I think it's still fine to define these URNs in the proto.



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedTransformConstants.java:
##########
@@ -50,9 +50,27 @@ public class ManagedTransformConstants {
   private static final Map<String, String> KAFKA_WRITE_MAPPINGS =
       ImmutableMap.<String, String>builder().put("data_format", 
"format").build();
 
+  private static final Map<String, String> BIGQUERY_READ_MAPPINGS =
+      ImmutableMap.<String, String>builder()
+          .put("table", "table_spec")
+          .put("fields", "selected_fields")
+          .build();
+
+  private static final Map<String, String> BIGQUERY_WRITE_MAPPINGS =
+      ImmutableMap.<String, String>builder()
+          .put("at_least_once", "use_at_least_once_semantics")

Review Comment:
   I assume these mappings are the same for all read/write modes ?



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryManagedIT.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
+import org.apache.beam.sdk.managed.Managed;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PeriodicImpulse;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** This class tests the execution of {@link Managed} BigQueryIO. */
+@RunWith(JUnit4.class)
+public class BigQueryManagedIT {
+  @Rule public TestName testName = new TestName();
+  @Rule public transient TestPipeline writePipeline = TestPipeline.create();
+  @Rule public transient TestPipeline readPipeline = TestPipeline.create();
+
+  private static final Schema SCHEMA =
+      Schema.of(
+          Schema.Field.of("str", Schema.FieldType.STRING),
+          Schema.Field.of("number", Schema.FieldType.INT64));
+
+  private static final List<Row> ROWS =
+      LongStream.range(0, 20)
+          .mapToObj(
+              i ->
+                  Row.withSchema(SCHEMA)
+                      .withFieldValue("str", Long.toString(i))
+                      .withFieldValue("number", i)
+                      .build())
+          .collect(Collectors.toList());
+
+  private static final BigqueryClient BQ_CLIENT = new 
BigqueryClient("BigQueryManagedIT");
+
+  private static final String PROJECT =
+      TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+  private static final String BIG_QUERY_DATASET_ID = "bigquery_managed_" + 
System.nanoTime();
+
+  @BeforeClass
+  public static void setUpTestEnvironment() throws IOException, 
InterruptedException {
+    // Create one BQ dataset for all test cases.
+    BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null);
+  }
+
+  @AfterClass
+  public static void cleanup() {
+    BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
+  }
+
+  @Test
+  public void testBatchFileLoadsWriteRead() {
+    String table =
+        String.format("%s:%s.%s", PROJECT, BIG_QUERY_DATASET_ID, 
testName.getMethodName());
+    Map<String, Object> config = ImmutableMap.of("table", table);
+
+    // file loads requires a GCS temp location
+    String tempLocation = 
writePipeline.getOptions().as(TestPipelineOptions.class).getTempRoot();
+    writePipeline.getOptions().setTempLocation(tempLocation);
+
+    // batch write
+    PCollectionRowTuple.of("input", getInput(writePipeline, false))
+        .apply(Managed.write(Managed.BIGQUERY).withConfig(config));
+    writePipeline.run().waitUntilFinish();
+
+    // read and validate
+    PCollection<Row> outputRows =
+        readPipeline
+            .apply(Managed.read(Managed.BIGQUERY).withConfig(config))
+            .getSinglePCollection();
+    PAssert.that(outputRows).containsInAnyOrder(ROWS);
+
+    readPipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testStreamingStorageWriteRead() {
+    String table =
+        String.format("%s:%s.%s", PROJECT, BIG_QUERY_DATASET_ID, 
testName.getMethodName());
+    Map<String, Object> config = ImmutableMap.of("table", table);
+
+    // streaming write
+    PCollectionRowTuple.of("input", getInput(writePipeline, true))
+        .apply(Managed.write(Managed.BIGQUERY).withConfig(config));
+    writePipeline.run().waitUntilFinish();
+
+    // read and validate
+    PCollection<Row> outputRows =
+        readPipeline
+            .apply(Managed.read(Managed.BIGQUERY).withConfig(config))
+            .getSinglePCollection();
+    PAssert.that(outputRows).containsInAnyOrder(ROWS);

Review Comment:
   Ditto.



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryManagedIT.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
+import org.apache.beam.sdk.managed.Managed;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PeriodicImpulse;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** This class tests the execution of {@link Managed} BigQueryIO. */
+@RunWith(JUnit4.class)
+public class BigQueryManagedIT {
+  @Rule public TestName testName = new TestName();
+  @Rule public transient TestPipeline writePipeline = TestPipeline.create();
+  @Rule public transient TestPipeline readPipeline = TestPipeline.create();
+
+  private static final Schema SCHEMA =
+      Schema.of(
+          Schema.Field.of("str", Schema.FieldType.STRING),
+          Schema.Field.of("number", Schema.FieldType.INT64));
+
+  private static final List<Row> ROWS =
+      LongStream.range(0, 20)
+          .mapToObj(
+              i ->
+                  Row.withSchema(SCHEMA)
+                      .withFieldValue("str", Long.toString(i))
+                      .withFieldValue("number", i)
+                      .build())
+          .collect(Collectors.toList());
+
+  private static final BigqueryClient BQ_CLIENT = new 
BigqueryClient("BigQueryManagedIT");
+
+  private static final String PROJECT =
+      TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+  private static final String BIG_QUERY_DATASET_ID = "bigquery_managed_" + 
System.nanoTime();
+
+  @BeforeClass
+  public static void setUpTestEnvironment() throws IOException, 
InterruptedException {
+    // Create one BQ dataset for all test cases.
+    BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null);
+  }
+
+  @AfterClass
+  public static void cleanup() {
+    BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
+  }
+
+  @Test
+  public void testBatchFileLoadsWriteRead() {
+    String table =
+        String.format("%s:%s.%s", PROJECT, BIG_QUERY_DATASET_ID, 
testName.getMethodName());
+    Map<String, Object> config = ImmutableMap.of("table", table);
+
+    // file loads requires a GCS temp location
+    String tempLocation = 
writePipeline.getOptions().as(TestPipelineOptions.class).getTempRoot();
+    writePipeline.getOptions().setTempLocation(tempLocation);
+
+    // batch write
+    PCollectionRowTuple.of("input", getInput(writePipeline, false))
+        .apply(Managed.write(Managed.BIGQUERY).withConfig(config));
+    writePipeline.run().waitUntilFinish();
+
+    // read and validate
+    PCollection<Row> outputRows =
+        readPipeline
+            .apply(Managed.read(Managed.BIGQUERY).withConfig(config))
+            .getSinglePCollection();
+    PAssert.that(outputRows).containsInAnyOrder(ROWS);

Review Comment:
   Also confirm that we end up using the correct sink here.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryWriteConfiguration.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.providers;
+
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
+
+/**
+ * Configuration for writing to BigQuery with SchemaTransforms. Used by {@link
+ * BigQueryStorageWriteApiSchemaTransformProvider} and {@link
+ * BigQueryFileLoadsWriteSchemaTransformProvider}.
+ */
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class BigQueryWriteConfiguration {
+  protected static final String DYNAMIC_DESTINATIONS = "DYNAMIC_DESTINATIONS";
+
+  @AutoValue
+  public abstract static class ErrorHandling {
+    @SchemaFieldDescription("The name of the output PCollection containing 
failed writes.")
+    public abstract String getOutput();
+
+    public static Builder builder() {
+      return new AutoValue_BigQueryWriteConfiguration_ErrorHandling.Builder();
+    }
+
+    @AutoValue.Builder
+    public abstract static class Builder {
+      public abstract Builder setOutput(String output);
+
+      public abstract ErrorHandling build();
+    }
+  }
+
+  public void validate() {
+    String invalidConfigMessage = "Invalid BigQuery Storage Write 
configuration: ";
+
+    // validate output table spec
+    checkArgument(
+        !Strings.isNullOrEmpty(this.getTable()),
+        invalidConfigMessage + "Table spec for a BigQuery Write must be 
specified.");
+
+    // if we have an input table spec, validate it
+    if (!this.getTable().equals(DYNAMIC_DESTINATIONS)) {
+      checkNotNull(BigQueryHelpers.parseTableSpec(this.getTable()));
+    }
+
+    // validate create and write dispositions
+    String createDisposition = getCreateDisposition();
+    if (createDisposition != null && !createDisposition.isEmpty()) {
+      List<String> createDispositions =
+          Arrays.stream(BigQueryIO.Write.CreateDisposition.values())
+              .map(c -> c.name())
+              .collect(Collectors.toList());
+      Preconditions.checkArgument(
+          createDispositions.contains(createDisposition.toUpperCase()),
+          "Invalid create disposition (%s) was specified. Available 
dispositions are: %s",
+          createDisposition,
+          createDispositions);
+    }
+    String writeDisposition = getWriteDisposition();
+    if (writeDisposition != null && !writeDisposition.isEmpty()) {
+      List<String> writeDispostions =
+          Arrays.stream(BigQueryIO.Write.WriteDisposition.values())
+              .map(w -> w.name())
+              .collect(Collectors.toList());
+      Preconditions.checkArgument(
+          writeDispostions.contains(writeDisposition.toUpperCase()),
+          "Invalid write disposition (%s) was specified. Available 
dispositions are: %s",
+          writeDisposition,
+          writeDispostions);
+    }
+
+    ErrorHandling errorHandling = getErrorHandling();
+    if (errorHandling != null) {
+      checkArgument(
+          !Strings.isNullOrEmpty(errorHandling.getOutput()),
+          invalidConfigMessage + "Output must not be empty if error handling 
specified.");
+    }
+
+    Boolean autoSharding = getAutoSharding();
+    Integer numStreams = getNumStreams();
+    if (autoSharding != null && autoSharding && numStreams != null) {
+      checkArgument(
+          numStreams == 0,
+          invalidConfigMessage
+              + "Cannot set a fixed number of streams when auto-sharding is 
enabled. Please pick only one of the two options.");
+    }
+  }
+
+  /** Instantiates a {@link BigQueryWriteConfiguration.Builder} instance. */
+  public static Builder builder() {
+    return new AutoValue_BigQueryWriteConfiguration.Builder();
+  }
+
+  @SchemaFieldDescription(
+      "The bigquery table to write to. Format: 
[${PROJECT}:]${DATASET}.${TABLE}")
+  public abstract String getTable();
+
+  @SchemaFieldDescription(
+      "Optional field that specifies whether the job is allowed to create new 
tables. "
+          + "The following values are supported: CREATE_IF_NEEDED (the job may 
create the table), CREATE_NEVER ("
+          + "the job must fail if the table does not exist already).")
+  @Nullable
+  public abstract String getCreateDisposition();
+
+  @SchemaFieldDescription(
+      "Specifies the action that occurs if the destination table already 
exists. "
+          + "The following values are supported: "
+          + "WRITE_TRUNCATE (overwrites the table data), "
+          + "WRITE_APPEND (append the data to the table), "
+          + "WRITE_EMPTY (job must fail if the table is not empty).")
+  @Nullable
+  public abstract String getWriteDisposition();
+
+  @SchemaFieldDescription(
+      "Determines how often to 'commit' progress into BigQuery. Default is 
every 5 seconds.")
+  @Nullable
+  public abstract Long getTriggeringFrequencySeconds();
+
+  @SchemaFieldDescription(
+      "This option enables lower latency for insertions to BigQuery but may 
ocassionally "
+          + "duplicate data elements.")
+  @Nullable
+  public abstract Boolean getUseAtLeastOnceSemantics();

Review Comment:
   Seems like this is not usable since we do not expose the write method 
through this API and hence that will be explicitly set ?



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java:
##########
@@ -124,7 +129,9 @@ public static ManagedTransform read(String source) {
    * managed sinks are:
    *
    * <ul>
-   *   <li>{@link Managed#ICEBERG} : Write to Apache Iceberg
+   *   <li>{@link Managed#ICEBERG} : Write to Apache Iceberg tables
+   *   <li>{@link Managed#KAFKA} : Write to Apache Kafka topics
+   *   <li>{@link Managed#BIGQUERY} : Write to GCP BigQuery tables

Review Comment:
   Probably also mention that read/write methods will be determined dynamically.



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