ahmedabu98 commented on code in PR #27213:
URL: https://github.com/apache/beam/pull/27213#discussion_r1257328681


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaChangeIT.java:
##########
@@ -0,0 +1,515 @@
+/*
+ * 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.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+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.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+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.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(Parameterized.class)
+public class StorageApiSinkSchemaChangeIT {
+  @Parameterized.Parameters
+  public static Iterable<Object[]> data() {
+    return ImmutableList.of(new Object[] {true}, new Object[] {false});
+  }
+
+  @Parameterized.Parameter(0)
+  public boolean useWriteSchema;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StorageApiSinkSchemaChangeIT.class);
+
+  private static final BigqueryClient BQ_CLIENT =
+      new BigqueryClient("StorageApiSinkSchemaChangeIT");
+  private static final String PROJECT =
+      TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+  private static final String BIG_QUERY_DATASET_ID =
+      "storage_api_sink_schema_change" + System.nanoTime();
+
+  private static final String[] FIELDS = {
+    "BOOL",
+    "BOOLEAN",
+    "BYTES",
+    "INT64",
+    "INTEGER",
+    "FLOAT",
+    "FLOAT64",
+    "NUMERIC",
+    "STRING",
+    "DATE",
+    "TIMESTAMP"
+  };
+
+  private static final int MAX_N = 50;
+
+  private static final long RANDOM_SEED = 1;
+
+  @Rule public transient ExpectedLogs loggedBigQueryIO = 
ExpectedLogs.none(BigQueryIO.class);
+
+  @BeforeClass
+  public static void setUpTestEnvironment() throws IOException, 
InterruptedException {
+    // Create one BQ dataset for all test cases.
+    BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID);
+  }
+
+  @AfterClass
+  public static void cleanup() {
+    LOG.info("Start to clean up tables and datasets.");
+    BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
+  }
+
+  private static String createTable(TableSchema tableSchema)
+      throws IOException, InterruptedException {
+    String tableId = "table" + System.nanoTime();
+    BQ_CLIENT.deleteTable(PROJECT, BIG_QUERY_DATASET_ID, tableId);
+    BQ_CLIENT.createNewTable(
+        PROJECT,
+        BIG_QUERY_DATASET_ID,
+        new Table()
+            .setSchema(tableSchema)
+            .setTableReference(
+                new TableReference()
+                    .setTableId(tableId)
+                    .setDatasetId(BIG_QUERY_DATASET_ID)
+                    .setProjectId(PROJECT)));
+    return tableId;
+  }
+
+  private static boolean checkRowCompleteness(String tableSpec)
+      throws IOException, InterruptedException {
+    TableRow queryResponse =
+        Iterables.getOnlyElement(
+            BQ_CLIENT.queryUnflattened(
+                String.format("SELECT COUNT(DISTINCT(id)), MIN(id), MAX(id) 
FROM %s", tableSpec),
+                PROJECT,
+                true,
+                true));
+
+    int distinctCount = Integer.parseInt((String) queryResponse.get("f0_"));
+    int rangeMin = Integer.parseInt((String) queryResponse.get("f1_"));
+    int rangeMax = Integer.parseInt((String) queryResponse.get("f2_"));
+
+    LOG.info("total distinct count = {}, min = {}, max = {}", distinctCount, 
rangeMin, rangeMax);
+
+    return (rangeMax - rangeMin + 1) == distinctCount && distinctCount == 
MAX_N;
+  }
+
+  private static boolean checkRowDuplication(String tableSpec)
+      throws IOException, InterruptedException {
+    TableRow queryResponse =
+        Iterables.getOnlyElement(
+            BQ_CLIENT.queryUnflattened(
+                String.format("SELECT COUNT(DISTINCT(id)), COUNT(id) FROM %s", 
tableSpec),
+                PROJECT,
+                true,
+                true));
+
+    int distinctCount = Integer.parseInt((String) queryResponse.get("f0_"));
+    int totalCount = Integer.parseInt((String) queryResponse.get("f1_"));
+
+    LOG.info("total distinct count = {}, total count = {}", distinctCount, 
totalCount);
+
+    return distinctCount == totalCount;
+  }
+
+  static class UpdateSchemaDoFn extends DoFn<KV<Integer, Long>, Long> {
+
+    private final String projectId;
+    private final String datasetId;
+    private final String tableId;
+
+    private final String schemaString;
+
+    private transient BigqueryClient bqClient;
+
+    private static final String MY_COUNTER = "myCounter";
+
+    @StateId(MY_COUNTER)
+    @SuppressWarnings("unused")
+    private final StateSpec<@org.jetbrains.annotations.NotNull 
ValueState<Integer>> counter;
+
+    public UpdateSchemaDoFn(
+        String projectId, String datasetId, String tableId, TableSchema 
schema) {
+      this.projectId = projectId;
+      this.datasetId = datasetId;
+      this.tableId = tableId;
+      this.schemaString = BigQueryHelpers.toJsonString(schema);
+      this.bqClient = null;
+      this.counter = StateSpecs.value();
+    }
+
+    private int getRowCount(String tableSpec) throws IOException, 
InterruptedException {
+      TableRow queryResponse =
+          Iterables.getOnlyElement(
+              bqClient.queryUnflattened(
+                  String.format("SELECT COUNT(*) FROM %s", tableSpec), 
PROJECT, true, true));
+      return Integer.parseInt((String) queryResponse.get("f0_"));
+    }
+
+    @Setup
+    public void setup() {
+      bqClient = new 
BigqueryClient("StorageApiSinkSchemaChangeIT_UpdateSchema");
+    }
+
+    @Teardown
+    public void tearDown() {
+      return;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, @StateId(MY_COUNTER) 
ValueState<Integer> counter)
+        throws InterruptedException {
+      int current = firstNonNull(counter.read(), 0);
+      if (current == 0) {
+        int rowCount = 0;
+        try {
+          rowCount = this.getRowCount(this.projectId + "." + this.datasetId + 
"." + this.tableId);
+        } catch (Exception e) {
+          LOG.error(e.toString());

Review Comment:
   P.S. maybe we should rethrow the error here instead of just logging it? if 
`getRowCount` starts failing for whatever reason, `rowCount` will stay at 0 and 
the schema will never be updated, making the tests obsolete (since the purpose 
is to test the schema update feature).



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