Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit c9ad32e

Browse files
authored
[Managed Iceberg] Support writing to partitioned tables (#32102)
* support writing partitioned data * trigger integration tests * partitioned record writer to manage writers for different partitions * partitioned record writer * reject rows when we are saturated with record writers * refactor record writer manager * add tests * add more tests * make record writer manager transient * clean up test path * cleanup * cleanup * address comments * revert readability change * add to changes md
1 parent f1e2147 commit c9ad32e

File tree

8 files changed

+843
-303
lines changed

8 files changed

+843
-303
lines changed

CHANGES.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -64,6 +64,7 @@
6464

6565
* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
6666
* Improvements to the performance of BigqueryIO when using withPropagateSuccessfulStorageApiWrites(true) method (Java) ([#31840](https://github.com/apache/beam/pull/31840)).
67+
* [Managed Iceberg] Added support for writing to partitioned tables ([#32102](https://github.com/apache/beam/pull/32102))
6768

6869
## New Features / Improvements
6970

sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java

Lines changed: 46 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,12 @@
1717
*/
1818
package org.apache.beam.sdk.io.iceberg;
1919

20-
import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamRowToIcebergRecord;
21-
2220
import java.io.IOException;
23-
import org.apache.beam.sdk.values.Row;
21+
import org.apache.beam.sdk.metrics.Counter;
22+
import org.apache.beam.sdk.metrics.Metrics;
2423
import org.apache.iceberg.DataFile;
2524
import org.apache.iceberg.FileFormat;
26-
import org.apache.iceberg.ManifestFile;
27-
import org.apache.iceberg.ManifestFiles;
28-
import org.apache.iceberg.ManifestWriter;
25+
import org.apache.iceberg.PartitionKey;
2926
import org.apache.iceberg.Table;
3027
import org.apache.iceberg.avro.Avro;
3128
import org.apache.iceberg.catalog.Catalog;
@@ -34,23 +31,37 @@
3431
import org.apache.iceberg.io.DataWriter;
3532
import org.apache.iceberg.io.OutputFile;
3633
import org.apache.iceberg.parquet.Parquet;
34+
import org.slf4j.Logger;
35+
import org.slf4j.LoggerFactory;
3736

3837
class RecordWriter {
39-
38+
private static final Logger LOG = LoggerFactory.getLogger(RecordWriter.class);
39+
private final Counter activeWriters = Metrics.counter(RecordWriterManager.class, "activeWriters");
4040
private final DataWriter<Record> icebergDataWriter;
41-
4241
private final Table table;
4342
private final String absoluteFilename;
43+
private final FileFormat fileFormat;
4444

45-
RecordWriter(Catalog catalog, IcebergDestination destination, String filename)
45+
RecordWriter(
46+
Catalog catalog, IcebergDestination destination, String filename, PartitionKey partitionKey)
4647
throws IOException {
4748
this(
48-
catalog.loadTable(destination.getTableIdentifier()), destination.getFileFormat(), filename);
49+
catalog.loadTable(destination.getTableIdentifier()),
50+
destination.getFileFormat(),
51+
filename,
52+
partitionKey);
4953
}
5054

51-
RecordWriter(Table table, FileFormat fileFormat, String filename) throws IOException {
55+
RecordWriter(Table table, FileFormat fileFormat, String filename, PartitionKey partitionKey)
56+
throws IOException {
5257
this.table = table;
53-
this.absoluteFilename = table.location() + "/" + filename;
58+
this.fileFormat = fileFormat;
59+
if (table.spec().isUnpartitioned()) {
60+
absoluteFilename = table.locationProvider().newDataLocation(filename);
61+
} else {
62+
absoluteFilename =
63+
table.locationProvider().newDataLocation(table.spec(), partitionKey, filename);
64+
}
5465
OutputFile outputFile = table.io().newOutputFile(absoluteFilename);
5566

5667
switch (fileFormat) {
@@ -60,6 +71,7 @@ class RecordWriter {
6071
.createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create)
6172
.schema(table.schema())
6273
.withSpec(table.spec())
74+
.withPartition(partitionKey)
6375
.overwrite()
6476
.build();
6577
break;
@@ -69,6 +81,7 @@ class RecordWriter {
6981
.createWriterFunc(GenericParquetWriter::buildWriter)
7082
.schema(table.schema())
7183
.withSpec(table.spec())
84+
.withPartition(partitionKey)
7285
.overwrite()
7386
.build();
7487
break;
@@ -77,34 +90,38 @@ class RecordWriter {
7790
default:
7891
throw new RuntimeException("Unknown File Format: " + fileFormat);
7992
}
93+
activeWriters.inc();
94+
LOG.info(
95+
"Opened {} writer for table {}, partition {}. Writing to path: {}",
96+
fileFormat,
97+
table.name(),
98+
partitionKey,
99+
absoluteFilename);
80100
}
81101

82-
public void write(Row row) {
83-
Record record = beamRowToIcebergRecord(table.schema(), row);
102+
public void write(Record record) {
84103
icebergDataWriter.write(record);
85104
}
86105

87106
public void close() throws IOException {
88-
icebergDataWriter.close();
89-
}
90-
91-
public Table getTable() {
92-
return table;
107+
try {
108+
icebergDataWriter.close();
109+
} catch (IOException e) {
110+
throw new IOException(
111+
String.format(
112+
"Failed to close %s writer for table %s, path: %s",
113+
fileFormat, table.name(), absoluteFilename),
114+
e);
115+
}
116+
activeWriters.dec();
117+
LOG.info("Closed {} writer for table {}, path: {}", fileFormat, table.name(), absoluteFilename);
93118
}
94119

95120
public long bytesWritten() {
96121
return icebergDataWriter.length();
97122
}
98123

99-
public ManifestFile getManifestFile() throws IOException {
100-
String manifestFilename = FileFormat.AVRO.addExtension(absoluteFilename + ".manifest");
101-
OutputFile outputFile = table.io().newOutputFile(manifestFilename);
102-
ManifestWriter<DataFile> manifestWriter;
103-
try (ManifestWriter<DataFile> openWriter = ManifestFiles.write(getTable().spec(), outputFile)) {
104-
openWriter.add(icebergDataWriter.toDataFile());
105-
manifestWriter = openWriter;
106-
}
107-
108-
return manifestWriter.toManifestFile();
124+
public DataFile getDataFile() {
125+
return icebergDataWriter.toDataFile();
109126
}
110127
}

0 commit comments

Comments
 (0)