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

Skip to content

Commit d50cc15

Browse files
authored
[Managed Iceberg] Fix partition value race condition (#33549)
* fix and update tests * dont mention df yet * add PR link * whitespace
1 parent c6f8aae commit d50cc15

File tree

4 files changed

+50
-10
lines changed

4 files changed

+50
-10
lines changed
Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
11
{
2-
"comment": "Modify this file in a trivial way to cause this test suite to run",
2+
"comment": "Modify this file in a trivial way to cause this test suite to run.",
33
"modification": 1
44
}

CHANGES.md

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -115,6 +115,7 @@
115115
## Bugfixes
116116

117117
* Fixed EventTimeTimer ordering in Prism. ([#32222](https://github.com/apache/beam/issues/32222)).
118+
* [Managed Iceberg] Fixed a bug where DataFile metadata was assigned incorrect partition values ([#33549](https://github.com/apache/beam/pull/33549)).
118119

119120
## Security Fixes
120121

@@ -157,6 +158,11 @@
157158
* Adding flag to support conditionally disabling auto-commit in JdbcIO ReadFn ([#31111](https://github.com/apache/beam/issues/31111))
158159
* (Python) Fixed BigQuery Enrichment bug that can lead to multiple conditions returning duplicate rows, batching returning incorrect results and conditions not scoped by row during batching ([#32780](https://github.com/apache/beam/pull/32780)).
159160

161+
## Known Issues
162+
163+
* [Managed Iceberg] DataFile metadata is assigned incorrect partition values ([#33497](https://github.com/apache/beam/issues/33497)).
164+
* Fixed in 2.62.0
165+
160166
# [2.60.0] - 2024-10-17
161167

162168
## Highlights
@@ -211,6 +217,8 @@ when running on 3.8. ([#31192](https://github.com/apache/beam/issues/31192))
211217
* Duplicate Rows: Multiple conditions may be applied incorrectly, leading to the duplication of rows in the output.
212218
* Incorrect Results with Batched Requests: Conditions may not be correctly scoped to individual rows within the batch, potentially causing inaccurate results.
213219
* Fixed in 2.61.0.
220+
* [Managed Iceberg] DataFile metadata is assigned incorrect partition values ([#33497](https://github.com/apache/beam/issues/33497)).
221+
* Fixed in 2.62.0
214222

215223
# [2.59.0] - 2024-09-11
216224

@@ -259,6 +267,8 @@ when running on 3.8. ([#31192](https://github.com/apache/beam/issues/31192))
259267
* Duplicate Rows: Multiple conditions may be applied incorrectly, leading to the duplication of rows in the output.
260268
* Incorrect Results with Batched Requests: Conditions may not be correctly scoped to individual rows within the batch, potentially causing inaccurate results.
261269
* Fixed in 2.61.0.
270+
* [Managed Iceberg] DataFile metadata is assigned incorrect partition values ([#33497](https://github.com/apache/beam/issues/33497)).
271+
* Fixed in 2.62.0
262272

263273
# [2.58.1] - 2024-08-15
264274

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

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -96,7 +96,9 @@ class DestinationState {
9696
private final IcebergDestination icebergDestination;
9797
private final PartitionSpec spec;
9898
private final org.apache.iceberg.Schema schema;
99-
private final PartitionKey partitionKey;
99+
// used to determine the partition to which a record belongs
100+
// must not be directly used to create a writer
101+
private final PartitionKey routingPartitionKey;
100102
private final Table table;
101103
private final String stateToken = UUID.randomUUID().toString();
102104
final Cache<PartitionKey, RecordWriter> writers;
@@ -109,7 +111,7 @@ class DestinationState {
109111
this.icebergDestination = icebergDestination;
110112
this.schema = table.schema();
111113
this.spec = table.spec();
112-
this.partitionKey = new PartitionKey(spec, schema);
114+
this.routingPartitionKey = new PartitionKey(spec, schema);
113115
this.table = table;
114116
for (PartitionField partitionField : spec.fields()) {
115117
partitionFieldMap.put(partitionField.name(), partitionField);
@@ -154,12 +156,12 @@ class DestinationState {
154156
* can't create a new writer, the {@link Record} is rejected and {@code false} is returned.
155157
*/
156158
boolean write(Record record) {
157-
partitionKey.partition(getPartitionableRecord(record));
159+
routingPartitionKey.partition(getPartitionableRecord(record));
158160

159-
if (!writers.asMap().containsKey(partitionKey) && openWriters >= maxNumWriters) {
161+
if (!writers.asMap().containsKey(routingPartitionKey) && openWriters >= maxNumWriters) {
160162
return false;
161163
}
162-
RecordWriter writer = fetchWriterForPartition(partitionKey);
164+
RecordWriter writer = fetchWriterForPartition(routingPartitionKey);
163165
writer.write(record);
164166
return true;
165167
}
@@ -173,10 +175,12 @@ private RecordWriter fetchWriterForPartition(PartitionKey partitionKey) {
173175
RecordWriter recordWriter = writers.getIfPresent(partitionKey);
174176

175177
if (recordWriter == null || recordWriter.bytesWritten() > maxFileSize) {
178+
// each writer must have its own PartitionKey object
179+
PartitionKey copy = partitionKey.copy();
176180
// calling invalidate for a non-existent key is a safe operation
177-
writers.invalidate(partitionKey);
178-
recordWriter = createWriter(partitionKey);
179-
writers.put(partitionKey, recordWriter);
181+
writers.invalidate(copy);
182+
recordWriter = createWriter(copy);
183+
writers.put(copy, recordWriter);
180184
}
181185
return recordWriter;
182186
}

sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,10 +28,13 @@
2828
import java.io.Serializable;
2929
import java.util.ArrayList;
3030
import java.util.Arrays;
31+
import java.util.Collections;
3132
import java.util.HashMap;
3233
import java.util.List;
3334
import java.util.Map;
35+
import java.util.Set;
3436
import java.util.UUID;
37+
import java.util.function.BiFunction;
3538
import java.util.stream.Collectors;
3639
import java.util.stream.LongStream;
3740
import java.util.stream.Stream;
@@ -65,6 +68,7 @@
6568
import org.apache.iceberg.TableScan;
6669
import org.apache.iceberg.catalog.Catalog;
6770
import org.apache.iceberg.catalog.TableIdentifier;
71+
import org.apache.iceberg.data.IdentityPartitionConverters;
6872
import org.apache.iceberg.data.Record;
6973
import org.apache.iceberg.data.parquet.GenericParquetReaders;
7074
import org.apache.iceberg.data.parquet.GenericParquetWriter;
@@ -74,7 +78,10 @@
7478
import org.apache.iceberg.io.InputFile;
7579
import org.apache.iceberg.io.OutputFile;
7680
import org.apache.iceberg.parquet.Parquet;
81+
import org.apache.iceberg.types.Type;
82+
import org.apache.iceberg.types.TypeUtil;
7783
import org.apache.iceberg.util.DateTimeUtil;
84+
import org.apache.iceberg.util.PartitionUtil;
7885
import org.checkerframework.checker.nullness.qual.Nullable;
7986
import org.joda.time.DateTime;
8087
import org.joda.time.DateTimeZone;
@@ -295,6 +302,22 @@ private List<Row> populateTable(Table table) throws IOException {
295302
return expectedRows;
296303
}
297304

305+
private static Map<Integer, ?> constantsMap(
306+
FileScanTask task,
307+
BiFunction<Type, Object, Object> converter,
308+
org.apache.iceberg.Schema schema) {
309+
PartitionSpec spec = task.spec();
310+
Set<Integer> idColumns = spec.identitySourceIds();
311+
org.apache.iceberg.Schema partitionSchema = TypeUtil.select(schema, idColumns);
312+
boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty();
313+
314+
if (projectsIdentityPartitionColumns) {
315+
return PartitionUtil.constantsMap(task, converter);
316+
} else {
317+
return Collections.emptyMap();
318+
}
319+
}
320+
298321
private List<Record> readRecords(Table table) {
299322
org.apache.iceberg.Schema tableSchema = table.schema();
300323
TableScan tableScan = table.newScan().project(tableSchema);
@@ -303,13 +326,16 @@ private List<Record> readRecords(Table table) {
303326
InputFilesDecryptor descryptor =
304327
new InputFilesDecryptor(task, table.io(), table.encryption());
305328
for (FileScanTask fileTask : task.files()) {
329+
Map<Integer, ?> idToConstants =
330+
constantsMap(fileTask, IdentityPartitionConverters::convertConstant, tableSchema);
306331
InputFile inputFile = descryptor.getInputFile(fileTask);
307332
CloseableIterable<Record> iterable =
308333
Parquet.read(inputFile)
309334
.split(fileTask.start(), fileTask.length())
310335
.project(tableSchema)
311336
.createReaderFunc(
312-
fileSchema -> GenericParquetReaders.buildReader(tableSchema, fileSchema))
337+
fileSchema ->
338+
GenericParquetReaders.buildReader(tableSchema, fileSchema, idToConstants))
313339
.filter(fileTask.residual())
314340
.build();
315341

0 commit comments

Comments
 (0)