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


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Arrays;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableBiFunction;
+import org.apache.beam.sdk.transforms.SerializableFunctions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.Record;
+import org.checkerframework.checker.nullness.qual.NonNull;
+
+public class IcebergIO {
+
+  public static <ElementT, DestinationT> Write<ElementT, DestinationT> 
writeToDestinations(
+      IcebergCatalog catalog,
+      DynamicDestinations<ElementT, DestinationT> dynamicDestinations,
+      SerializableBiFunction<Record, ElementT, Record> toRecord) {
+    return new Write<>(catalog, dynamicDestinations, toRecord);
+  }
+
+  public static TableFactory<String> forCatalog(final IcebergCatalog catalog) {

Review Comment:
   TBD. Leaving all "catalog" questions unresolved for this revision.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Collections;
+import java.util.UUID;
+import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.ShardedKeyCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.ShardedKey;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+class WriteToDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, 
IcebergWriteResult<DestinationT, ElementT>> {
+
+  @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20;
+  @VisibleForTesting static final int DEFAULT_MAX_FILES_PER_PARTITION = 10_000;
+  @VisibleForTesting static final long DEFAULT_MAX_BYTES_PER_PARTITION = 10L * 
(1L << 40); // 10TB
+  static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB
+  static final int DEFAULT_NUM_FILE_SHARDS = 0;
+  static final int FILE_TRIGGERING_RECORD_COUNT = 50_000;
+
+  private final Coder<DestinationT> destinationCoder;
+
+  private final RecordWriterFactory<ElementT, DestinationT> 
recordWriterFactory;
+  private final TableFactory<String> tableFactory;
+
+  WriteToDestinations(
+      Coder<DestinationT> destinationCoder,
+      RecordWriterFactory<ElementT, DestinationT> recordWriterFactory,
+      TableFactory<String> tableFactory) {
+    this.destinationCoder = destinationCoder;
+    this.recordWriterFactory = recordWriterFactory;
+    this.tableFactory = tableFactory;
+  }
+
+  private PCollectionView<String> createJobIdPrefixView(Pipeline p) {
+
+    final String jobName = p.getOptions().getJobName();
+
+    return p.apply("JobIdCreationRoot_", Create.of((Void) null))
+        .apply(
+            "CreateJobId",
+            ParDo.of(
+                new DoFn<Void, String>() {
+                  @ProcessElement
+                  public void process(ProcessContext c) {
+                    c.output(jobName + "-" + UUID.randomUUID().toString());
+                  }
+                }))
+        .apply("JobIdSideInput", View.asSingleton());
+  }
+
+  @Override
+  public IcebergWriteResult<DestinationT, ElementT> expand(
+      PCollection<KV<DestinationT, ElementT>> input) {
+
+    final PCollectionView<String> fileView = 
createJobIdPrefixView(input.getPipeline());
+    // We always do the equivalent of a dynamically sharded file creation
+    TupleTag<Result<DestinationT>> writtenFilesTag = new 
TupleTag<>("writtenFiles");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> successfulWritesTag =
+        new TupleTag<>("successfulWrites");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> failedWritesTag =
+        new TupleTag<>("failedWrites");
+    TupleTag<KV<TableIdentifier, Snapshot>> snapshotsTag = new 
TupleTag<>("snapshots");
+
+    final Coder<ElementT> elementCoder =
+        ((KvCoder<DestinationT, ElementT>) input.getCoder()).getValueCoder();
+
+    // Write everything to files
+    PCollectionTuple writeBundlesToFiles =
+        input.apply(
+            "Write Bundles To Files",
+            ParDo.of(
+                    new WriteBundlesToFiles<>(
+                        fileView,
+                        successfulWritesTag,
+                        failedWritesTag,
+                        DEFAULT_MAX_WRITERS_PER_BUNDLE,
+                        DEFAULT_MAX_BYTES_PER_FILE,
+                        recordWriterFactory))
+                .withSideInputs(fileView)
+                .withOutputTags(
+                    writtenFilesTag,
+                    TupleTagList.of(ImmutableList.of(successfulWritesTag, 
failedWritesTag))));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> successfulWrites =
+        writeBundlesToFiles
+            .get(successfulWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> failedWrites =
+        writeBundlesToFiles
+            .get(failedWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<Result<DestinationT>> writtenFilesGrouped =
+        failedWrites
+            .apply("Group By Destination", GroupByKey.create())
+            .apply(
+                "Strip Shard ID",
+                MapElements.via(
+                    new SimpleFunction<
+                        KV<ShardedKey<DestinationT>, Iterable<ElementT>>,
+                        KV<DestinationT, Iterable<ElementT>>>() {
+                      @Override
+                      public KV<DestinationT, Iterable<ElementT>> apply(
+                          KV<ShardedKey<DestinationT>, Iterable<ElementT>> 
input11) {
+                        return KV.of(input11.getKey().getKey(), 
input11.getValue());
+                      }
+                    }))
+            .setCoder(KvCoder.of(destinationCoder, 
IterableCoder.of(elementCoder)))
+            .apply(
+                "Write Grouped Records",
+                ParDo.of(
+                    new WriteBundlesToFiles.WriteGroupedRecordsToFiles<>(
+                        fileView, DEFAULT_MAX_BYTES_PER_FILE, 
recordWriterFactory)))
+            .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder));
+
+    PCollection<Result<DestinationT>> catalogUpdates =
+        PCollectionList.of(
+                writeBundlesToFiles
+                    .get(writtenFilesTag)
+                    
.setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)))
+            .and(writtenFilesGrouped)
+            .apply("Flatten Files", Flatten.pCollections())
+            .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder));
+
+    // Apply any sharded writes and flatten everything for catalog updates
+    PCollection<KV<String, Snapshot>> snapshots =
+        catalogUpdates
+            .apply(
+                "Extract Data File",
+                ParDo.of(
+                    new DoFn<Result<DestinationT>, KV<String, 
MetadataUpdate>>() {
+                      @ProcessElement
+                      public void processElement(
+                          ProcessContext c, @Element Result<DestinationT> 
element) {
+                        c.output(
+                            KV.of(
+                                element.tableId,
+                                new MetadataUpdate(
+                                    element.partitionSpec.partitionType(),
+                                    element.update.getDataFiles(),
+                                    Collections.emptyList())));
+                      }
+                    }))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), MetadataUpdate.coder()))
+            .apply(GroupByKey.create())
+            .apply("Write Metadata Updates", ParDo.of(new 
MetadataUpdates<>(tableFactory)))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), 
SerializableCoder.of(Snapshot.class)));
+
+    return new IcebergWriteResult<>(
+        input.getPipeline(),
+        successfulWrites,
+        catalogUpdates,
+        snapshots,
+        successfulWritesTag,
+        writtenFilesTag,
+        snapshotsTag);
+  }
+
+  public enum WriteFormat {
+    AVRO,
+    PARQUET,
+    ORC
+  }
+
+  public static class MetadataUpdates<IdentifierT>

Review Comment:
   Done, but I still need to refactor this out anyhow.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Collections;
+import java.util.UUID;
+import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.ShardedKeyCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.ShardedKey;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+class WriteToDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, 
IcebergWriteResult<DestinationT, ElementT>> {
+
+  @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20;

Review Comment:
   I have no idea. This number `20` must be just a guess. Some of the others 
appear to be BigQuery quota limitations that we can just ignore. One thing that 
we should do is that I read a lot online about ideal iceberg file size being 
512mb (that's what some internal iceberg code does I guess) so perhaps we 
follow that. I'm still learning the iceberg Java APIs and the best way to use 
their best practices.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Arrays;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableBiFunction;
+import org.apache.beam.sdk.transforms.SerializableFunctions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.Record;
+import org.checkerframework.checker.nullness.qual.NonNull;
+
+public class IcebergIO {
+
+  public static <ElementT, DestinationT> Write<ElementT, DestinationT> 
writeToDestinations(
+      IcebergCatalog catalog,
+      DynamicDestinations<ElementT, DestinationT> dynamicDestinations,

Review Comment:
   I left them in a little bit for abstraction, but it can be an implementation 
detail and `IcebergIO.writeToDestinations(...)` can just take the string 
pattern. I haven't done that part yet. I was mostly getting the main body of 
the transform to only do Rows



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Collections;
+import java.util.UUID;
+import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.ShardedKeyCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.ShardedKey;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+class WriteToDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, 
IcebergWriteResult<DestinationT, ElementT>> {
+
+  @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20;
+  @VisibleForTesting static final int DEFAULT_MAX_FILES_PER_PARTITION = 10_000;
+  @VisibleForTesting static final long DEFAULT_MAX_BYTES_PER_PARTITION = 10L * 
(1L << 40); // 10TB
+  static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB
+  static final int DEFAULT_NUM_FILE_SHARDS = 0;
+  static final int FILE_TRIGGERING_RECORD_COUNT = 50_000;
+
+  private final Coder<DestinationT> destinationCoder;
+
+  private final RecordWriterFactory<ElementT, DestinationT> 
recordWriterFactory;
+  private final TableFactory<String> tableFactory;
+
+  WriteToDestinations(
+      Coder<DestinationT> destinationCoder,
+      RecordWriterFactory<ElementT, DestinationT> recordWriterFactory,
+      TableFactory<String> tableFactory) {
+    this.destinationCoder = destinationCoder;
+    this.recordWriterFactory = recordWriterFactory;
+    this.tableFactory = tableFactory;
+  }
+
+  private PCollectionView<String> createJobIdPrefixView(Pipeline p) {
+
+    final String jobName = p.getOptions().getJobName();
+
+    return p.apply("JobIdCreationRoot_", Create.of((Void) null))
+        .apply(
+            "CreateJobId",
+            ParDo.of(
+                new DoFn<Void, String>() {
+                  @ProcessElement
+                  public void process(ProcessContext c) {
+                    c.output(jobName + "-" + UUID.randomUUID().toString());
+                  }
+                }))
+        .apply("JobIdSideInput", View.asSingleton());
+  }
+
+  @Override
+  public IcebergWriteResult<DestinationT, ElementT> expand(
+      PCollection<KV<DestinationT, ElementT>> input) {
+
+    final PCollectionView<String> fileView = 
createJobIdPrefixView(input.getPipeline());
+    // We always do the equivalent of a dynamically sharded file creation
+    TupleTag<Result<DestinationT>> writtenFilesTag = new 
TupleTag<>("writtenFiles");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> successfulWritesTag =
+        new TupleTag<>("successfulWrites");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> failedWritesTag =
+        new TupleTag<>("failedWrites");
+    TupleTag<KV<TableIdentifier, Snapshot>> snapshotsTag = new 
TupleTag<>("snapshots");
+
+    final Coder<ElementT> elementCoder =
+        ((KvCoder<DestinationT, ElementT>) input.getCoder()).getValueCoder();
+
+    // Write everything to files
+    PCollectionTuple writeBundlesToFiles =
+        input.apply(
+            "Write Bundles To Files",
+            ParDo.of(
+                    new WriteBundlesToFiles<>(
+                        fileView,
+                        successfulWritesTag,
+                        failedWritesTag,
+                        DEFAULT_MAX_WRITERS_PER_BUNDLE,
+                        DEFAULT_MAX_BYTES_PER_FILE,
+                        recordWriterFactory))
+                .withSideInputs(fileView)
+                .withOutputTags(
+                    writtenFilesTag,
+                    TupleTagList.of(ImmutableList.of(successfulWritesTag, 
failedWritesTag))));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> successfulWrites =
+        writeBundlesToFiles
+            .get(successfulWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> failedWrites =

Review Comment:
   I just left it out for now.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Arrays;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableBiFunction;
+import org.apache.beam.sdk.transforms.SerializableFunctions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.Record;
+import org.checkerframework.checker.nullness.qual.NonNull;
+
+public class IcebergIO {
+
+  public static <ElementT, DestinationT> Write<ElementT, DestinationT> 
writeToDestinations(
+      IcebergCatalog catalog,
+      DynamicDestinations<ElementT, DestinationT> dynamicDestinations,
+      SerializableBiFunction<Record, ElementT, Record> toRecord) {
+    return new Write<>(catalog, dynamicDestinations, toRecord);
+  }
+
+  public static TableFactory<String> forCatalog(final IcebergCatalog catalog) {
+    return new TableFactory<String>() {
+      @Override
+      public Table getTable(String id) {
+        TableIdentifier tableId = TableIdentifier.parse(id);
+        // If the first element in the namespace is our catalog, remove that.
+        if (tableId.hasNamespace()) {
+          Namespace ns = tableId.namespace();
+          if (catalog.catalog().name().equals(ns.level(0))) {
+            String[] levels = ns.levels();
+            @SuppressWarnings("nullness") // we know that copyOfRange will not 
do any padding
+            @NonNull
+            String[] levelsMinusFirst = Arrays.copyOfRange(levels, 1, 
levels.length);
+            tableId = TableIdentifier.of(Namespace.of(levelsMinusFirst), 
tableId.name());
+          }
+        }
+        return catalog.catalog().loadTable(tableId);
+      }
+    };
+  }
+
+  public static class Write<ElementT, DestinationT>

Review Comment:
   Done (and even simpler)



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Collections;
+import java.util.UUID;
+import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.ShardedKeyCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.ShardedKey;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+class WriteToDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, 
IcebergWriteResult<DestinationT, ElementT>> {
+
+  @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20;
+  @VisibleForTesting static final int DEFAULT_MAX_FILES_PER_PARTITION = 10_000;
+  @VisibleForTesting static final long DEFAULT_MAX_BYTES_PER_PARTITION = 10L * 
(1L << 40); // 10TB
+  static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB
+  static final int DEFAULT_NUM_FILE_SHARDS = 0;
+  static final int FILE_TRIGGERING_RECORD_COUNT = 50_000;
+
+  private final Coder<DestinationT> destinationCoder;
+
+  private final RecordWriterFactory<ElementT, DestinationT> 
recordWriterFactory;
+  private final TableFactory<String> tableFactory;
+
+  WriteToDestinations(
+      Coder<DestinationT> destinationCoder,
+      RecordWriterFactory<ElementT, DestinationT> recordWriterFactory,
+      TableFactory<String> tableFactory) {
+    this.destinationCoder = destinationCoder;
+    this.recordWriterFactory = recordWriterFactory;
+    this.tableFactory = tableFactory;
+  }
+
+  private PCollectionView<String> createJobIdPrefixView(Pipeline p) {
+
+    final String jobName = p.getOptions().getJobName();
+
+    return p.apply("JobIdCreationRoot_", Create.of((Void) null))
+        .apply(
+            "CreateJobId",
+            ParDo.of(
+                new DoFn<Void, String>() {
+                  @ProcessElement
+                  public void process(ProcessContext c) {
+                    c.output(jobName + "-" + UUID.randomUUID().toString());
+                  }
+                }))
+        .apply("JobIdSideInput", View.asSingleton());
+  }
+
+  @Override
+  public IcebergWriteResult<DestinationT, ElementT> expand(
+      PCollection<KV<DestinationT, ElementT>> input) {
+
+    final PCollectionView<String> fileView = 
createJobIdPrefixView(input.getPipeline());
+    // We always do the equivalent of a dynamically sharded file creation
+    TupleTag<Result<DestinationT>> writtenFilesTag = new 
TupleTag<>("writtenFiles");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> successfulWritesTag =
+        new TupleTag<>("successfulWrites");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> failedWritesTag =
+        new TupleTag<>("failedWrites");
+    TupleTag<KV<TableIdentifier, Snapshot>> snapshotsTag = new 
TupleTag<>("snapshots");
+
+    final Coder<ElementT> elementCoder =
+        ((KvCoder<DestinationT, ElementT>) input.getCoder()).getValueCoder();
+
+    // Write everything to files
+    PCollectionTuple writeBundlesToFiles =
+        input.apply(
+            "Write Bundles To Files",
+            ParDo.of(
+                    new WriteBundlesToFiles<>(
+                        fileView,
+                        successfulWritesTag,
+                        failedWritesTag,
+                        DEFAULT_MAX_WRITERS_PER_BUNDLE,
+                        DEFAULT_MAX_BYTES_PER_FILE,
+                        recordWriterFactory))
+                .withSideInputs(fileView)
+                .withOutputTags(
+                    writtenFilesTag,
+                    TupleTagList.of(ImmutableList.of(successfulWritesTag, 
failedWritesTag))));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> successfulWrites =
+        writeBundlesToFiles
+            .get(successfulWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> failedWrites =
+        writeBundlesToFiles
+            .get(failedWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<Result<DestinationT>> writtenFilesGrouped =
+        failedWrites
+            .apply("Group By Destination", GroupByKey.create())
+            .apply(
+                "Strip Shard ID",
+                MapElements.via(
+                    new SimpleFunction<
+                        KV<ShardedKey<DestinationT>, Iterable<ElementT>>,
+                        KV<DestinationT, Iterable<ElementT>>>() {
+                      @Override
+                      public KV<DestinationT, Iterable<ElementT>> apply(
+                          KV<ShardedKey<DestinationT>, Iterable<ElementT>> 
input11) {
+                        return KV.of(input11.getKey().getKey(), 
input11.getValue());
+                      }
+                    }))
+            .setCoder(KvCoder.of(destinationCoder, 
IterableCoder.of(elementCoder)))
+            .apply(
+                "Write Grouped Records",
+                ParDo.of(
+                    new WriteBundlesToFiles.WriteGroupedRecordsToFiles<>(
+                        fileView, DEFAULT_MAX_BYTES_PER_FILE, 
recordWriterFactory)))
+            .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder));
+
+    PCollection<Result<DestinationT>> catalogUpdates =
+        PCollectionList.of(
+                writeBundlesToFiles
+                    .get(writtenFilesTag)
+                    
.setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)))
+            .and(writtenFilesGrouped)
+            .apply("Flatten Files", Flatten.pCollections())
+            .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder));
+
+    // Apply any sharded writes and flatten everything for catalog updates
+    PCollection<KV<String, Snapshot>> snapshots =
+        catalogUpdates
+            .apply(
+                "Extract Data File",
+                ParDo.of(
+                    new DoFn<Result<DestinationT>, KV<String, 
MetadataUpdate>>() {
+                      @ProcessElement
+                      public void processElement(
+                          ProcessContext c, @Element Result<DestinationT> 
element) {
+                        c.output(
+                            KV.of(
+                                element.tableId,
+                                new MetadataUpdate(
+                                    element.partitionSpec.partitionType(),
+                                    element.update.getDataFiles(),
+                                    Collections.emptyList())));
+                      }
+                    }))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), MetadataUpdate.coder()))
+            .apply(GroupByKey.create())
+            .apply("Write Metadata Updates", ParDo.of(new 
MetadataUpdates<>(tableFactory)))

Review Comment:
   (unresolved)



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.io.iceberg;
+
+import static 
org.apache.beam.sdk.values.TypeDescriptors.extractFromTypeParameters;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+public abstract class DynamicDestinations<T, DestinationT> implements 
Serializable {

Review Comment:
   Gotcha. I removed actually all the logic and just do something extremely 
basic for now. I guess DLQ could be update-incompatible change so I better get 
that done really quick too.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.iceberg;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.checkerframework.dataflow.qual.Pure;
+
+@AutoValue
+public abstract class IcebergCatalog implements Serializable {
+
+  @Pure
+  public abstract String getName();
+
+  /* Core Properties */
+  @Pure
+  public abstract @Nullable String getIcebergCatalogType();
+
+  @Pure
+  public abstract @Nullable String getCatalogImplementation();
+
+  @Pure
+  public abstract @Nullable String getFileIOImplementation();
+
+  @Pure
+  public abstract @Nullable String getWarehouseLocation();
+
+  @Pure
+  public abstract @Nullable String getMetricsReporterImplementation();
+
+  /* Caching */
+  @Pure
+  public abstract boolean getCacheEnabled();
+
+  @Pure
+  public abstract boolean getCacheCaseSensitive();
+
+  @Pure
+  public abstract long getCacheExpirationIntervalMillis();
+
+  @Pure
+  public abstract boolean getIOManifestCacheEnabled();
+
+  @Pure
+  public abstract long getIOManifestCacheExpirationIntervalMillis();
+
+  @Pure
+  public abstract long getIOManifestCacheMaxTotalBytes();
+
+  @Pure
+  public abstract long getIOManifestCacheMaxContentLength();
+
+  @Pure
+  public abstract @Nullable String getUri();
+
+  @Pure
+  public abstract int getClientPoolSize();
+
+  @Pure
+  public abstract long getClientPoolEvictionIntervalMs();
+
+  @Pure
+  public abstract @Nullable String getClientPoolCacheKeys();
+
+  @Pure
+  public abstract @Nullable String getLockImplementation();
+
+  @Pure
+  public abstract long getLockHeartbeatIntervalMillis();
+
+  @Pure
+  public abstract long getLockHeartbeatTimeoutMillis();
+
+  @Pure
+  public abstract int getLockHeartbeatThreads();
+
+  @Pure
+  public abstract long getLockAcquireIntervalMillis();
+
+  @Pure
+  public abstract long getLockAcquireTimeoutMillis();
+
+  @Pure
+  public abstract @Nullable String getAppIdentifier();
+
+  @Pure
+  public abstract @Nullable String getUser();
+
+  @Pure
+  public abstract long getAuthSessionTimeoutMillis();
+
+  @Pure
+  public abstract @Nullable Configuration getConfiguration();

Review Comment:
   That makes sense. Leaving this unresolved as I did not get to this yet.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.io.iceberg;
+
+import java.util.Collections;
+import java.util.UUID;
+import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.ShardedKeyCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.ShardedKey;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+class WriteToDestinations<DestinationT, ElementT>
+    extends PTransform<
+        PCollection<KV<DestinationT, ElementT>>, 
IcebergWriteResult<DestinationT, ElementT>> {
+
+  @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20;
+  @VisibleForTesting static final int DEFAULT_MAX_FILES_PER_PARTITION = 10_000;
+  @VisibleForTesting static final long DEFAULT_MAX_BYTES_PER_PARTITION = 10L * 
(1L << 40); // 10TB
+  static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB
+  static final int DEFAULT_NUM_FILE_SHARDS = 0;
+  static final int FILE_TRIGGERING_RECORD_COUNT = 50_000;
+
+  private final Coder<DestinationT> destinationCoder;
+
+  private final RecordWriterFactory<ElementT, DestinationT> 
recordWriterFactory;
+  private final TableFactory<String> tableFactory;
+
+  WriteToDestinations(
+      Coder<DestinationT> destinationCoder,
+      RecordWriterFactory<ElementT, DestinationT> recordWriterFactory,
+      TableFactory<String> tableFactory) {
+    this.destinationCoder = destinationCoder;
+    this.recordWriterFactory = recordWriterFactory;
+    this.tableFactory = tableFactory;
+  }
+
+  private PCollectionView<String> createJobIdPrefixView(Pipeline p) {
+
+    final String jobName = p.getOptions().getJobName();
+
+    return p.apply("JobIdCreationRoot_", Create.of((Void) null))
+        .apply(
+            "CreateJobId",
+            ParDo.of(
+                new DoFn<Void, String>() {
+                  @ProcessElement
+                  public void process(ProcessContext c) {
+                    c.output(jobName + "-" + UUID.randomUUID().toString());
+                  }
+                }))
+        .apply("JobIdSideInput", View.asSingleton());
+  }
+
+  @Override
+  public IcebergWriteResult<DestinationT, ElementT> expand(
+      PCollection<KV<DestinationT, ElementT>> input) {
+
+    final PCollectionView<String> fileView = 
createJobIdPrefixView(input.getPipeline());
+    // We always do the equivalent of a dynamically sharded file creation
+    TupleTag<Result<DestinationT>> writtenFilesTag = new 
TupleTag<>("writtenFiles");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> successfulWritesTag =
+        new TupleTag<>("successfulWrites");
+    TupleTag<KV<ShardedKey<DestinationT>, ElementT>> failedWritesTag =
+        new TupleTag<>("failedWrites");
+    TupleTag<KV<TableIdentifier, Snapshot>> snapshotsTag = new 
TupleTag<>("snapshots");
+
+    final Coder<ElementT> elementCoder =
+        ((KvCoder<DestinationT, ElementT>) input.getCoder()).getValueCoder();
+
+    // Write everything to files
+    PCollectionTuple writeBundlesToFiles =
+        input.apply(
+            "Write Bundles To Files",
+            ParDo.of(
+                    new WriteBundlesToFiles<>(
+                        fileView,
+                        successfulWritesTag,
+                        failedWritesTag,
+                        DEFAULT_MAX_WRITERS_PER_BUNDLE,
+                        DEFAULT_MAX_BYTES_PER_FILE,
+                        recordWriterFactory))
+                .withSideInputs(fileView)
+                .withOutputTags(
+                    writtenFilesTag,
+                    TupleTagList.of(ImmutableList.of(successfulWritesTag, 
failedWritesTag))));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> successfulWrites =
+        writeBundlesToFiles
+            .get(successfulWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<KV<ShardedKey<DestinationT>, ElementT>> failedWrites =
+        writeBundlesToFiles
+            .get(failedWritesTag)
+            .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), 
elementCoder));
+
+    PCollection<Result<DestinationT>> writtenFilesGrouped =
+        failedWrites

Review Comment:
   I have now totally refactored this and renamed everything. Thanks for your 
description; it helped a lot to understand how to organize it.



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