Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 merged PR #32102: URL: https://github.com/apache/beam/pull/32102 -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1717091763 ## sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java: ## @@ -17,15 +17,12 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamRowToIcebergRecord; Review Comment: done -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
kennknowles commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716894956 ## 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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * After closing, the resulting {@link ManifestFile}s can be retrieved using {@link + * #getManifestFiles()}. + */ +class RecordWriterManager { Review Comment: Fair enough. A reason I don't like the "Manager" name is that I don't know what it is doing in the managing, and it really doesn't communicate that this is a thing that does the writing. We have so many things called "manager" and none of them have anything in common. -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
kennknowles commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716892836 ## 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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * 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}. + * + * 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 dataFiles = Lists.newArrayList(); +@VisibleForTesting final Cache writers; +@VisibleForTesting final Map 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(); +
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
liferoad commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716733416 ## sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java: ## @@ -17,15 +17,12 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamRowToIcebergRecord; Review Comment: can we add this to CHANGES.md to call it out? -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716261018 ## 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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * 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}. + * + * 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 dataFiles = Lists.newArrayList(); +@VisibleForTesting final Cache writers; +@VisibleForTesting final Map 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(); +
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716250544 ## 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, Iterable> element) throws Exception { +ProcessContext c, +@Element KV, Iterable> element, +BoundedWindow window, +PaneInfo pane) +throws Exception { Row destMetadata = element.getKey().getKey(); IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); - RecordWriter writer = createWriter(destination); + WindowedValue windowedDestination = + WindowedValue.of(destination, window.maxTimestamp(), window, pane); + RecordWriterManager writer = + new RecordWriterManager(getCatalog(), filePrefix, maxFileSize, Integer.MAX_VALUE); Review Comment: Done -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716250430 ## 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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * After closing, the resulting {@link ManifestFile}s can be retrieved using {@link + * #getManifestFiles()}. + */ +class RecordWriterManager { Review Comment: I originally had it as `PartitionedRecordWriter` but thought it may be misleading since the class is also used for unpartitioned writes Added Autocloseable implementation -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716250187 ## 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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * 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}. + * + * 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 dataFiles = Lists.newArrayList(); +@VisibleForTesting final Cache writers; +@VisibleForTesting final Map 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(); +
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on code in PR #32102: URL: https://github.com/apache/beam/pull/32102#discussion_r1716250044 ## 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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * 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}. + * + * 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 dataFiles = Lists.newArrayList(); +@VisibleForTesting final Cache writers; +@VisibleForTesting final Map 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(); +
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
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). + * + * 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}: + * + * + * Close all underlying {@link RecordWriter}s + * Collect all {@link DataFile}s + * Create a new {@link ManifestFile} referencing these {@link DataFile}s + * + * + * 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}. + * + * 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 dataFiles = Lists.newArrayList(); +@VisibleForTesting final Cache writers; +@VisibleForTesting final Map 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(); +
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on PR #32102: URL: https://github.com/apache/beam/pull/32102#issuecomment-2287472903 CC: @arthurpessoa, @chamikaramj -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
github-actions[bot] commented on PR #32102: URL: https://github.com/apache/beam/pull/32102#issuecomment-2287390706 Assigning reviewers. If you would like to opt out of this review, comment `assign to next reviewer`: R: @m-trieu for label java. R: @damccorm for label build. R: @Abacn for label io. Available commands: - `stop reviewer notifications` - opt out of the automated review tooling - `remind me after tests pass` - tag the comment author after tests pass - `waiting on author` - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers) The PR bot will only process comments in the main thread (not review comments). -- 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
Re: [PR] [Managed Iceberg] Support writing to partitioned tables [beam]
ahmedabu98 commented on PR #32102: URL: https://github.com/apache/beam/pull/32102#issuecomment-2287389712 assign set of reviewers -- 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