You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/04/08 12:45:18 UTC

[GitHub] [beam] kennknowles commented on a diff in pull request #17070: Handle changing schemas in Storage API sink

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


##########
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);

Review Comment:
   This variable should be `@Nullable`, clearly since you check it in the next line. And you can use `checkArgumentNotNull` to safely convert it to a non-null typed variable.



##########
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:
   Didn't click through because GitHub isn't that friendly for it, but is `getTable` result marked `@Nullable` or not?



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

Review Comment:
   This comment seems to be on the wrong element.



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

Review Comment:
   Would it make sense instead to return the tables created? Or, two outputs:
   
   1. PCollection of elements where it is now safe to write them to their destination table.
   2. PCollection of tables that have been created
   
   I assume the design currently is so the downstream stuff blocks on elements arriving in PCollection 1?



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

Review Comment:
   Why add this to new code? Is there something that cannot be easily made nullness-correct? At least localize it to a specific line / declaration.



##########
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:
   Is `computeIfAbsent` better than an `if` statement here? Seeing it used primarily for side effects, particularly side effects on another storage system, seems like it would be better expressed as an imperative block than this quasi-functional style.



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org