Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static org.apache.beam.sdk.io.iceberg.RecordWriterManager.getPartitionDataPath;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.apache.beam.sdk.coders.IterableCoder;
Expand All @@ -31,23 +32,28 @@
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.WindowedValue;
import org.apache.beam.sdk.values.WindowedValues;
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.TableProperties;
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.apache.iceberg.util.PropertyUtil;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
Expand Down Expand Up @@ -110,7 +116,10 @@ public void setup() {

@ProcessElement
public void processElement(
@Element KV<Row, Iterable<Row>> element, OutputReceiver<FileWriteResult> out)
@Element KV<Row, Iterable<Row>> element,
BoundedWindow window,
PaneInfo paneInfo,
OutputReceiver<FileWriteResult> out)
throws Exception {
String tableIdentifier = checkStateNotNull(element.getKey().getString(DESTINATION));
String partitionPath = checkStateNotNull(element.getKey().getString(PARTITION));
Expand All @@ -130,24 +139,32 @@ public void processElement(
destination
.getFileFormat()
.addExtension(String.format("%s-%s", filePrefix, UUID.randomUUID()));

RecordWriter writer =
new RecordWriter(table, destination.getFileFormat(), fileName, partitionData);
try {
System.out.println(partitionData + fileName);
Comment thread
ahmedabu98 marked this conversation as resolved.
Outdated

long maxFileSize =
PropertyUtil.propertyAsLong(
table.properties(),
TableProperties.WRITE_TARGET_FILE_SIZE_BYTES,
TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT);
WindowedValue<IcebergDestination> windowedDestination =
WindowedValues.of(destination, window.maxTimestamp(), window, paneInfo);
RecordWriterManager writer =
new RecordWriterManager(catalogConfig, filePrefix, maxFileSize, Integer.MAX_VALUE);
try (writer) {
for (Row row : element.getValue()) {
Record record = IcebergUtils.beamRowToIcebergRecord(table.schema(), row);
writer.write(record);
writer.write(windowedDestination, row);
}
} finally {
writer.close();
}
Comment thread
ahmedabu98 marked this conversation as resolved.

SerializableDataFile sdf = SerializableDataFile.from(writer.getDataFile(), partitionPath);
out.output(
FileWriteResult.builder()
.setTableIdentifier(destination.getTableIdentifier())
.setSerializableDataFile(sdf)
.build());
List<SerializableDataFile> serializableDataFiles =
checkStateNotNull(writer.getSerializableDataFiles().get(windowedDestination));
for (SerializableDataFile dataFile : serializableDataFiles) {
out.output(
FileWriteResult.builder()
.setTableIdentifier(destination.getTableIdentifier())
.setSerializableDataFile(dataFile)
.build());
}
Comment thread
ahmedabu98 marked this conversation as resolved.
}

private Map<String, PartitionField> getPartitionFieldMap(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,9 +58,11 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DistributionMode;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.data.IcebergGenerics;
import org.apache.iceberg.data.Record;
Expand Down Expand Up @@ -491,6 +493,56 @@ public void writePartitionedData(boolean autosharding) {
p.run();
}

@Test
public void testWritingPartitionedDataRespectsFileSizePropertyWithSpillOver() {
Schema schema = Schema.builder().addStringField("str").addInt32Field("int").build();
org.apache.iceberg.Schema icebergSchema = IcebergUtils.beamSchemaToIcebergSchema(schema);
PartitionSpec spec = PartitionSpec.builderFor(icebergSchema).identity("str").build();
String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16);

Table table =
warehouse.createTable(
TableIdentifier.parse(identifier),
icebergSchema,
spec,
ImmutableMap.of(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "1"));
Map<String, Object> config =
ImmutableMap.of(
"table",
identifier,
"catalog_properties",
ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location),
"distribution_mode",
distributionMode.name());

List<Row> rows =
Arrays.asList(
Row.withSchema(schema).addValues("a", 1).build(),
Row.withSchema(schema).addValues("a", 2).build(),
Row.withSchema(schema).addValues("a", 3).build(),
Row.withSchema(schema).addValues("b", 1).build(),
Row.withSchema(schema).addValues("b", 2).build(),
Row.withSchema(schema).addValues("b", 3).build());

testPipeline
.apply("Records To Add", Create.of(rows))
.setRowSchema(schema)
.apply(Managed.write(Managed.ICEBERG).withConfig(config));
testPipeline.run().waitUntilFinish();

Pipeline p = Pipeline.create(TestPipeline.testingPipelineOptions());
PCollection<Row> readRows =
p.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection();
PAssert.that(readRows).containsInAnyOrder(rows);

table.refresh();
List<DataFile> datafiles = new ArrayList<>();
table.snapshots().forEach(s -> s.addedDataFiles(table.io()).forEach(datafiles::add));
assertEquals(6, datafiles.size());

p.run();
}

@Test
public void testWriteCreateTableWithPartitionSpec() {
String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16);
Expand Down
Loading