kennknowles commented on code in PR #17070:
URL: https://github.com/apache/beam/pull/17070#discussion_r849475973


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java:
##########
@@ -1013,4 +1018,28 @@ public static ServiceCallMetric 
readCallMetric(TableReference tableReference) {
   public static ServiceCallMetric writeCallMetric(TableReference 
tableReference) {
     return callMetricForMethod(tableReference, "BigQueryBatchWrite");
   }
+
+  /**
+   * Hashes a schema descriptor using a deterministic hash function.
+   *
+   * <p>Warning! These hashes are encoded into messages, so changing this 
function will cause

Review Comment:
   Very smart to add this warning. We should do it a lot more throughout the 
codebase.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java:
##########
@@ -1013,4 +1018,28 @@ public static ServiceCallMetric 
readCallMetric(TableReference tableReference) {
   public static ServiceCallMetric writeCallMetric(TableReference 
tableReference) {
     return callMetricForMethod(tableReference, "BigQueryBatchWrite");
   }
+
+  /**
+   * Hashes a schema descriptor using a deterministic hash function.
+   *
+   * <p>Warning! These hashes are encoded into messages, so changing this 
function will cause
+   * pipelines to get stuck on update!
+   */
+  public static long hashSchemaDescriptorDeterministic(Descriptor descriptor) {

Review Comment:
   I browsed the code for FieldDescriptor and did notice that there are a lot 
more fields. Are we super confident that this is the full set needed for this 
use case?
   
   FieldDescriptor doesn't define its own equals or hashCode but it does define 
comparable and treats equality is just field number equality, so there's no 
obvious precedent I am aware of. Just asking if there's a risk of the fields 
here being insufficient or changing.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableDestinations.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Preconditions.checkArgument;
+
+import com.google.api.services.bigquery.model.TableSchema;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+
+/**
+ * Creates any tables needed before performing streaming writes to the tables. 
This is a side-effect
+ * {@link DoFn}, and returns the original collection unchanged.
+ */
+public class CreateTableDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, PCollection<KV<DestinationT, 
ElementT>>> {
+  private final CreateDisposition createDisposition;
+  private final BigQueryServices bqServices;
+  private final DynamicDestinations<?, DestinationT> dynamicDestinations;
+  @Nullable private final String kmsKey;
+
+  /**
+   * The list of tables created so far, so we don't try the creation each time.
+   *
+   * <p>TODO: We should put a bound on memory usage of this. Use guava cache 
instead.
+   */
+  public CreateTableDestinations(
+      CreateDisposition createDisposition,
+      DynamicDestinations<?, DestinationT> dynamicDestinations) {
+    this(createDisposition, new BigQueryServicesImpl(), dynamicDestinations, 
null);
+  }
+
+  public CreateTableDestinations(
+      CreateDisposition createDisposition,
+      BigQueryServices bqServices,
+      DynamicDestinations<?, DestinationT> dynamicDestinations,
+      @Nullable String kmsKey) {
+    this.createDisposition = createDisposition;
+    this.bqServices = bqServices;
+    this.dynamicDestinations = dynamicDestinations;
+    this.kmsKey = kmsKey;
+  }
+
+  CreateTableDestinations<DestinationT, ElementT> withKmsKey(String kmsKey) {
+    return new CreateTableDestinations<>(
+        createDisposition, bqServices, dynamicDestinations, kmsKey);
+  }
+
+  CreateTableDestinations<DestinationT, ElementT> 
withTestServices(BigQueryServices bqServices) {
+    return new CreateTableDestinations<>(
+        createDisposition, bqServices, dynamicDestinations, kmsKey);
+  }
+
+  @Override
+  public PCollection<KV<DestinationT, ElementT>> expand(
+      PCollection<KV<DestinationT, ElementT>> input) {
+    List<PCollectionView<?>> sideInputs = Lists.newArrayList();
+    sideInputs.addAll(dynamicDestinations.getSideInputs());
+
+    return input.apply("CreateTables", ParDo.of(new 
CreateTablesFn()).withSideInputs(sideInputs));
+  }
+
+  private class CreateTablesFn
+      extends DoFn<KV<DestinationT, ElementT>, KV<DestinationT, ElementT>> {
+    private Map<DestinationT, TableDestination> destinations = 
Maps.newHashMap();
+
+    @StartBundle
+    public void startBundle() {
+      destinations = Maps.newHashMap();
+    }
+
+    @ProcessElement
+    @SuppressWarnings({
+      "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+    })
+    public void processElement(
+        ProcessContext context,
+        @Element KV<DestinationT, ElementT> element,
+        OutputReceiver<KV<DestinationT, ElementT>> o) {
+      dynamicDestinations.setSideInputAccessorFromProcessContext(context);
+      destinations.computeIfAbsent(
+          element.getKey(),
+          dest -> {
+            TableDestination tableDestination1 = 
dynamicDestinations.getTable(dest);
+            checkArgument(
+                tableDestination1 != null,
+                "DynamicDestinations.getTable() may not return null, "
+                    + "but %s returned null for destination %s",
+                dynamicDestinations,
+                dest);
+            Supplier<TableSchema> schemaSupplier = () -> 
dynamicDestinations.getSchema(dest);
+            return CreateTableHelpers.possiblyCreateTable(

Review Comment:
   Isn't it identical to pulling from the map and if the result is null 
computing it and inserting it? I didn't realize it was more than a convenience.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableDestinations.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Preconditions.checkArgument;
+
+import com.google.api.services.bigquery.model.TableSchema;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+
+/**
+ * Creates any tables needed before performing streaming writes to the tables. 
This is a side-effect
+ * {@link DoFn}, and returns the original collection unchanged.
+ */
+public class CreateTableDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, PCollection<KV<DestinationT, 
ElementT>>> {
+  private final CreateDisposition createDisposition;
+  private final BigQueryServices bqServices;
+  private final DynamicDestinations<?, DestinationT> dynamicDestinations;
+  @Nullable private final String kmsKey;
+
+  /**
+   * The list of tables created so far, so we don't try the creation each time.
+   *
+   * <p>TODO: We should put a bound on memory usage of this. Use guava cache 
instead.
+   */
+  public CreateTableDestinations(
+      CreateDisposition createDisposition,
+      DynamicDestinations<?, DestinationT> dynamicDestinations) {
+    this(createDisposition, new BigQueryServicesImpl(), dynamicDestinations, 
null);
+  }
+
+  public CreateTableDestinations(
+      CreateDisposition createDisposition,
+      BigQueryServices bqServices,
+      DynamicDestinations<?, DestinationT> dynamicDestinations,
+      @Nullable String kmsKey) {
+    this.createDisposition = createDisposition;
+    this.bqServices = bqServices;
+    this.dynamicDestinations = dynamicDestinations;
+    this.kmsKey = kmsKey;
+  }
+
+  CreateTableDestinations<DestinationT, ElementT> withKmsKey(String kmsKey) {
+    return new CreateTableDestinations<>(
+        createDisposition, bqServices, dynamicDestinations, kmsKey);
+  }
+
+  CreateTableDestinations<DestinationT, ElementT> 
withTestServices(BigQueryServices bqServices) {
+    return new CreateTableDestinations<>(
+        createDisposition, bqServices, dynamicDestinations, kmsKey);
+  }
+
+  @Override
+  public PCollection<KV<DestinationT, ElementT>> expand(
+      PCollection<KV<DestinationT, ElementT>> input) {
+    List<PCollectionView<?>> sideInputs = Lists.newArrayList();
+    sideInputs.addAll(dynamicDestinations.getSideInputs());
+
+    return input.apply("CreateTables", ParDo.of(new 
CreateTablesFn()).withSideInputs(sideInputs));
+  }
+
+  private class CreateTablesFn
+      extends DoFn<KV<DestinationT, ElementT>, KV<DestinationT, ElementT>> {
+    private Map<DestinationT, TableDestination> destinations = 
Maps.newHashMap();
+
+    @StartBundle
+    public void startBundle() {
+      destinations = Maps.newHashMap();
+    }
+
+    @ProcessElement
+    @SuppressWarnings({
+      "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+    })
+    public void processElement(
+        ProcessContext context,
+        @Element KV<DestinationT, ElementT> element,
+        OutputReceiver<KV<DestinationT, ElementT>> o) {
+      dynamicDestinations.setSideInputAccessorFromProcessContext(context);
+      destinations.computeIfAbsent(
+          element.getKey(),
+          dest -> {
+            TableDestination tableDestination1 = 
dynamicDestinations.getTable(dest);
+            checkArgument(
+                tableDestination1 != null,
+                "DynamicDestinations.getTable() may not return null, "

Review Comment:
   Ah. Just checking since the `checkArgument` and the types on things didn't 
line up. Now they match anyhow. Incidentally if you use `checkArgumentNotNull` 
it adds the necessary assertions and whatnot so checker knows it is non-null 
afterwards.



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