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


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java:
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.Row;
+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.base.Preconditions;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A writer that manages multiple {@link RecordWriter}s to write to multiple 
tables and partitions.
+ * Assigns one {@link DestinationState} per windowed destination. A {@link 
DestinationState} assigns
+ * one writer per partition in table destination. If the Iceberg {@link Table} 
is un-partitioned,
+ * the data is written normally using one {@link RecordWriter} (i.e. the 
{@link DestinationState}
+ * has one writer). At any given moment, the number of open data writers 
should be less than or
+ * equal to the number of total partitions (across all windowed destinations).
+ *
+ * <p>A {@link DestinationState} maintains its writers in a {@link Cache}. If 
a {@link RecordWriter}
+ * is inactive for 1 minute, the {@link DestinationState} will automatically 
close it to free up
+ * resources. Calling {@link #close()} on this {@link RecordWriterManager} 
will do the following for
+ * each {@link DestinationState}:
+ *
+ * <ol>
+ *   <li>Close all underlying {@link RecordWriter}s
+ *   <li>Collect all {@link DataFile}s
+ *   <li>Create a new {@link ManifestFile} referencing these {@link DataFile}s
+ * </ol>
+ *
+ * <p>After closing, the resulting {@link ManifestFile}s can be retrieved 
using {@link
+ * #getManifestFiles()}.
+ */
+class RecordWriterManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RecordWriterManager.class);
+
+  /**
+   * Represents the state of one Iceberg table destination. Creates one {@link 
RecordWriter} per
+   * partition and manages them in a {@link Cache}.
+   *
+   * <p>On closing, each writer's output {@link DataFile} is collected.
+   */
+  class DestinationState {
+    private final IcebergDestination icebergDestination;
+    private final PartitionSpec spec;
+    private final org.apache.iceberg.Schema schema;
+    private final PartitionKey partitionKey;
+    private final String tableLocation;
+    private final FileIO fileIO;
+    private final String stateToken = UUID.randomUUID().toString();
+    private final List<DataFile> dataFiles = Lists.newArrayList();
+    @VisibleForTesting final Cache<PartitionKey, RecordWriter> writers;
+    @VisibleForTesting final Map<PartitionKey, Integer> writerCounts = 
Maps.newHashMap();
+
+    DestinationState(IcebergDestination icebergDestination, Table table) {
+      this.icebergDestination = icebergDestination;
+      this.schema = table.schema();
+      this.spec = table.spec();
+      this.partitionKey = new PartitionKey(spec, schema);
+      this.tableLocation = table.location();
+      this.fileIO = table.io();
+
+      // build a cache of RecordWriters.
+      // writers will expire after 1 min of idle time.
+      // when a writer expires, its data file is collected.
+      this.writers =
+          CacheBuilder.newBuilder()
+              .expireAfterAccess(1, TimeUnit.MINUTES)
+              .removalListener(
+                  (RemovalNotification<PartitionKey, RecordWriter> removal) -> 
{
+                    final RecordWriter recordWriter =
+                        Preconditions.checkNotNull(removal.getValue());
+                    try {
+                      recordWriter.close();
+                      openWriters--;
+                      dataFiles.add(recordWriter.getDataFile());
+                    } catch (IOException e) {
+                      throw new RuntimeException(e);

Review Comment:
   This exception only comes from `recordWriter.close()` right? Perhaps narrow 
the scope of the catch-and-rethrow to make this clear? Or if it could come from 
multiple lines, it would be best to catch them separate and add some 
information to the propagated exception.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java:
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.Row;
+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.base.Preconditions;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A writer that manages multiple {@link RecordWriter}s to write to multiple 
tables and partitions.
+ * Assigns one {@link DestinationState} per windowed destination. A {@link 
DestinationState} assigns
+ * one writer per partition in table destination. If the Iceberg {@link Table} 
is un-partitioned,
+ * the data is written normally using one {@link RecordWriter} (i.e. the 
{@link DestinationState}
+ * has one writer). At any given moment, the number of open data writers 
should be less than or
+ * equal to the number of total partitions (across all windowed destinations).
+ *
+ * <p>A {@link DestinationState} maintains its writers in a {@link Cache}. If 
a {@link RecordWriter}
+ * is inactive for 1 minute, the {@link DestinationState} will automatically 
close it to free up
+ * resources. Calling {@link #close()} on this {@link RecordWriterManager} 
will do the following for
+ * each {@link DestinationState}:
+ *
+ * <ol>
+ *   <li>Close all underlying {@link RecordWriter}s
+ *   <li>Collect all {@link DataFile}s
+ *   <li>Create a new {@link ManifestFile} referencing these {@link DataFile}s
+ * </ol>
+ *
+ * <p>After closing, the resulting {@link ManifestFile}s can be retrieved 
using {@link
+ * #getManifestFiles()}.
+ */
+class RecordWriterManager {

Review Comment:
   Based on the documentation, this might be accurately named 
`PartitionedRecordWriter` or something. That communicates better than "Manager" 
which could mean almost anything. And it seems it could be `Autocloseable` 
which would enable using it in try-with-resources blocks, no?



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java:
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.Row;
+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.base.Preconditions;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A writer that manages multiple {@link RecordWriter}s to write to multiple 
tables and partitions.
+ * Assigns one {@link DestinationState} per windowed destination. A {@link 
DestinationState} assigns
+ * one writer per partition in table destination. If the Iceberg {@link Table} 
is un-partitioned,
+ * the data is written normally using one {@link RecordWriter} (i.e. the 
{@link DestinationState}
+ * has one writer). At any given moment, the number of open data writers 
should be less than or
+ * equal to the number of total partitions (across all windowed destinations).
+ *
+ * <p>A {@link DestinationState} maintains its writers in a {@link Cache}. If 
a {@link RecordWriter}
+ * is inactive for 1 minute, the {@link DestinationState} will automatically 
close it to free up
+ * resources. Calling {@link #close()} on this {@link RecordWriterManager} 
will do the following for
+ * each {@link DestinationState}:
+ *
+ * <ol>
+ *   <li>Close all underlying {@link RecordWriter}s
+ *   <li>Collect all {@link DataFile}s
+ *   <li>Create a new {@link ManifestFile} referencing these {@link DataFile}s
+ * </ol>
+ *
+ * <p>After closing, the resulting {@link ManifestFile}s can be retrieved 
using {@link
+ * #getManifestFiles()}.
+ */
+class RecordWriterManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RecordWriterManager.class);
+
+  /**
+   * Represents the state of one Iceberg table destination. Creates one {@link 
RecordWriter} per
+   * partition and manages them in a {@link Cache}.
+   *
+   * <p>On closing, each writer's output {@link DataFile} is collected.
+   */
+  class DestinationState {
+    private final IcebergDestination icebergDestination;
+    private final PartitionSpec spec;
+    private final org.apache.iceberg.Schema schema;
+    private final PartitionKey partitionKey;
+    private final String tableLocation;
+    private final FileIO fileIO;
+    private final String stateToken = UUID.randomUUID().toString();
+    private final List<DataFile> dataFiles = Lists.newArrayList();
+    @VisibleForTesting final Cache<PartitionKey, RecordWriter> writers;
+    @VisibleForTesting final Map<PartitionKey, Integer> writerCounts = 
Maps.newHashMap();
+
+    DestinationState(IcebergDestination icebergDestination, Table table) {
+      this.icebergDestination = icebergDestination;
+      this.schema = table.schema();
+      this.spec = table.spec();
+      this.partitionKey = new PartitionKey(spec, schema);
+      this.tableLocation = table.location();
+      this.fileIO = table.io();
+
+      // build a cache of RecordWriters.
+      // writers will expire after 1 min of idle time.
+      // when a writer expires, its data file is collected.
+      this.writers =
+          CacheBuilder.newBuilder()
+              .expireAfterAccess(1, TimeUnit.MINUTES)
+              .removalListener(
+                  (RemovalNotification<PartitionKey, RecordWriter> removal) -> 
{

Review Comment:
   Readability nit: I think this lambda would actually be more readable as a 
named method, especially since it mutates instance fields.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java:
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.Row;
+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.base.Preconditions;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A writer that manages multiple {@link RecordWriter}s to write to multiple 
tables and partitions.
+ * Assigns one {@link DestinationState} per windowed destination. A {@link 
DestinationState} assigns
+ * one writer per partition in table destination. If the Iceberg {@link Table} 
is un-partitioned,
+ * the data is written normally using one {@link RecordWriter} (i.e. the 
{@link DestinationState}
+ * has one writer). At any given moment, the number of open data writers 
should be less than or
+ * equal to the number of total partitions (across all windowed destinations).
+ *
+ * <p>A {@link DestinationState} maintains its writers in a {@link Cache}. If 
a {@link RecordWriter}
+ * is inactive for 1 minute, the {@link DestinationState} will automatically 
close it to free up
+ * resources. Calling {@link #close()} on this {@link RecordWriterManager} 
will do the following for
+ * each {@link DestinationState}:
+ *
+ * <ol>
+ *   <li>Close all underlying {@link RecordWriter}s
+ *   <li>Collect all {@link DataFile}s
+ *   <li>Create a new {@link ManifestFile} referencing these {@link DataFile}s
+ * </ol>
+ *
+ * <p>After closing, the resulting {@link ManifestFile}s can be retrieved 
using {@link
+ * #getManifestFiles()}.
+ */
+class RecordWriterManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RecordWriterManager.class);
+
+  /**
+   * Represents the state of one Iceberg table destination. Creates one {@link 
RecordWriter} per
+   * partition and manages them in a {@link Cache}.
+   *
+   * <p>On closing, each writer's output {@link DataFile} is collected.
+   */
+  class DestinationState {
+    private final IcebergDestination icebergDestination;
+    private final PartitionSpec spec;
+    private final org.apache.iceberg.Schema schema;
+    private final PartitionKey partitionKey;
+    private final String tableLocation;
+    private final FileIO fileIO;
+    private final String stateToken = UUID.randomUUID().toString();
+    private final List<DataFile> dataFiles = Lists.newArrayList();
+    @VisibleForTesting final Cache<PartitionKey, RecordWriter> writers;
+    @VisibleForTesting final Map<PartitionKey, Integer> writerCounts = 
Maps.newHashMap();
+
+    DestinationState(IcebergDestination icebergDestination, Table table) {
+      this.icebergDestination = icebergDestination;
+      this.schema = table.schema();
+      this.spec = table.spec();
+      this.partitionKey = new PartitionKey(spec, schema);
+      this.tableLocation = table.location();
+      this.fileIO = table.io();
+
+      // build a cache of RecordWriters.
+      // writers will expire after 1 min of idle time.
+      // when a writer expires, its data file is collected.
+      this.writers =
+          CacheBuilder.newBuilder()
+              .expireAfterAccess(1, TimeUnit.MINUTES)
+              .removalListener(
+                  (RemovalNotification<PartitionKey, RecordWriter> removal) -> 
{
+                    final RecordWriter recordWriter =
+                        Preconditions.checkNotNull(removal.getValue());

Review Comment:
   Recommend using Beam's fork of `Preconditions` which raises correct 
exceptions and differentiates `IllegalStateException` and 
`IllegalArgumentException`. Guava is incorrect - it just raises NPE which 
should only be raised when the code in question has failed, not in response to 
bad data.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java:
##########
@@ -75,28 +84,35 @@ private org.apache.iceberg.catalog.Catalog getCatalog() {
       return catalog;
     }
 
-    private RecordWriter createWriter(IcebergDestination destination) throws 
IOException {
-      return new RecordWriter(getCatalog(), destination, "-" + 
UUID.randomUUID());
-    }
-
     @ProcessElement
     public void processElement(
-        ProcessContext c, @Element KV<ShardedKey<Row>, Iterable<Row>> element) 
throws Exception {
+        ProcessContext c,
+        @Element KV<ShardedKey<Row>, Iterable<Row>> element,
+        BoundedWindow window,
+        PaneInfo pane)
+        throws Exception {
 
       Row destMetadata = element.getKey().getKey();
       IcebergDestination destination = 
dynamicDestinations.instantiateDestination(destMetadata);
-      RecordWriter writer = createWriter(destination);
+      WindowedValue<IcebergDestination> windowedDestination =
+          WindowedValue.of(destination, window.maxTimestamp(), window, pane);
+      RecordWriterManager writer =
+          new RecordWriterManager(getCatalog(), filePrefix, maxFileSize, 
Integer.MAX_VALUE);

Review Comment:
   With above change to autocloseable and rename, everything up to `close()` 
becomes:
   
   ```
   try(PartitionedWriter writer = new PartitionedWriter(getCatalog(), 
filePrefix, maxFileSize, Integer.MAX_VALUE) {
      ...
   }
   ```
   
   Otherwise, you want a `try { ... } finally { close() }` pattern I think. But 
try-with-resources is easier to read and more robust.



-- 
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-unsubscr...@beam.apache.org

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

Reply via email to