-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Bug]: Managed IO for Iceberg cannot query by partition #33497
Comments
@ahmedabu98 any ideas here? |
Hey @saathwik-tk, can you provide a little more context?
Also a reproducible example would be helpful to find an RCA |
Hi @ahmedabu98 , Yes, I have created the iceberg table beforehand with the query, and my configuration map looks as below, Map<String,Object> configProperties = ImmutableMap.<String,Object>builder() |
I've been trying to reproduce it but haven't gotten something yet. |
Nevermind I got a reproduction: Code snippetpublic static void main(String[] args) throws IOException {
String project = "apache-beam-testing";
String BQMS_CATALOG = "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog";
String warehouse = "gs://ahmedabualsaud-apache-beam-testing/iceberg-testing";
Catalog catalog =
CatalogUtil.loadCatalog(
BQMS_CATALOG,
"bqms_catalog",
ImmutableMap.<String, String>builder()
.put("gcp_project", project)
.put("gcp_location", "us-central1")
.put("warehouse", warehouse)
.build(),
new Configuration());
Schema schema = Schema.builder().addInt64Field("long").addStringField("str").build();
org.apache.iceberg.Schema iceSchema = IcebergUtils.beamSchemaToIcebergSchema(schema);
PartitionSpec spec = PartitionSpec.builderFor(iceSchema).identity("str").build();
String tableId = "ahmedabualsaud.my_table_with_spec";
TableIdentifier identifier = TableIdentifier.parse(tableId);
Table table = catalog.createTable(identifier, iceSchema, spec);
List<Row> rows =
LongStream.range(0, 1000)
.mapToObj(l -> Row.withSchema(schema).addValues(l, "value_" + l).build())
.collect(Collectors.toList());
Pipeline p = Pipeline.create();
p.apply(Create.of(rows))
.setRowSchema(schema)
.apply(
Managed.write(Managed.ICEBERG)
.withConfig(
ImmutableMap.of(
"table",
tableId,
"catalog_properties",
ImmutableMap.of(
"catalog-impl", BQMS_CATALOG,
"gcp_project", project,
"gcp_location", "us-central1",
"warehouse", warehouse))));
p.run().waitUntilFinish();
} I get no results when executing this query in BigQuery: SELECT * FROM `apache-beam-testing.ahmedabualsaud.my_table_with_spec` where str = "value_811" This works however: SELECT * FROM `apache-beam-testing.ahmedabualsaud.my_table_with_spec` where str like "value_811" |
So it looks like we're writing the correct records to the correct partitions, but we get the wrong results when reading it back... I can see the issue even when reading it with the Beam Iceberg connector: Code snippetp
.apply(
Managed.read(Managed.ICEBERG)
.withConfig(
ImmutableMap.of(
"table",
tableId,
"catalog_properties",
ImmutableMap.of(
"catalog-impl", BQMS_CATALOG,
"gcp_project", project,
"gcp_location", "us-central1",
"warehouse", warehouse))))
.getSinglePCollection()
.apply(
MapElements.into(TypeDescriptors.nulls())
.via(
r -> {
long l = r.getInt64("long");
String s = r.getString("str");
if (!s.contains(String.valueOf(l))) {
System.out.println("WEIRD! " + r);
}
return null;
}));
p.run().waitUntilFinish();
|
I may have missed it, but what is the result of I'm trying to distinguish which of these is true, and maybe I missed it but I don't see the answer here:
|
It's the latter of those |
OK that's probably the simpler case. It sounds like that data file isn't "part of the table". |
Okay when I read datafiles like this, I do see the bad data: Code snippetFileIO io = table.io();
EncryptionManager encryptionManager = table.encryption();
TableScan scan = table.newScan();
for (CombinedScanTask combinedScanTask : scan.planTasks()) {
InputFilesDecryptor decryptor =
new InputFilesDecryptor(combinedScanTask, io, encryptionManager);
for (FileScanTask fileScanTask : combinedScanTask.tasks()) {
Map<Integer, ?> idToConstants =
constantsMap(fileScanTask, IdentityPartitionConverters::convertConstant, iceSchema);
InputFile inputFile = decryptor.getInputFile(fileScanTask);
CloseableIterable<Record> iterable =
Parquet.read(inputFile)
.split(fileScanTask.start(), fileScanTask.length())
.project(table.schema())
.createReaderFunc(
fileSchema ->
GenericParquetReaders.buildReader(
table.schema(), fileSchema, idToConstants))
.filter(fileScanTask.residual())
.build();
for (Record rec : iterable) {
System.out.println("xxx " + rec);
}
}
}
...
private static Map<Integer, ?> constantsMap(
FileScanTask task,
BiFunction<Type, Object, Object> converter,
org.apache.iceberg.Schema schema) {
PartitionSpec spec = task.spec();
Set<Integer> idColumns = spec.identitySourceIds();
org.apache.iceberg.Schema partitionSchema = TypeUtil.select(schema, idColumns);
boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty();
if (projectsIdentityPartitionColumns) {
return PartitionUtil.constantsMap(task, converter);
} else {
return Collections.emptyMap();
}
} The key difference being the additional So the datafile is indeed part of the table but the partition value takes precedence. Seems that we are probably setting the wrong partition value. |
Ahh the problem is that we're using a single PartitionKey object to resolve partitions, and using the same object to create writers. What ends up happening is a race condition on which partition we resolve to by the end of the bundle. That partition value gets set for all writers in that bundle. Instead, we should be making one copy of partitionKey for each new writer. |
@saathwik-tk I opened a PR to fix this in #33549. I ran another set of questions and from what I can tell it also fixes your original issue. |
What happened?
Lets say, I have field1 (string type) as my partition field, and there are many other fields, field2, field3, field4....
Querying the data with " select * from iceberg_table where field1 = 'field1_example_value'; " returns no data
but when querying with " select * from iceberg_table where field2 = 'field2_example_value'; " returns the data as expected.
I have ingested the data through Managed IO of Iceberg,
pipeline.apply(Managed.write(Managed.ICEBERG).withConfig(config_map);
Please have a look into this issue.
Issue Priority
Priority: 1 (data loss / total loss of function)
Issue Components
The text was updated successfully, but these errors were encountered: