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

Skip to content

Commit 77f38cd

Browse files
authored
Cherrypick #33549 to Release 2.62.0 branch (Iceberg partition value race condition #33497)
1 parent 0c54e6a commit 77f38cd

File tree

4 files changed

+51
-11
lines changed

4 files changed

+51
-11
lines changed
Lines changed: 2 additions & 2 deletions
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",
3-
"modification": 5
2+
"comment": "Modify this file in a trivial way to cause this test suite to run.",
3+
"modification": 1
44
}

CHANGES.md

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -92,6 +92,7 @@
9292
* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
9393
* Fixed EventTimeTimer ordering in Prism. ([#32222](https://github.com/apache/beam/issues/32222)).
9494
* [BigQueryIO] Fixed an issue where Storage Write API sometimes doesn't pick up auto-schema updates ([#33231](https://github.com/apache/beam/pull/33231))
95+
* [Managed Iceberg] Fixed a bug where DataFile metadata was assigned incorrect partition values ([#33549](https://github.com/apache/beam/pull/33549)).
9596

9697
## Security Fixes
9798
* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)).
@@ -138,6 +139,11 @@
138139
* Adding flag to support conditionally disabling auto-commit in JdbcIO ReadFn ([#31111](https://github.com/apache/beam/issues/31111))
139140
* (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)).
140141

142+
## Known Issues
143+
144+
* [Managed Iceberg] DataFile metadata is assigned incorrect partition values ([#33497](https://github.com/apache/beam/issues/33497)).
145+
* Fixed in 2.62.0
146+
141147
# [2.60.0] - 2024-10-17
142148

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

196204
# [2.59.0] - 2024-09-11
197205

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

244254
# [2.58.1] - 2024-08-15
245255

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/IcebergIOIT.java

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,13 @@
2929
import java.io.Serializable;
3030
import java.util.ArrayList;
3131
import java.util.Arrays;
32+
import java.util.Collections;
3233
import java.util.HashMap;
3334
import java.util.List;
3435
import java.util.Map;
36+
import java.util.Set;
3537
import java.util.UUID;
38+
import java.util.function.BiFunction;
3639
import java.util.stream.Collectors;
3740
import java.util.stream.LongStream;
3841
import java.util.stream.Stream;
@@ -68,6 +71,7 @@
6871
import org.apache.iceberg.TableScan;
6972
import org.apache.iceberg.catalog.Catalog;
7073
import org.apache.iceberg.catalog.TableIdentifier;
74+
import org.apache.iceberg.data.IdentityPartitionConverters;
7175
import org.apache.iceberg.data.Record;
7276
import org.apache.iceberg.data.parquet.GenericParquetReaders;
7377
import org.apache.iceberg.data.parquet.GenericParquetWriter;
@@ -78,7 +82,10 @@
7882
import org.apache.iceberg.io.InputFile;
7983
import org.apache.iceberg.io.OutputFile;
8084
import org.apache.iceberg.parquet.Parquet;
85+
import org.apache.iceberg.types.Type;
86+
import org.apache.iceberg.types.TypeUtil;
8187
import org.apache.iceberg.util.DateTimeUtil;
88+
import org.apache.iceberg.util.PartitionUtil;
8289
import org.checkerframework.checker.nullness.qual.Nullable;
8390
import org.joda.time.DateTime;
8491
import org.joda.time.DateTimeZone;
@@ -263,6 +270,22 @@ private List<Row> populateTable(Table table) throws IOException {
263270
return expectedRows;
264271
}
265272

273+
private static Map<Integer, ?> constantsMap(
274+
FileScanTask task,
275+
BiFunction<Type, Object, Object> converter,
276+
org.apache.iceberg.Schema schema) {
277+
PartitionSpec spec = task.spec();
278+
Set<Integer> idColumns = spec.identitySourceIds();
279+
org.apache.iceberg.Schema partitionSchema = TypeUtil.select(schema, idColumns);
280+
boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty();
281+
282+
if (projectsIdentityPartitionColumns) {
283+
return PartitionUtil.constantsMap(task, converter);
284+
} else {
285+
return Collections.emptyMap();
286+
}
287+
}
288+
266289
private List<Record> readRecords(Table table) {
267290
Schema tableSchema = table.schema();
268291
TableScan tableScan = table.newScan().project(tableSchema);
@@ -271,13 +294,16 @@ private List<Record> readRecords(Table table) {
271294
InputFilesDecryptor descryptor =
272295
new InputFilesDecryptor(task, table.io(), table.encryption());
273296
for (FileScanTask fileTask : task.files()) {
297+
Map<Integer, ?> idToConstants =
298+
constantsMap(fileTask, IdentityPartitionConverters::convertConstant, tableSchema);
274299
InputFile inputFile = descryptor.getInputFile(fileTask);
275300
CloseableIterable<Record> iterable =
276301
Parquet.read(inputFile)
277302
.split(fileTask.start(), fileTask.length())
278303
.project(tableSchema)
279304
.createReaderFunc(
280-
fileSchema -> GenericParquetReaders.buildReader(tableSchema, fileSchema))
305+
fileSchema ->
306+
GenericParquetReaders.buildReader(tableSchema, fileSchema, idToConstants))
281307
.filter(fileTask.residual())
282308
.build();
283309

0 commit comments

Comments
 (0)