Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
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
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 2
"modification": 3
}
Original file line number Diff line number Diff line change
Expand Up @@ -516,37 +516,22 @@ static <W, T> T transformValue(Transform<W, T> transform, Type type, ByteBuffer

private Table getOrCreateTable(String filePath, FileFormat format) throws IOException {
TableIdentifier tableId = TableIdentifier.parse(identifier);
@Nullable Table t;
try {
t = catalogConfig.catalog().loadTable(tableId);
return catalogConfig.catalog().loadTable(tableId);
} catch (NoSuchTableException e) {
try {
org.apache.iceberg.Schema schema = getSchema(filePath, format);
PartitionSpec spec = PartitionUtils.toPartitionSpec(partitionFields, schema);

t =
tableProps == null
? catalogConfig
.catalog()
.createTable(TableIdentifier.parse(identifier), schema, spec)
: catalogConfig
.catalog()
.createTable(TableIdentifier.parse(identifier), schema, spec, tableProps);
return tableProps == null
? catalogConfig.catalog().createTable(TableIdentifier.parse(identifier), schema, spec)
: catalogConfig
.catalog()
.createTable(TableIdentifier.parse(identifier), schema, spec, tableProps);
} catch (AlreadyExistsException e2) { // if table already exists, just load it
t = catalogConfig.catalog().loadTable(TableIdentifier.parse(identifier));
return catalogConfig.catalog().loadTable(TableIdentifier.parse(identifier));
}
}
ensureNameMappingPresent(t);
return t;
}

private static void ensureNameMappingPresent(Table table) {
if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
// Forces Name based resolution instead of position based resolution
NameMapping mapping = MappingUtil.create(table.schema());
String mappingJson = NameMappingParser.toJson(mapping);
table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson).commit();
}
}

/**
Expand Down Expand Up @@ -743,6 +728,15 @@ public CommitManifestFilesDoFn(IcebergCatalogConfig catalogConfig, String identi
this.identifier = identifier;
}

private static void ensureNameMappingPresent(Table table) {
if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
// Forces Name based resolution instead of position based resolution
NameMapping mapping = MappingUtil.create(table.schema());
String mappingJson = NameMappingParser.toJson(mapping);
table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson).commit();
}
}

@ProcessElement
public void process(
@Element KV<String, Iterable<byte[]>> batch,
Expand All @@ -758,6 +752,7 @@ public void process(
table = catalogConfig.catalog().loadTable(TableIdentifier.parse(identifier));
}
table.refresh();
ensureNameMappingPresent(table);

if (shouldSkip(commitId, lastCommitTimestamp.read())) {
return;
Expand Down
Loading