-
Notifications
You must be signed in to change notification settings - Fork 1.3k
[iceberg] support migrate iceberg table suffering schema evolution #5078
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
Closed
LsomeYeah
wants to merge
13
commits into
apache:master
from
LsomeYeah:migrate-iceberg-evolution-commit
Closed
Changes from all commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
2f9f2a0
[iceberg] Introduce feature and IT cases to migrate table from iceber…
LsomeYeah 9e84b46
[core][hive] delete iceberg physical data using fileIO because iceber…
LsomeYeah 09df7c5
[procedure] remove 'target_table', delete source table and rename pai…
LsomeYeah 1e7937e
[pom][hive] remove useless dependency
LsomeYeah 1380215
[procedure] small fix
LsomeYeah 6d9a905
[procedure] use MigrateIcebergTableProcedure for migration
LsomeYeah 2727667
[procedure][fix] change the port num for TestHiveMetastore in Migrate…
LsomeYeah 68fee7f
[procedure][test] make testMigrateIcebergTableProcedure a random test
LsomeYeah 6b84657
[core][hive] remove FileIO argument in constructor
LsomeYeah ef138c3
[core][wip] support iceberg schema evolution
LsomeYeah 18247a2
[core] delete useless segments
LsomeYeah c42678d
[core] change the way of getting schema id from manifest file for avo…
LsomeYeah f08bcaa
resolve rebase conflicts
LsomeYeah File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -43,19 +43,26 @@ | |
| import org.apache.paimon.migrate.Migrator; | ||
| import org.apache.paimon.options.Options; | ||
| import org.apache.paimon.schema.Schema; | ||
| import org.apache.paimon.schema.SchemaManager; | ||
| import org.apache.paimon.schema.TableSchema; | ||
| import org.apache.paimon.table.FileStoreTable; | ||
| import org.apache.paimon.table.Table; | ||
| import org.apache.paimon.table.sink.BatchTableCommit; | ||
| import org.apache.paimon.table.sink.CommitMessage; | ||
| import org.apache.paimon.types.DataField; | ||
| import org.apache.paimon.utils.JsonSerdeUtil; | ||
| import org.apache.paimon.utils.Preconditions; | ||
|
|
||
| import org.apache.avro.file.DataFileStream; | ||
| import org.apache.avro.generic.GenericDatumReader; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.concurrent.Callable; | ||
|
|
@@ -136,15 +143,28 @@ public IcebergMigrator( | |
|
|
||
| @Override | ||
| public void executeMigrate() throws Exception { | ||
| Schema paimonSchema = icebergSchemaToPaimonSchema(icebergMetadata); | ||
| List<TableSchema> paimonSchemas = icebergSchemasToPaimonSchemas(icebergMetadata); | ||
| Preconditions.checkArgument( | ||
| !paimonSchemas.isEmpty(), | ||
| "paimon schemas transformed from iceberg table is empty."); | ||
| Identifier paimonIdentifier = Identifier.create(paimonDatabaseName, paimonTableName); | ||
|
|
||
| paimonCatalog.createDatabase(paimonDatabaseName, true); | ||
| paimonCatalog.createTable(paimonIdentifier, paimonSchema, false); | ||
| TableSchema firstSchema = paimonSchemas.get(0); | ||
| Preconditions.checkArgument(firstSchema.id() == 0, "Unexpected, first schema id is not 0."); | ||
| paimonCatalog.createTable(paimonIdentifier, firstSchema.toSchema(), false); | ||
|
|
||
| try { | ||
| FileStoreTable paimonTable = (FileStoreTable) paimonCatalog.getTable(paimonIdentifier); | ||
| FileIO fileIO = paimonTable.fileIO(); | ||
| SchemaManager schemaManager = paimonTable.schemaManager(); | ||
| // commit all the iceberg schemas | ||
| for (int i = 1; i < paimonSchemas.size(); i++) { | ||
| LOG.info( | ||
| "commit new schema from iceberg, new schema id:{}", | ||
| paimonSchemas.get(i).id()); | ||
| schemaManager.commit(paimonSchemas.get(i)); | ||
| } | ||
|
|
||
| IcebergManifestFile manifestFile = | ||
| IcebergManifestFile.create(paimonTable, icebergMetaPathFactory); | ||
|
|
@@ -157,25 +177,36 @@ public void executeMigrate() throws Exception { | |
| // check manifest file with 'DELETE' kind | ||
| checkAndFilterManifestFiles(icebergManifestFileMetas); | ||
|
|
||
| // get all live iceberg entries | ||
| List<IcebergManifestEntry> icebergEntries = | ||
| icebergManifestFileMetas.stream() | ||
| .flatMap(fileMeta -> manifestFile.read(fileMeta).stream()) | ||
| .filter(IcebergManifestEntry::isLive) | ||
| .collect(Collectors.toList()); | ||
| if (icebergEntries.isEmpty()) { | ||
| Map<Long, List<IcebergManifestEntry>> icebergEntries = new HashMap<>(); | ||
| for (IcebergManifestFileMeta icebergManifestFileMeta : icebergManifestFileMetas) { | ||
| long schemaId = | ||
| getSchemaIdFromIcebergManifestFile( | ||
| new Path(icebergManifestFileMeta.manifestPath()), fileIO); | ||
| List<IcebergManifestEntry> entries = manifestFile.read(icebergManifestFileMeta); | ||
| icebergEntries | ||
| .computeIfAbsent(schemaId, v -> new ArrayList<>()) | ||
| .addAll( | ||
| entries.stream() | ||
| .filter(IcebergManifestEntry::isLive) | ||
| .collect(Collectors.toList())); | ||
| } | ||
|
|
||
| List<IcebergDataFileMeta> icebergDataFileMetas = new ArrayList<>(); | ||
| // write schema id to IcebergDataFileMeta | ||
| for (Map.Entry<Long, List<IcebergManifestEntry>> kv : icebergEntries.entrySet()) { | ||
| icebergDataFileMetas.addAll( | ||
| kv.getValue().stream() | ||
| .map(entry -> entry.file().withSchemaId(kv.getKey())) | ||
| .collect(Collectors.toList())); | ||
| } | ||
|
|
||
| if (icebergDataFileMetas.isEmpty()) { | ||
| LOG.info( | ||
| "No live manifest entry in iceberg table for snapshot {}, iceberg table meta path is {}.", | ||
| "No live iceberg data files in iceberg table for snapshot {}, iceberg table meta path is {}.", | ||
| icebergMetadata.currentSnapshotId(), | ||
| icebergLatestMetadataLocation); | ||
| return; | ||
| } | ||
|
|
||
| List<IcebergDataFileMeta> icebergDataFileMetas = | ||
| icebergEntries.stream() | ||
| .map(IcebergManifestEntry::file) | ||
| .collect(Collectors.toList()); | ||
|
|
||
| // Again, check if delete File exists | ||
| checkAndFilterDataFiles(icebergDataFileMetas); | ||
|
|
||
|
|
@@ -246,10 +277,21 @@ public void renameTable(boolean ignoreIfNotExists) throws Exception { | |
| paimonCatalog.renameTable(targetTableId, sourceTableId, ignoreIfNotExists); | ||
| } | ||
|
|
||
| public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { | ||
| // get iceberg current schema | ||
| IcebergSchema icebergSchema = | ||
| icebergMetadata.schemas().get(icebergMetadata.currentSchemaId()); | ||
| public List<TableSchema> icebergSchemasToPaimonSchemas(IcebergMetadata icebergMetadata) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. private |
||
| return icebergMetadata.schemas().stream() | ||
| .map( | ||
| icebergSchema -> { | ||
| LOG.info( | ||
| "Convert iceberg schema to paimon schema, iceberg schema id: {}", | ||
| icebergSchema.schemaId()); | ||
| return TableSchema.create( | ||
| icebergSchema.schemaId(), | ||
| icebergSchemaToPaimonSchema(icebergSchema)); | ||
| }) | ||
| .collect(Collectors.toList()); | ||
| } | ||
|
|
||
| public Schema icebergSchemaToPaimonSchema(IcebergSchema icebergSchema) { | ||
|
|
||
| // get iceberg current partition spec | ||
| int currentPartitionSpecId = icebergMetadata.defaultSpecId(); | ||
|
|
@@ -289,6 +331,18 @@ private void checkAndFilterDataFiles(List<IcebergDataFileMeta> icebergDataFileMe | |
| } | ||
| } | ||
|
|
||
| public long getSchemaIdFromIcebergManifestFile(Path manifestPath, FileIO fileIO) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. private |
||
|
|
||
| try (DataFileStream<GenericRecord> dataFileStream = | ||
| new DataFileStream<>( | ||
| fileIO.newInputStream(manifestPath), new GenericDatumReader<>())) { | ||
| String schema = dataFileStream.getMetaString("schema"); | ||
| return JsonSerdeUtil.fromJson(schema, IcebergSchema.class).schemaId(); | ||
| } catch (IOException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
|
|
||
| private static List<DataFileMeta> construct( | ||
| List<IcebergDataFileMeta> icebergDataFileMetas, | ||
| FileIO fileIO, | ||
|
|
@@ -318,7 +372,9 @@ private static DataFileMeta constructFileMeta( | |
| e); | ||
| } | ||
| String format = icebergDataFileMeta.fileFormat(); | ||
| return FileMetaUtils.constructFileMeta(format, status, fileIO, table, dir, rollback); | ||
| long schemaId = icebergDataFileMeta.schemaId(); | ||
| return FileMetaUtils.constructFileMeta( | ||
| format, status, fileIO, table, dir, rollback, schemaId); | ||
| } | ||
|
|
||
| private MigrateTask importUnPartitionedTable( | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it is better : "only used for migrate iceberg table to paimon with schema evolution".
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Before migration, we do not check whether the Iceberg table has undergone schema evolution.
schemaIdwill be used in all cases for iceberg migration.