|
94 | 94 | import org.apache.iceberg.PartitionSpec; |
95 | 95 | import org.apache.iceberg.Snapshot; |
96 | 96 | import org.apache.iceberg.Table; |
| 97 | +import org.apache.iceberg.TableProperties; |
97 | 98 | import org.apache.iceberg.avro.Avro; |
98 | 99 | import org.apache.iceberg.catalog.TableIdentifier; |
99 | 100 | import org.apache.iceberg.exceptions.AlreadyExistsException; |
|
102 | 103 | import org.apache.iceberg.io.OutputFile; |
103 | 104 | import org.apache.iceberg.mapping.MappingUtil; |
104 | 105 | import org.apache.iceberg.mapping.NameMapping; |
| 106 | +import org.apache.iceberg.mapping.NameMappingParser; |
105 | 107 | import org.apache.iceberg.orc.OrcMetrics; |
106 | 108 | import org.apache.iceberg.parquet.ParquetSchemaUtil; |
107 | 109 | import org.apache.iceberg.parquet.ParquetUtil; |
@@ -514,22 +516,37 @@ static <W, T> T transformValue(Transform<W, T> transform, Type type, ByteBuffer |
514 | 516 |
|
515 | 517 | private Table getOrCreateTable(String filePath, FileFormat format) throws IOException { |
516 | 518 | TableIdentifier tableId = TableIdentifier.parse(identifier); |
| 519 | + @Nullable Table t; |
517 | 520 | try { |
518 | | - return catalogConfig.catalog().loadTable(tableId); |
| 521 | + t = catalogConfig.catalog().loadTable(tableId); |
519 | 522 | } catch (NoSuchTableException e) { |
520 | 523 | try { |
521 | 524 | org.apache.iceberg.Schema schema = getSchema(filePath, format); |
522 | 525 | PartitionSpec spec = PartitionUtils.toPartitionSpec(partitionFields, schema); |
523 | 526 |
|
524 | | - return tableProps == null |
525 | | - ? catalogConfig.catalog().createTable(TableIdentifier.parse(identifier), schema, spec) |
526 | | - : catalogConfig |
527 | | - .catalog() |
528 | | - .createTable(TableIdentifier.parse(identifier), schema, spec, tableProps); |
| 527 | + t = |
| 528 | + tableProps == null |
| 529 | + ? catalogConfig |
| 530 | + .catalog() |
| 531 | + .createTable(TableIdentifier.parse(identifier), schema, spec) |
| 532 | + : catalogConfig |
| 533 | + .catalog() |
| 534 | + .createTable(TableIdentifier.parse(identifier), schema, spec, tableProps); |
529 | 535 | } catch (AlreadyExistsException e2) { // if table already exists, just load it |
530 | | - return catalogConfig.catalog().loadTable(TableIdentifier.parse(identifier)); |
| 536 | + t = catalogConfig.catalog().loadTable(TableIdentifier.parse(identifier)); |
531 | 537 | } |
532 | 538 | } |
| 539 | + ensureNameMappingPresent(t); |
| 540 | + return t; |
| 541 | + } |
| 542 | + |
| 543 | + private static void ensureNameMappingPresent(Table table) { |
| 544 | + if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) { |
| 545 | + // Forces Name based resolution instead of position based resolution |
| 546 | + NameMapping mapping = MappingUtil.create(table.schema()); |
| 547 | + String mappingJson = NameMappingParser.toJson(mapping); |
| 548 | + table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson).commit(); |
| 549 | + } |
533 | 550 | } |
534 | 551 |
|
535 | 552 | /** |
|
0 commit comments