chamikaramj commented on code in PR #38061:
URL: https://github.com/apache/beam/pull/38061#discussion_r3134725516
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java:
##########
@@ -443,6 +451,15 @@ public WriteRows withDirectWriteByteLimit(Integer
directWriteByteLimit) {
return toBuilder().setDirectWriteByteLimit(directWriteByteLimit).build();
}
+ /**
+ * Groups incoming rows by partition before sending to writes, ensuring
that a given bundle is
Review Comment:
Probably summarize the three potential values here and mention the default.
https://github.com/apache/iceberg/blob/8f611675e343c845c9df6e56718f1492aeb143e0/api/src/main/java/org/apache/iceberg/DistributionMode.java#L39
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java:
##########
@@ -464,15 +481,31 @@ public IcebergWriteResult expand(PCollection<Row> input) {
IcebergUtils.isUnbounded(input),
"Must only provide direct write limit for unbounded pipelines.");
}
- return input
- .apply("Assign Table Destinations", new
AssignDestinations(destinations))
- .apply(
- "Write Rows to Destinations",
- new WriteToDestinations(
- getCatalogConfig(),
- destinations,
- getTriggeringFrequency(),
- getDirectWriteByteLimit()));
+
+ switch (getDistributionMode()) {
+ case NONE:
Review Comment:
We don't support "RANGE" ?
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/BeamRowWrapper.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.Date;
+import org.apache.beam.sdk.schemas.logicaltypes.DateTime;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
+import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant;
+import org.apache.beam.sdk.schemas.logicaltypes.Time;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.UUIDUtil;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+public class BeamRowWrapper implements StructLike {
+
+ private final FieldType[] types;
+ private final @Nullable PositionalGetter<?>[] getters;
+ private @Nullable Row row = null;
+
+ public BeamRowWrapper(Schema schema, Types.StructType struct) {
+ int size = schema.getFieldCount();
+
+ types = (FieldType[]) Array.newInstance(FieldType.class, size);
+ getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class,
size);
+
+ for (int i = 0; i < size; i++) {
+ types[i] = schema.getField(i).getType();
+ getters[i] = buildGetter(types[i], struct.fields().get(i).type());
+ }
+ }
+
+ public BeamRowWrapper wrap(@Nullable Row row) {
+ this.row = row;
+ return this;
+ }
+
+ @Override
+ public int size() {
+ return types.length;
+ }
+
+ @Override
+ public <T> @Nullable T get(int pos, Class<T> javaClass) {
+ if (row == null || row.getValue(pos) == null) {
+ return null;
+ } else if (getters[pos] != null) {
+ return javaClass.cast(getters[pos].get(checkStateNotNull(row), pos));
+ }
+
+ return javaClass.cast(checkStateNotNull(row).getValue(pos));
+ }
+
+ @Override
+ public <T> void set(int pos, T value) {
+ throw new UnsupportedOperationException(
+ "Could not set a field in the BeamRowWrapper because rowData is
read-only");
+ }
+
+ private interface PositionalGetter<T> {
+ T get(Row data, int pos);
+ }
+
+ private static @Nullable PositionalGetter<?> buildGetter(FieldType beamType,
Type icebergType) {
Review Comment:
I think this also came up in a previous PR. Can you clarify why we would
need the wrapper class here ? Also is there a way to avoid having to iterate
through the set of types in multiple places ?
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/BeamRowWrapper.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.Date;
+import org.apache.beam.sdk.schemas.logicaltypes.DateTime;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
+import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant;
+import org.apache.beam.sdk.schemas.logicaltypes.Time;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.UUIDUtil;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+public class BeamRowWrapper implements StructLike {
Review Comment:
Please add documentation on why this class is needed and how to use it. A
BeamRowWrapper sounds like a more general utility that might have to live
outside Iceberg ?
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java:
##########
@@ -134,6 +135,12 @@ public static Builder builder() {
+ " please visit
https://iceberg.apache.org/docs/latest/configuration/#table-properties.")
public abstract @Nullable Map<String, String> getTableProperties();
+ @SchemaFieldDescription(
+ "Defines distribution of write data. Supported distributions:"
+ + "\n- none: don't shuffle rows (default)"
Review Comment:
Probably cleaner to make this setHashDistributionMode and make it a boolean ?
##########
sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java:
##########
@@ -687,6 +687,32 @@ public void testWriteToPartitionedTable() throws
IOException {
returnedRecords,
containsInAnyOrder(inputRows.stream().map(RECORD_FUNC::apply).toArray()));
}
+ @Test
+ public void testWriteToPartitionedTableWithHashDistribution() throws
IOException {
+ Map<String, Object> config = new
HashMap<>(managedIcebergConfig(tableId()));
+ int truncLength = "value_x".length();
+ List<String> partitionFields =
+ Arrays.asList("bool_field", "hour(datetime)", "truncate(str, " +
truncLength + ")");
+ config.put("partition_fields", partitionFields);
+ config.put("distribution_mode", "hash");
+ PCollection<Row> input =
pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA);
+ input.apply(Managed.write(ICEBERG).withConfig(config));
+ pipeline.run().waitUntilFinish();
+
+ // Read back and check records are correct
Review Comment:
Is it possible to add a test to confirm that less files get created when
distribution mode is set to HASH ?
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AssignDestinationsAndPartitions.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+
+/**
+ * Assigns destination metadata for each input record.
+ *
+ * <p>The output will have the format { {destination, partition}, data }
+ */
+class AssignDestinationsAndPartitions
+ extends PTransform<PCollection<Row>, PCollection<KV<Row, Row>>> {
+
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+ static final String DESTINATION = "destination";
+ static final String PARTITION = "partition";
+ static final org.apache.beam.sdk.schemas.Schema OUTPUT_SCHEMA =
+ org.apache.beam.sdk.schemas.Schema.builder()
+ .addStringField(DESTINATION)
+ .addStringField(PARTITION)
+ .build();
+
+ public AssignDestinationsAndPartitions(
+ DynamicDestinations dynamicDestinations, IcebergCatalogConfig
catalogConfig) {
+ this.dynamicDestinations = dynamicDestinations;
+ this.catalogConfig = catalogConfig;
+ }
+
+ @Override
+ public PCollection<KV<Row, Row>> expand(PCollection<Row> input) {
+ return input
+ .apply(ParDo.of(new AssignDoFn(dynamicDestinations, catalogConfig)))
+ .setCoder(
+ KvCoder.of(
+ RowCoder.of(OUTPUT_SCHEMA),
RowCoder.of(dynamicDestinations.getDataSchema())));
+ }
+
+ static class AssignDoFn extends DoFn<Row, KV<Row, Row>> {
+ private final Map<String, PartitionKey> partitionKeys = new HashMap<>();
+ private transient @MonotonicNonNull Map<String, BeamRowWrapper> wrappers;
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+
+ AssignDoFn(DynamicDestinations dynamicDestinations, IcebergCatalogConfig
catalogConfig) {
+ this.dynamicDestinations = dynamicDestinations;
+ this.catalogConfig = catalogConfig;
+ }
+
+ @Setup
+ public void setup() {
+ this.wrappers = new HashMap<>();
+ }
+
+ @ProcessElement
+ public void processElement(
+ @Element Row element,
+ BoundedWindow window,
+ PaneInfo paneInfo,
+ @Timestamp Instant timestamp,
+ OutputReceiver<KV<Row, Row>> out) {
+ String tableIdentifier =
+ dynamicDestinations.getTableStringIdentifier(
+ ValueInSingleWindow.of(element, timestamp, window, paneInfo));
+ Row data = dynamicDestinations.getData(element);
+
+ @Nullable PartitionKey partitionKey = partitionKeys.get(tableIdentifier);
+ @Nullable BeamRowWrapper wrapper =
checkStateNotNull(wrappers).get(tableIdentifier);
+ if (partitionKey == null || wrapper == null) {
+ PartitionSpec spec = PartitionSpec.unpartitioned();
+ Schema schema =
IcebergUtils.beamSchemaToIcebergSchema(data.getSchema());
+ @Nullable
+ IcebergTableCreateConfig createConfig =
+
dynamicDestinations.instantiateDestination(tableIdentifier).getTableCreateConfig();
+ if (createConfig != null && createConfig.getPartitionFields() != null)
{
+ spec =
+
PartitionUtils.toPartitionSpec(createConfig.getPartitionFields(),
data.getSchema());
+ } else {
+ try {
+ // see if table already exists with a spec
+ // TODO(ahmedabu98): improve this by periodically refreshing the
table to fetch updated
Review Comment:
Add a Github issue to the TODO ?
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WritePartitionedRowsToFiles.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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 static
org.apache.beam.sdk.io.iceberg.AssignDestinationsAndPartitions.DESTINATION;
+import static
org.apache.beam.sdk.io.iceberg.AssignDestinationsAndPartitions.PARTITION;
+import static
org.apache.beam.sdk.io.iceberg.RecordWriterManager.getPartitionDataPath;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.ShardedKey;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+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.collect.Maps;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class WritePartitionedRowsToFiles
+ extends PTransform<
+ PCollection<KV<ShardedKey<Row>, Iterable<Row>>>,
PCollection<FileWriteResult>> {
+ private static final Logger LOG =
LoggerFactory.getLogger(WritePartitionedRowsToFiles.class);
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+ private final String filePrefix;
+
+ WritePartitionedRowsToFiles(
+ IcebergCatalogConfig catalogConfig,
+ DynamicDestinations dynamicDestinations,
+ String filePrefix) {
+ this.catalogConfig = catalogConfig;
+ this.dynamicDestinations = dynamicDestinations;
+ this.filePrefix = filePrefix;
+ }
+
+ @Override
+ public PCollection<FileWriteResult> expand(
+ PCollection<KV<ShardedKey<Row>, Iterable<Row>>> input) {
+ Schema dataSchema =
+ ((RowCoder)
+ ((IterableCoder<Row>)
+ ((KvCoder<ShardedKey<Row>, Iterable<Row>>)
input.getCoder())
+ .getValueCoder())
+ .getElemCoder())
+ .getSchema();
+ return input.apply(
+ ParDo.of(new WriteDoFn(catalogConfig, dynamicDestinations, filePrefix,
dataSchema)));
+ }
+
+ private static class WriteDoFn extends DoFn<KV<ShardedKey<Row>,
Iterable<Row>>, FileWriteResult> {
+
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+ private final String filePrefix;
+ private final Schema dataSchema;
+ static final Cache<TableIdentifier, LastRefreshedTable>
LAST_REFRESHED_TABLE_CACHE =
+ CacheBuilder.newBuilder().expireAfterAccess(10,
TimeUnit.MINUTES).build();
+
+ WriteDoFn(
+ IcebergCatalogConfig catalogConfig,
+ DynamicDestinations dynamicDestinations,
+ String filePrefix,
+ Schema dataSchema) {
+ this.catalogConfig = catalogConfig;
+ this.dynamicDestinations = dynamicDestinations;
+ this.filePrefix = filePrefix;
+ this.dataSchema = dataSchema;
+ }
+
+ @ProcessElement
+ public void processElement(
+ @Element KV<ShardedKey<Row>, Iterable<Row>> element,
OutputReceiver<FileWriteResult> out)
+ throws Exception {
+ String tableIdentifier =
checkStateNotNull(element.getKey().getKey().getString(DESTINATION));
+ String partitionPath =
checkStateNotNull(element.getKey().getKey().getString(PARTITION));
+
+ IcebergDestination destination =
dynamicDestinations.instantiateDestination(tableIdentifier);
+ Table table = getOrCreateTable(destination, dataSchema);
+
+ // TODO(ahmedabu98): cache this
Review Comment:
Ditto.
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteToPartitions.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+
+import java.util.UUID;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.transforms.GroupIntoBatches;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.ShardedKey;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+
+class WriteToPartitions extends PTransform<PCollection<KV<Row, Row>>,
IcebergWriteResult> {
+ private static final long DEFAULT_BYTES_PER_FILE = (1L << 29); // 512mb
Review Comment:
This is known to be a good value for Iceberg ?
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WritePartitionedRowsToFiles.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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 static
org.apache.beam.sdk.io.iceberg.AssignDestinationsAndPartitions.DESTINATION;
+import static
org.apache.beam.sdk.io.iceberg.AssignDestinationsAndPartitions.PARTITION;
+import static
org.apache.beam.sdk.io.iceberg.RecordWriterManager.getPartitionDataPath;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.ShardedKey;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+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.collect.Maps;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class WritePartitionedRowsToFiles
+ extends PTransform<
+ PCollection<KV<ShardedKey<Row>, Iterable<Row>>>,
PCollection<FileWriteResult>> {
+ private static final Logger LOG =
LoggerFactory.getLogger(WritePartitionedRowsToFiles.class);
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+ private final String filePrefix;
+
+ WritePartitionedRowsToFiles(
+ IcebergCatalogConfig catalogConfig,
+ DynamicDestinations dynamicDestinations,
+ String filePrefix) {
+ this.catalogConfig = catalogConfig;
+ this.dynamicDestinations = dynamicDestinations;
+ this.filePrefix = filePrefix;
+ }
+
+ @Override
+ public PCollection<FileWriteResult> expand(
+ PCollection<KV<ShardedKey<Row>, Iterable<Row>>> input) {
+ Schema dataSchema =
+ ((RowCoder)
+ ((IterableCoder<Row>)
+ ((KvCoder<ShardedKey<Row>, Iterable<Row>>)
input.getCoder())
+ .getValueCoder())
+ .getElemCoder())
+ .getSchema();
+ return input.apply(
+ ParDo.of(new WriteDoFn(catalogConfig, dynamicDestinations, filePrefix,
dataSchema)));
+ }
+
+ private static class WriteDoFn extends DoFn<KV<ShardedKey<Row>,
Iterable<Row>>, FileWriteResult> {
+
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+ private final String filePrefix;
+ private final Schema dataSchema;
+ static final Cache<TableIdentifier, LastRefreshedTable>
LAST_REFRESHED_TABLE_CACHE =
+ CacheBuilder.newBuilder().expireAfterAccess(10,
TimeUnit.MINUTES).build();
+
+ WriteDoFn(
+ IcebergCatalogConfig catalogConfig,
+ DynamicDestinations dynamicDestinations,
+ String filePrefix,
+ Schema dataSchema) {
+ this.catalogConfig = catalogConfig;
+ this.dynamicDestinations = dynamicDestinations;
+ this.filePrefix = filePrefix;
+ this.dataSchema = dataSchema;
+ }
+
+ @ProcessElement
+ public void processElement(
+ @Element KV<ShardedKey<Row>, Iterable<Row>> element,
OutputReceiver<FileWriteResult> out)
+ throws Exception {
+ String tableIdentifier =
checkStateNotNull(element.getKey().getKey().getString(DESTINATION));
+ String partitionPath =
checkStateNotNull(element.getKey().getKey().getString(PARTITION));
+
+ IcebergDestination destination =
dynamicDestinations.instantiateDestination(tableIdentifier);
+ Table table = getOrCreateTable(destination, dataSchema);
+
+ // TODO(ahmedabu98): cache this
+ Map<String, PartitionField> partitionFieldMap = Maps.newHashMap();
+ for (PartitionField partitionField : table.spec().fields()) {
+ partitionFieldMap.put(partitionField.name(), partitionField);
+ }
+ partitionPath = getPartitionDataPath(partitionPath, partitionFieldMap);
+
+ StructLike partitionData =
+ table.spec().isPartitioned()
+ ? DataFiles.data(table.spec(), partitionPath)
+ : new PartitionKey(table.spec(), table.schema());
+
+ String fileName =
+ destination
+ .getFileFormat()
+ .addExtension(String.format("%s-%s", filePrefix,
UUID.randomUUID()));
+
+ RecordWriter writer =
+ new RecordWriter(table, destination.getFileFormat(), fileName,
partitionData);
+ for (Row row : element.getValue()) {
+ Record record = IcebergUtils.beamRowToIcebergRecord(table.schema(),
row);
+ writer.write(record);
+ }
+ writer.close();
+
+ SerializableDataFile sdf =
SerializableDataFile.from(writer.getDataFile(), partitionPath);
+ out.output(
+ FileWriteResult.builder()
+ .setTableIdentifier(destination.getTableIdentifier())
+ .setSerializableDataFile(sdf)
+ .build());
+ }
+
+ static final class LastRefreshedTable {
+ final Table table;
+ volatile Instant lastRefreshTime;
+ static final Duration STALENESS_THRESHOLD = Duration.ofMinutes(2);
+
+ LastRefreshedTable(Table table, Instant lastRefreshTime) {
+ this.table = table;
+ this.lastRefreshTime = lastRefreshTime;
+ }
+
+ /**
+ * Refreshes the table metadata if it is considered stale (older than 2
minutes).
+ *
+ * <p>This method first performs a non-synchronized check on the table's
freshness. This
+ * provides a lock-free fast path that avoids synchronization overhead
in the common case
+ * where the table does not need to be refreshed. If the table might be
stale, it then enters
+ * a synchronized block to ensure that only one thread performs the
refresh operation.
+ */
+ void refreshIfStale() {
+ // Fast path: Avoid entering the synchronized block if the table is
not stale.
+ if (lastRefreshTime.isAfter(Instant.now().minus(STALENESS_THRESHOLD)))
{
+ return;
+ }
+ synchronized (this) {
+ if
(lastRefreshTime.isBefore(Instant.now().minus(STALENESS_THRESHOLD))) {
+ table.refresh();
+ lastRefreshTime = Instant.now();
+ }
+ }
+ }
+ }
+
+ Table getOrCreateTable(IcebergDestination destination, Schema dataSchema) {
+ Catalog catalog = catalogConfig.catalog();
+ TableIdentifier identifier = destination.getTableIdentifier();
+ @Nullable
+ LastRefreshedTable lastRefreshedTable =
LAST_REFRESHED_TABLE_CACHE.getIfPresent(identifier);
+ if (lastRefreshedTable != null && lastRefreshedTable.table != null) {
+ lastRefreshedTable.refreshIfStale();
+ return lastRefreshedTable.table;
+ }
+
+ Namespace namespace = identifier.namespace();
+ @Nullable IcebergTableCreateConfig createConfig =
destination.getTableCreateConfig();
+ PartitionSpec partitionSpec =
+ createConfig != null ? createConfig.getPartitionSpec() :
PartitionSpec.unpartitioned();
+ Map<String, String> tableProperties =
+ createConfig != null && createConfig.getTableProperties() != null
+ ? createConfig.getTableProperties()
+ : Maps.newHashMap();
+
+ @Nullable Table table = null;
+ synchronized (LAST_REFRESHED_TABLE_CACHE) {
+ // Create namespace if it does not exist yet
+ if (!namespace.isEmpty() && catalog instanceof SupportsNamespaces) {
+ SupportsNamespaces supportsNamespaces = (SupportsNamespaces) catalog;
+ if (!supportsNamespaces.namespaceExists(namespace)) {
+ try {
+ supportsNamespaces.createNamespace(namespace);
+ LOG.info("Created new namespace '{}'.", namespace);
+ } catch (AlreadyExistsException ignored) {
+ // race condition: another worker already created this namespace
Review Comment:
Let's at least log it.
##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AssignDestinationsAndPartitions.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+
+/**
+ * Assigns destination metadata for each input record.
+ *
+ * <p>The output will have the format { {destination, partition}, data }
+ */
+class AssignDestinationsAndPartitions
+ extends PTransform<PCollection<Row>, PCollection<KV<Row, Row>>> {
+
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+ static final String DESTINATION = "destination";
+ static final String PARTITION = "partition";
+ static final org.apache.beam.sdk.schemas.Schema OUTPUT_SCHEMA =
+ org.apache.beam.sdk.schemas.Schema.builder()
+ .addStringField(DESTINATION)
+ .addStringField(PARTITION)
+ .build();
+
+ public AssignDestinationsAndPartitions(
+ DynamicDestinations dynamicDestinations, IcebergCatalogConfig
catalogConfig) {
+ this.dynamicDestinations = dynamicDestinations;
+ this.catalogConfig = catalogConfig;
+ }
+
+ @Override
+ public PCollection<KV<Row, Row>> expand(PCollection<Row> input) {
+ return input
+ .apply(ParDo.of(new AssignDoFn(dynamicDestinations, catalogConfig)))
+ .setCoder(
+ KvCoder.of(
+ RowCoder.of(OUTPUT_SCHEMA),
RowCoder.of(dynamicDestinations.getDataSchema())));
+ }
+
+ static class AssignDoFn extends DoFn<Row, KV<Row, Row>> {
+ private final Map<String, PartitionKey> partitionKeys = new HashMap<>();
+ private transient @MonotonicNonNull Map<String, BeamRowWrapper> wrappers;
+ private final DynamicDestinations dynamicDestinations;
+ private final IcebergCatalogConfig catalogConfig;
+
+ AssignDoFn(DynamicDestinations dynamicDestinations, IcebergCatalogConfig
catalogConfig) {
+ this.dynamicDestinations = dynamicDestinations;
+ this.catalogConfig = catalogConfig;
+ }
+
+ @Setup
+ public void setup() {
+ this.wrappers = new HashMap<>();
+ }
+
+ @ProcessElement
+ public void processElement(
+ @Element Row element,
+ BoundedWindow window,
+ PaneInfo paneInfo,
+ @Timestamp Instant timestamp,
+ OutputReceiver<KV<Row, Row>> out) {
+ String tableIdentifier =
+ dynamicDestinations.getTableStringIdentifier(
+ ValueInSingleWindow.of(element, timestamp, window, paneInfo));
+ Row data = dynamicDestinations.getData(element);
+
+ @Nullable PartitionKey partitionKey = partitionKeys.get(tableIdentifier);
+ @Nullable BeamRowWrapper wrapper =
checkStateNotNull(wrappers).get(tableIdentifier);
+ if (partitionKey == null || wrapper == null) {
+ PartitionSpec spec = PartitionSpec.unpartitioned();
Review Comment:
Is this guaranteed to be unique across bundles ?
--
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]