From a7601c6d598c0ed05ff0897b30cb179a8f7b38c2 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 21 Nov 2024 15:11:50 +0800 Subject: [PATCH 01/30] first version # Conflicts: # paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java --- paimon-core/pom.xml | 2 - .../paimon/iceberg/IcebergMigrator.java | 156 +++++++++++++++++ .../iceberg/metadata/IcebergDataField.java | 66 +++++++ .../apache/paimon/migrate/FileMetaUtils.java | 31 ++++ .../paimon/iceberg/IcebergMigrateTest.java | 164 ++++++++++++++++++ 5 files changed, 417 insertions(+), 2 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index e137d57a6db1..c5a82b14d60d 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -203,14 +203,12 @@ under the License. org.apache.iceberg iceberg-core ${iceberg.version} - test org.apache.iceberg iceberg-data ${iceberg.version} - test diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java new file mode 100644 index 000000000000..b218e536fe28 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java @@ -0,0 +1,156 @@ +package org.apache.paimon.iceberg; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.manifest.IcebergDataFileMeta; +import org.apache.paimon.iceberg.manifest.IcebergManifestEntry; +import org.apache.paimon.iceberg.manifest.IcebergManifestFile; +import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta; +import org.apache.paimon.iceberg.manifest.IcebergManifestList; +import org.apache.paimon.iceberg.metadata.IcebergDataField; +import org.apache.paimon.iceberg.metadata.IcebergMetadata; +import org.apache.paimon.iceberg.metadata.IcebergPartitionField; +import org.apache.paimon.iceberg.metadata.IcebergPartitionSpec; +import org.apache.paimon.iceberg.metadata.IcebergSchema; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.migrate.FileMetaUtils; +import org.apache.paimon.migrate.Migrator; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.types.DataField; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** doc. */ +public class IcebergMigrator implements Migrator { + private static final String VERSION_HINT_FILENAME = "version-hint.text"; + + private final Catalog paimonCatalog; + // TODO:paimon的FileIO应该和iceberg的FileIO区分开来?用各自的FileIO去读各自的文件 + private final FileIO fileIO; + private final String paimonDatabaseName; + private final String paimonTableNameame; + + private final IcebergPathFactory icebergPathFactory; + private final IcebergMetadata icebergMetadata; + + public IcebergMigrator( + Catalog paimonCatalog, + IcebergPathFactory icebergPathFactory, + String paimonDatabaseName, + String paimonTableNameame) { + this.paimonCatalog = paimonCatalog; + this.fileIO = paimonCatalog.fileIO(); + this.paimonDatabaseName = paimonDatabaseName; + this.paimonTableNameame = paimonTableNameame; + + this.icebergPathFactory = icebergPathFactory; + this.icebergMetadata = + IcebergMetadata.fromPath( + fileIO, icebergPathFactory.toMetadataPath(getIcebergNewestSnapshotId())); + } + + @Override + public void executeMigrate() throws Exception { + + Schema paimonSchema = icebergSchemaToPaimonSchema(icebergMetadata); + Identifier paimonIdentifier = Identifier.create(paimonDatabaseName, paimonTableNameame); + paimonCatalog.createDatabase(paimonDatabaseName, true); + paimonCatalog.createTable(paimonIdentifier, paimonSchema, false); + + FileStoreTable paimonTable = (FileStoreTable) paimonCatalog.getTable(paimonIdentifier); + + IcebergManifestFile manifestFile = + IcebergManifestFile.create(paimonTable, icebergPathFactory); + IcebergManifestList manifestList = + IcebergManifestList.create(paimonTable, icebergPathFactory); + + // get all IcebergManifestEntrys so that we can get all datafiles + List icebergManifestFileMetas = + manifestList.read(icebergMetadata.currentSnapshot().manifestList()); + List icebergEntries = + icebergManifestFileMetas.stream() + .flatMap( + fileMeta -> + manifestFile + .read(new Path(fileMeta.manifestPath()).getName()) + .stream()) + .filter(IcebergManifestEntry::isLive) + .collect(Collectors.toList()); + + // construct paimon DataFileMeta from iceberg DataFileMeta + List icebergDataFileMetas = + icebergEntries.stream() + .map(IcebergManifestEntry::file) + .collect(Collectors.toList()); + + BinaryRow partitionRow = BinaryRow.EMPTY_ROW; + Path newDir = paimonTable.store().pathFactory().bucketPath(partitionRow, 0); + // TODO:需要获取到iceberg目前有几个分区,光有分区字段不够,需要真实的分区,如2000/12,2001/12这种 + if (!paimonTable.partitionKeys().isEmpty()) { + newDir = paimonTable.store().pathFactory().bucketPath(BinaryRow.EMPTY_ROW, 0); + } + Map rollBack = new ConcurrentHashMap<>(); + List dataFileMetas = + FileMetaUtils.construct( + icebergDataFileMetas, fileIO, paimonTable, newDir, rollBack); + CommitMessage commitMessage = FileMetaUtils.commitFile(partitionRow, dataFileMetas); + try (BatchTableCommit commit = paimonTable.newBatchWriteBuilder().newCommit()) { + commit.commit(Collections.singletonList(commitMessage)); + } + } + + @Override + public void deleteOriginTable(boolean delete) throws Exception {} + + @Override + public void renameTable(boolean ignoreIfNotExists) throws Exception {} + + public int getIcebergNewestSnapshotId() { + Path versionHintPath = + new Path(icebergPathFactory.metadataDirectory(), VERSION_HINT_FILENAME); + try { + return Integer.parseInt(fileIO.readFileUtf8(versionHintPath)); + } catch (IOException e) { + throw new RuntimeException( + "read iceberg version-hint.text failed. Iceberg metadata path: " + + icebergPathFactory.metadataDirectory()); + } + } + + public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { + // get iceberg current schema + IcebergSchema icebergSchema = + icebergMetadata.schemas().get(icebergMetadata.currentSchemaId()); + + // get iceberg current partition spec + int currentPartitionSpecId = icebergMetadata.defaultSpecId(); + IcebergPartitionSpec currentIcebergPartitionSpec = + icebergMetadata.partitionSpecs().get(currentPartitionSpecId); + + List dataFields = + icebergSchema.fields().stream() + .map(IcebergDataField::toDatafield) + .collect(Collectors.toList()); + + List partitionKeys = + currentIcebergPartitionSpec.fields().stream() + .map(IcebergPartitionField::name) + .collect(Collectors.toList()); + + return new Schema( + dataFields, partitionKeys, Collections.emptyList(), Collections.emptyMap(), null); + } + + public void createPaimonTable() {} +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 4ecc77a13581..c589a55fa194 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -20,13 +20,22 @@ import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DateType; import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.FloatType; +import org.apache.paimon.types.IntType; import org.apache.paimon.types.LocalZonedTimestampType; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; import org.apache.paimon.types.TimestampType; +import org.apache.paimon.types.VarBinaryType; +import org.apache.paimon.types.VarCharType; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; @@ -190,6 +199,63 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { } } + // TODO: two datatype("uuid", "unknown") has not been processed + public DataType getDataType() { + String simpleType = type.toString(); + String typePrefix = + !simpleType.contains("(") + ? simpleType + : simpleType.substring(0, simpleType.indexOf("(")); + switch (typePrefix) { + case "boolean": + return new BooleanType(required); + case "int": + return new IntType(required); + case "long": + return new BigIntType(required); + case "float": + return new FloatType(required); + case "double": + return new DoubleType(required); + case "date": + return new DateType(required); + case "string": + return new VarCharType(required, VarCharType.MAX_LENGTH); + case "binary": + return new VarBinaryType(required, VarBinaryType.MAX_LENGTH); + case "fixed": + int fixedLength = + Integer.parseInt( + simpleType.substring( + simpleType.indexOf("(") + 1, simpleType.indexOf(")"))); + return new BinaryType(required, fixedLength); + case "decimal": + int precision = + Integer.parseInt( + simpleType.substring( + simpleType.indexOf("(") + 1, simpleType.indexOf(","))); + int scale = + Integer.parseInt( + simpleType.substring( + simpleType.indexOf(",") + 1, simpleType.indexOf(")"))); + return new DecimalType(required, precision, scale); + case "timestamp": + return new TimestampType(required, 6); + case "timestamptz": + return new LocalZonedTimestampType(required, 6); + case "timestamp_ns": + return new TimestampType(required, 9); + case "timestamptz_ns": + return new LocalZonedTimestampType(required, 9); + default: + throw new UnsupportedOperationException("Unsupported data type: " + type); + } + } + + public DataField toDatafield() { + return new DataField(id, name, getDataType(), doc); + } + @Override public int hashCode() { return Objects.hash(id, name, required, type, doc); diff --git a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java index 391c5f9bb615..dc87bd1fc4b3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java @@ -27,6 +27,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.manifest.IcebergDataFileMeta; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; @@ -60,6 +61,20 @@ public class FileMetaUtils { private static final Logger LOG = LoggerFactory.getLogger(FileMetaUtils.class); + public static List construct( + List icebergDataFileMetas, + FileIO fileIO, + Table paimonTable, + Path newDir, + Map rollback) { + return icebergDataFileMetas.stream() + .map( + icebergDataFileMeta -> + constructFileMeta( + icebergDataFileMeta, fileIO, paimonTable, newDir, rollback)) + .collect(Collectors.toList()); + } + public static List construct( FileIO fileIO, String format, @@ -93,6 +108,22 @@ public static CommitMessage commitFile(BinaryRow partition, List d } // -----------------------------private method--------------------------------------------- + private static DataFileMeta constructFileMeta( + IcebergDataFileMeta icebergDataFileMeta, + FileIO fileIO, + Table table, + Path dir, + Map rollback) { + FileStatus status; + try { + status = fileIO.getFileStatus(new Path(icebergDataFileMeta.filePath())); + } catch (IOException e) { + throw new RuntimeException( + "error when get file status. file path is " + icebergDataFileMeta.filePath()); + } + String format = icebergDataFileMeta.fileFormat(); + return constructFileMeta(format, status, fileIO, table, dir, rollback); + } private static DataFileMeta constructFileMeta( String format, diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java new file mode 100644 index 000000000000..185e60e9a6bf --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -0,0 +1,164 @@ +package org.apache.paimon.iceberg; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.DataFormatTestUtil; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.TableRead; + +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +import static org.apache.paimon.options.CatalogOptions.CACHE_ENABLED; + +/** doc. */ +public class IcebergMigrateTest { + @TempDir java.nio.file.Path tempDir; + @TempDir java.nio.file.Path tempDirPaimon; + + @Test + public void test1() throws Exception { + Table icebergTable = createIcebergTable("default", "test"); + IcebergPathFactory icebergPathFactory = + new IcebergPathFactory(new Path(icebergTable.location(), "metadata")); + + CatalogContext context = CatalogContext.create(new Path(tempDirPaimon.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator(catalog, icebergPathFactory, "default", "tb1"); + icebergMigrator.executeMigrate(); + } + + @Test + public void test2() throws Exception { + IcebergPathFactory icebergPathFactory = + new IcebergPathFactory(new Path("/Users/catyeah/Desktop", "metadata1")); + + CatalogContext context = CatalogContext.create(new Path(tempDirPaimon.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator(catalog, icebergPathFactory, "default", "tb1"); + icebergMigrator.executeMigrate(); + } + + @Test + public void test3() throws Exception { + List iceResults = + getIcebergResult( + icebergTable -> IcebergGenerics.read(icebergTable).build(), + Record::toString); + IcebergPathFactory icebergPathFactory = + new IcebergPathFactory( + new Path("/Users/catyeah/testHome/icebergtest/mydb.db/t", "metadata")); + + CatalogContext context = CatalogContext.create(new Path(tempDirPaimon.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator(catalog, icebergPathFactory, "default", "tb1"); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create("default", "tb1")); + List paiResults = getPaimonResult(paimonTable); + System.out.println(); + } + + private Table createIcebergTable(String database, String tableName) throws Exception { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), tempDir.toString()); + TableIdentifier icebergIdentifier = TableIdentifier.of(database, tableName); + + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.unpartitioned(); + + return catalog.buildTable(icebergIdentifier, schema).withPartitionSpec(spec).create(); + } + + private List getPaimonResult(FileStoreTable paimonTable) throws Exception { + List splits = paimonTable.newReadBuilder().newScan().plan().splits(); + TableRead read = paimonTable.newReadBuilder().newRead(); + try (RecordReader recordReader = read.createReader(splits)) { + List result = new ArrayList<>(); + recordReader.forEachRemaining( + row -> + result.add( + DataFormatTestUtil.internalRowToString( + row, paimonTable.rowType()))); + return result; + } + } + + private List getIcebergResult( + Function> query, + Function icebergRecordToString) + throws Exception { + HadoopCatalog icebergCatalog = + new HadoopCatalog(new Configuration(), "/Users/catyeah/testHome/icebergtest"); + TableIdentifier icebergIdentifier = TableIdentifier.of("mydb.db", "t"); + org.apache.iceberg.Table icebergTable = icebergCatalog.loadTable(icebergIdentifier); + List manifestFiles = + icebergTable.currentSnapshot().allManifests(icebergTable.io()); + + CloseableIterable result = query.apply(icebergTable); + List actual = new ArrayList<>(); + for (Record record : result) { + actual.add(icebergRecordToString.apply(record)); + } + result.close(); + return actual; + } + + // private void writeData(Table table) { + // File dataDir = new File("/tmp/iceberg_data"); // 本地或远程文件系统路径 + // dataDir.mkdirs(); // 创建目录 + // OutputFile outputFile = Files.localOutput("/tmp/iceberg_data/table.parquet"); + // + // FileAppender appender = + // GenericParquetWriter.buildWriter(table.schema(), table.spec(), outputFile, + // null); + // + // try (appender) { + // // 创建示例数据记录 + // GenericRecord record = GenericRecord.create(table.schema()); + // appender.add(record.copy("id", 1, "name", "Alice")); + // appender.add(record.copy("id", 2, "name", "Bob")); + // } + // + // // 将数据文件追加为一个新的数据文件 + // DataFile dataFile = + // DataFiles.builder(table.spec()) + // .withInputFile(new Files.LocalInputFile(new File(dataDir, + // "table.parquet"))) + // .withRecordCount(2) + // .build(); + // + // // 提交事务 + // table.newAppend().appendFile(dataFile).commit(); + // } +} From 7aef9fef56a56ecf82466c9fdd02d4a7b24897b5 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 28 Nov 2024 17:19:41 +0800 Subject: [PATCH 02/30] [iceberg] inherit seq num from manifest file when seq num is null in manifest entry --- .../IcebergManifestEntrySerializer.java | 28 ++++++++++ .../iceberg/manifest/IcebergManifestFile.java | 51 +++++++++++++++++++ 2 files changed, 79 insertions(+) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java index d93456c3fe20..cf55f9749228 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java @@ -54,4 +54,32 @@ public IcebergManifestEntry fromRow(InternalRow row) { row.getLong(3), fileSerializer.fromRow(row.getRow(4, fileSerializer.numFields()))); } + + public IcebergManifestEntry fromRow(InternalRow row, IcebergManifestFileMeta meta) { + IcebergManifestEntry.Status status = IcebergManifestEntry.Status.fromId(row.getInt(0)); + long snapshotId = !row.isNullAt(1) ? row.getLong(1) : meta.addedSnapshotId(); + long sequenceNumber = getOrInherit(row, meta, 2, status); + long fileSequenceNumber = getOrInherit(row, meta, 3, status); + + return new IcebergManifestEntry( + status, + snapshotId, + sequenceNumber, + fileSequenceNumber, + fileSerializer.fromRow(row.getRow(4, fileSerializer.numFields()))); + } + + private long getOrInherit( + InternalRow row, + IcebergManifestFileMeta meta, + int pos, + IcebergManifestEntry.Status status) { + long sequenceNumber = meta.sequenceNumber(); + if (row.isNullAt(pos) + && (sequenceNumber == 0 || status == IcebergManifestEntry.Status.ADDED)) { + return sequenceNumber; + } else { + return row.getLong(pos); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java index 5955da6220f8..c95d7be1a051 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java @@ -19,6 +19,8 @@ package org.apache.paimon.iceberg.manifest; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.FormatWriterFactory; @@ -38,9 +40,14 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.CloseableIterator; +import org.apache.paimon.utils.FileUtils; +import org.apache.paimon.utils.Filter; import org.apache.paimon.utils.ObjectsFile; import org.apache.paimon.utils.PathFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -109,6 +116,50 @@ public static IcebergManifestFile create(FileStoreTable table, IcebergPathFactor table.coreOptions().manifestTargetSize()); } + public List read(IcebergManifestFileMeta meta) { + return read(meta, null); + } + + public List read(IcebergManifestFileMeta meta, @Nullable Long fileSize) { + String fileName = new Path(meta.manifestPath()).getName(); + try { + Path path = pathFactory.toPath(fileName); + + return readFromIterator( + meta, + createIterator(path, fileSize), + (IcebergManifestEntrySerializer) serializer, + Filter.alwaysTrue()); + } catch (IOException e) { + throw new RuntimeException("Failed to read " + fileName, e); + } + } + + private CloseableIterator createIterator(Path file, @Nullable Long fileSize) + throws IOException { + return FileUtils.createFormatReader(fileIO, readerFactory, file, fileSize) + .toCloseableIterator(); + } + + private static List readFromIterator( + IcebergManifestFileMeta meta, + CloseableIterator inputIterator, + IcebergManifestEntrySerializer serializer, + Filter readFilter) { + try (CloseableIterator iterator = inputIterator) { + List result = new ArrayList<>(); + while (iterator.hasNext()) { + InternalRow row = iterator.next(); + if (readFilter.test(row)) { + result.add(serializer.fromRow(row, meta)); + } + } + return result; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public List rollingWrite( Iterator entries, long sequenceNumber) { RollingFileWriter writer = From 88cd737b856fb167f74f8dbce9f9429e7de29ae3 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 28 Nov 2024 17:20:43 +0800 Subject: [PATCH 03/30] [iceberg] change the type of snapshotId to long --- .../apache/paimon/iceberg/metadata/IcebergMetadata.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMetadata.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMetadata.java index 86fb4a5df75a..fbaf8060022f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMetadata.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMetadata.java @@ -106,7 +106,7 @@ public class IcebergMetadata { private final List snapshots; @JsonProperty(FIELD_CURRENT_SNAPSHOT_ID) - private final int currentSnapshotId; + private final long currentSnapshotId; @JsonProperty(FIELD_PROPERTIES) @Nullable @@ -122,7 +122,7 @@ public IcebergMetadata( List partitionSpecs, int lastPartitionId, List snapshots, - int currentSnapshotId) { + long currentSnapshotId) { this( CURRENT_FORMAT_VERSION, tableUuid, @@ -158,7 +158,7 @@ public IcebergMetadata( @JsonProperty(FIELD_SORT_ORDERS) List sortOrders, @JsonProperty(FIELD_DEFAULT_SORT_ORDER_ID) int defaultSortOrderId, @JsonProperty(FIELD_SNAPSHOTS) List snapshots, - @JsonProperty(FIELD_CURRENT_SNAPSHOT_ID) int currentSnapshotId, + @JsonProperty(FIELD_CURRENT_SNAPSHOT_ID) long currentSnapshotId, @JsonProperty(FIELD_PROPERTIES) @Nullable Map properties) { this.formatVersion = formatVersion; this.tableUuid = tableUuid; @@ -249,7 +249,7 @@ public List snapshots() { } @JsonGetter(FIELD_CURRENT_SNAPSHOT_ID) - public int currentSnapshotId() { + public long currentSnapshotId() { return currentSnapshotId; } From 0679c3cf23c3891f5ba1a5562dce8655aff67c70 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 2 Dec 2024 18:20:01 +0800 Subject: [PATCH 04/30] support more data types --- .../org/apache/paimon/iceberg/metadata/IcebergDataField.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index c589a55fa194..6b9d4eaee9be 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -229,6 +229,10 @@ public DataType getDataType() { simpleType.substring( simpleType.indexOf("(") + 1, simpleType.indexOf(")"))); return new BinaryType(required, fixedLength); + case "uuid": + // https://iceberg.apache.org/spec/?h=vector#primitive-types + // uuid should use 16-byte fixed + return new BinaryType(required, 16); case "decimal": int precision = Integer.parseInt( From 021a40747c098417b69610aa0abc677ad112863f Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 2 Dec 2024 18:20:36 +0800 Subject: [PATCH 05/30] second version and test --- .../paimon/iceberg/IcebergMigrator.java | 282 +++++++++++++++--- .../paimon/iceberg/IcebergMigrateTest.java | 244 ++++++++++----- 2 files changed, 398 insertions(+), 128 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java index b218e536fe28..d1684a634100 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.paimon.iceberg; import org.apache.paimon.catalog.Catalog; @@ -24,89 +42,152 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.types.DataField; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadPoolExecutor; import java.util.stream.Collectors; +import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; + /** doc. */ public class IcebergMigrator implements Migrator { + private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrator.class); private static final String VERSION_HINT_FILENAME = "version-hint.text"; + private final ThreadPoolExecutor executor; + private final Catalog paimonCatalog; - // TODO:paimon的FileIO应该和iceberg的FileIO区分开来?用各自的FileIO去读各自的文件 - private final FileIO fileIO; + private final FileIO paimonFileIO; private final String paimonDatabaseName; private final String paimonTableNameame; + private final int icebergNewestSnapshotId; + // Path factory for iceberg metadata private final IcebergPathFactory icebergPathFactory; + // metadata for newest iceberg snapshot private final IcebergMetadata icebergMetadata; + private boolean ignoreDelete = false; + public IcebergMigrator( Catalog paimonCatalog, - IcebergPathFactory icebergPathFactory, + Path icebergMetaPath, String paimonDatabaseName, - String paimonTableNameame) { + String paimonTableNameame, + Integer parallelism) { this.paimonCatalog = paimonCatalog; - this.fileIO = paimonCatalog.fileIO(); + this.paimonFileIO = paimonCatalog.fileIO(); this.paimonDatabaseName = paimonDatabaseName; this.paimonTableNameame = paimonTableNameame; - this.icebergPathFactory = icebergPathFactory; + this.icebergPathFactory = new IcebergPathFactory(icebergMetaPath); + this.icebergNewestSnapshotId = getIcebergNewestSnapshotId(); this.icebergMetadata = IcebergMetadata.fromPath( - fileIO, icebergPathFactory.toMetadataPath(getIcebergNewestSnapshotId())); + paimonFileIO, icebergPathFactory.toMetadataPath(icebergNewestSnapshotId)); + + this.executor = createCachedThreadPool(parallelism, "ICEBERG_MIGRATOR"); } @Override public void executeMigrate() throws Exception { - Schema paimonSchema = icebergSchemaToPaimonSchema(icebergMetadata); Identifier paimonIdentifier = Identifier.create(paimonDatabaseName, paimonTableNameame); - paimonCatalog.createDatabase(paimonDatabaseName, true); + + paimonCatalog.createDatabase(paimonDatabaseName, false); paimonCatalog.createTable(paimonIdentifier, paimonSchema, false); - FileStoreTable paimonTable = (FileStoreTable) paimonCatalog.getTable(paimonIdentifier); - - IcebergManifestFile manifestFile = - IcebergManifestFile.create(paimonTable, icebergPathFactory); - IcebergManifestList manifestList = - IcebergManifestList.create(paimonTable, icebergPathFactory); - - // get all IcebergManifestEntrys so that we can get all datafiles - List icebergManifestFileMetas = - manifestList.read(icebergMetadata.currentSnapshot().manifestList()); - List icebergEntries = - icebergManifestFileMetas.stream() - .flatMap( - fileMeta -> - manifestFile - .read(new Path(fileMeta.manifestPath()).getName()) - .stream()) - .filter(IcebergManifestEntry::isLive) - .collect(Collectors.toList()); + try { + FileStoreTable paimonTable = (FileStoreTable) paimonCatalog.getTable(paimonIdentifier); - // construct paimon DataFileMeta from iceberg DataFileMeta - List icebergDataFileMetas = - icebergEntries.stream() - .map(IcebergManifestEntry::file) - .collect(Collectors.toList()); + // TODO: IcebergManifestFile.create() is suitable here? + IcebergManifestFile manifestFile = + IcebergManifestFile.create(paimonTable, icebergPathFactory); + IcebergManifestList manifestList = + IcebergManifestList.create(paimonTable, icebergPathFactory); - BinaryRow partitionRow = BinaryRow.EMPTY_ROW; - Path newDir = paimonTable.store().pathFactory().bucketPath(partitionRow, 0); - // TODO:需要获取到iceberg目前有几个分区,光有分区字段不够,需要真实的分区,如2000/12,2001/12这种 - if (!paimonTable.partitionKeys().isEmpty()) { - newDir = paimonTable.store().pathFactory().bucketPath(BinaryRow.EMPTY_ROW, 0); - } - Map rollBack = new ConcurrentHashMap<>(); - List dataFileMetas = - FileMetaUtils.construct( - icebergDataFileMetas, fileIO, paimonTable, newDir, rollBack); - CommitMessage commitMessage = FileMetaUtils.commitFile(partitionRow, dataFileMetas); - try (BatchTableCommit commit = paimonTable.newBatchWriteBuilder().newCommit()) { - commit.commit(Collections.singletonList(commitMessage)); + List icebergManifestFileMetas = + manifestList.read(icebergMetadata.currentSnapshot().manifestList()); + + // check manifest file with 'DELETE' kind + icebergManifestFileMetas = checkAndFilterManifestFiles(icebergManifestFileMetas); + + // get all live iceberg entries + List icebergEntries = + icebergManifestFileMetas.stream() + .flatMap(fileMeta -> manifestFile.read(fileMeta).stream()) + .filter(IcebergManifestEntry::isLive) + .collect(Collectors.toList()); + if (icebergEntries.isEmpty()) { + LOG.info( + "No live manifest entry in iceberg table for snapshot {}, iceberg table meta path is {}.", + icebergNewestSnapshotId, + icebergPathFactory.toMetadataPath(icebergNewestSnapshotId)); + return; + } + + List icebergDataFileMetas = + icebergEntries.stream() + .map(IcebergManifestEntry::file) + .collect(Collectors.toList()); + + // Again, check if the file is a Delete File + icebergDataFileMetas = checkAndFilterDataFiles(icebergDataFileMetas); + + LOG.info( + "Begin to create Migrate Task, the number of iceberg data files is {}", + icebergDataFileMetas.size()); + + List tasks = new ArrayList<>(); + Map rollback = new ConcurrentHashMap<>(); + if (paimonTable.partitionKeys().isEmpty()) { + tasks.add(importUnPartitionedTable(icebergDataFileMetas, paimonTable, rollback)); + } else { + tasks.addAll(importPartitionedTable(icebergDataFileMetas, paimonTable, rollback)); + } + + List> futures = + tasks.stream().map(executor::submit).collect(Collectors.toList()); + List commitMessages = new ArrayList<>(); + try { + for (Future future : futures) { + commitMessages.add(future.get()); + } + } catch (Exception e) { + futures.forEach(f -> f.cancel(true)); + for (Future future : futures) { + // wait all task cancelled or finished + while (!future.isDone()) { + //noinspection BusyWait + Thread.sleep(100); + } + } + // roll back all renamed path + for (Map.Entry entry : rollback.entrySet()) { + Path newPath = entry.getKey(); + Path origin = entry.getValue(); + if (paimonFileIO.exists(newPath)) { + paimonFileIO.rename(newPath, origin); + } + } + + throw new RuntimeException("Migrating failed because exception happens", e); + } + try (BatchTableCommit commit = paimonTable.newBatchWriteBuilder().newCommit()) { + commit.commit(new ArrayList<>(commitMessages)); + } + } catch (Exception e) { + paimonCatalog.dropTable(paimonIdentifier, true); + throw new RuntimeException("Migrating failed", e); } } @@ -120,7 +201,7 @@ public int getIcebergNewestSnapshotId() { Path versionHintPath = new Path(icebergPathFactory.metadataDirectory(), VERSION_HINT_FILENAME); try { - return Integer.parseInt(fileIO.readFileUtf8(versionHintPath)); + return Integer.parseInt(paimonFileIO.readFileUtf8(versionHintPath)); } catch (IOException e) { throw new RuntimeException( "read iceberg version-hint.text failed. Iceberg metadata path: " @@ -152,5 +233,110 @@ public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { dataFields, partitionKeys, Collections.emptyList(), Collections.emptyMap(), null); } - public void createPaimonTable() {} + private List checkAndFilterManifestFiles( + List icebergManifestFileMetas) { + if (!ignoreDelete) { + for (IcebergManifestFileMeta meta : icebergManifestFileMetas) { + if (meta.content() == IcebergManifestFileMeta.Content.DELETES) { + throw new RuntimeException( + "IcebergMigrator don't support analyzing manifest file with 'DELETE' content. " + + "You can set 'ignore-delete' to ignore manifest file with 'DELETE' content."); + } + } + return icebergManifestFileMetas; + } else { + return icebergManifestFileMetas.stream() + .filter(meta -> meta.content() != IcebergManifestFileMeta.Content.DELETES) + .collect(Collectors.toList()); + } + } + + private List checkAndFilterDataFiles( + List icebergDataFileMetas) { + if (!ignoreDelete) { + for (IcebergDataFileMeta meta : icebergDataFileMetas) { + if (meta.content() != IcebergDataFileMeta.Content.DATA) { + throw new RuntimeException( + "IcebergMigrator don't support analyzing iceberg delete file. " + + "You can set 'ignore-delete' to ignore iceberg delete files."); + } + } + return icebergDataFileMetas; + } else { + return icebergDataFileMetas.stream() + .filter(meta -> meta.content() == IcebergDataFileMeta.Content.DATA) + .collect(Collectors.toList()); + } + } + + private MigrateTask importUnPartitionedTable( + List icebergDataFileMetas, + FileStoreTable paimonTable, + Map rollback) { + BinaryRow partitionRow = BinaryRow.EMPTY_ROW; + Path newDir = paimonTable.store().pathFactory().bucketPath(partitionRow, 0); + + return new MigrateTask( + icebergDataFileMetas, paimonFileIO, paimonTable, partitionRow, newDir, rollback); + } + + private List importPartitionedTable( + List icebergDataFileMetas, + FileStoreTable paimonTable, + Map rollback) { + Map> dataInPartition = + icebergDataFileMetas.stream() + .collect(Collectors.groupingBy(IcebergDataFileMeta::partition)); + List migrateTasks = new ArrayList<>(); + for (Map.Entry> entry : dataInPartition.entrySet()) { + BinaryRow partitionRow = entry.getKey(); + Path newDir = paimonTable.store().pathFactory().bucketPath(partitionRow, 0); + migrateTasks.add( + new MigrateTask( + entry.getValue(), + paimonFileIO, + paimonTable, + partitionRow, + newDir, + rollback)); + } + return migrateTasks; + } + + /** One import task for one partition. */ + public static class MigrateTask implements Callable { + + private final List icebergDataFileMetas; + private final FileIO fileIO; + private final FileStoreTable paimonTable; + private final BinaryRow partitionRow; + private final Path newDir; + private final Map rollback; + + public MigrateTask( + List icebergDataFileMetas, + FileIO fileIO, + FileStoreTable paimonTable, + BinaryRow partitionRow, + Path newDir, + Map rollback) { + this.icebergDataFileMetas = icebergDataFileMetas; + this.fileIO = fileIO; + this.paimonTable = paimonTable; + this.partitionRow = partitionRow; + this.newDir = newDir; + this.rollback = rollback; + } + + @Override + public CommitMessage call() throws Exception { + if (!fileIO.exists(newDir)) { + fileIO.mkdirs(newDir); + } + List fileMetas = + FileMetaUtils.construct( + icebergDataFileMetas, fileIO, paimonTable, newDir, rollback); + return FileMetaUtils.commitFile(partitionRow, fileMetas); + } + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java index 185e60e9a6bf..d1d28c25a8d1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.paimon.iceberg; import org.apache.paimon.catalog.Catalog; @@ -13,91 +31,185 @@ import org.apache.paimon.table.source.TableRead; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.UUID; import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.paimon.options.CatalogOptions.CACHE_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; /** doc. */ public class IcebergMigrateTest { - @TempDir java.nio.file.Path tempDir; - @TempDir java.nio.file.Path tempDirPaimon; + @TempDir java.nio.file.Path iceTempDir; + @TempDir java.nio.file.Path paiTempDir; + + String iceDatabase = "ice_db"; + String iceTable = "ice_t"; + + String paiDatabase = "pai_db"; + String paiTable = "pai_t"; + + Schema iceSchema = + new Schema( + Types.NestedField.required(1, "k", Types.IntegerType.get()), + Types.NestedField.required(2, "v", Types.IntegerType.get()), + Types.NestedField.required(3, "dt", Types.StringType.get()), + Types.NestedField.required(4, "hh", Types.StringType.get())); + + PartitionSpec icePartitionSpec = + PartitionSpec.builderFor(iceSchema).identity("dt").identity("hh").build(); + + @ParameterizedTest(name = "isPartitioned = {0}") + @ValueSource(booleans = {true, false}) + public void testMigrateOnlyAddedData(boolean isPartitioned) throws Exception { + Table icebergTable = createIcebergTable(isPartitioned); + List records1 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "00"), + toIcebergRecord(2, 2, "20240101", "00")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, records1); + } - @Test - public void test1() throws Exception { - Table icebergTable = createIcebergTable("default", "test"); - IcebergPathFactory icebergPathFactory = - new IcebergPathFactory(new Path(icebergTable.location(), "metadata")); + List records2 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "01"), + toIcebergRecord(2, 2, "20240101", "01")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + } else { + writeRecordsToIceberg(icebergTable, records2); + } - CatalogContext context = CatalogContext.create(new Path(tempDirPaimon.toString())); + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); context.options().set(CACHE_ENABLED, false); Catalog catalog = CatalogFactory.createCatalog(context); IcebergMigrator icebergMigrator = - new IcebergMigrator(catalog, icebergPathFactory, "default", "tb1"); + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + 1); icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + Stream.concat(records1.stream(), records2.stream()) + .map(GenericRecord::toString) + .collect(Collectors.toList())); } - @Test - public void test2() throws Exception { - IcebergPathFactory icebergPathFactory = - new IcebergPathFactory(new Path("/Users/catyeah/Desktop", "metadata1")); + private Table createIcebergTable(boolean isPartitioned) { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), iceTempDir.toString()); + TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); + + if (!isPartitioned) { + return catalog.buildTable(icebergIdentifier, iceSchema) + .withPartitionSpec(PartitionSpec.unpartitioned()) + .create(); + } else { + return catalog.buildTable(icebergIdentifier, iceSchema) + .withPartitionSpec(icePartitionSpec) + .create(); + } + } - CatalogContext context = CatalogContext.create(new Path(tempDirPaimon.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); - IcebergMigrator icebergMigrator = - new IcebergMigrator(catalog, icebergPathFactory, "default", "tb1"); - icebergMigrator.executeMigrate(); + private GenericRecord toIcebergRecord(int k, int v, String dt, String hh) { + GenericRecord record = GenericRecord.create(iceSchema); + record.set(0, k); + record.set(1, v); + record.set(2, dt); + record.set(3, hh); + return record; } - @Test - public void test3() throws Exception { - List iceResults = - getIcebergResult( - icebergTable -> IcebergGenerics.read(icebergTable).build(), - Record::toString); - IcebergPathFactory icebergPathFactory = - new IcebergPathFactory( - new Path("/Users/catyeah/testHome/icebergtest/mydb.db/t", "metadata")); - - CatalogContext context = CatalogContext.create(new Path(tempDirPaimon.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); - IcebergMigrator icebergMigrator = - new IcebergMigrator(catalog, icebergPathFactory, "default", "tb1"); - icebergMigrator.executeMigrate(); + private DataFile writeRecordsToIceberg( + Table icebergTable, List records, String... partitionValues) + throws IOException { + String filepath = icebergTable.location() + "/" + UUID.randomUUID(); + OutputFile file = icebergTable.io().newOutputFile(filepath); + + DataWriter dataWriter = + Parquet.writeData(file) + .schema(iceSchema) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .withSpec(PartitionSpec.unpartitioned()) + .build(); + + if (partitionValues.length != 0) { + dataWriter = + Parquet.writeData(file) + .schema(iceSchema) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .withSpec(icePartitionSpec) + .withPartition( + partitionKey( + icePartitionSpec, + icebergTable, + partitionValues[0], + partitionValues[1])) + .build(); + } - FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create("default", "tb1")); - List paiResults = getPaimonResult(paimonTable); - System.out.println(); + try { + for (GenericRecord r : records) { + dataWriter.write(r); + } + } finally { + dataWriter.close(); + } + DataFile dataFile = dataWriter.toDataFile(); + icebergTable.newAppend().appendFile(dataFile).commit(); + return dataFile; } - private Table createIcebergTable(String database, String tableName) throws Exception { - HadoopCatalog catalog = new HadoopCatalog(new Configuration(), tempDir.toString()); - TableIdentifier icebergIdentifier = TableIdentifier.of(database, tableName); + private PartitionKey partitionKey( + PartitionSpec spec, Table icergTable, String... partitionValues) { + Record record = + GenericRecord.create(icergTable.schema()) + .copy(ImmutableMap.of("dt", partitionValues[0], "hh", partitionValues[1])); - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "name", Types.StringType.get())); + PartitionKey partitionKey = new PartitionKey(spec, icergTable.schema()); + partitionKey.partition(record); - PartitionSpec spec = PartitionSpec.unpartitioned(); - - return catalog.buildTable(icebergIdentifier, schema).withPartitionSpec(spec).create(); + return partitionKey; } private List getPaimonResult(FileStoreTable paimonTable) throws Exception { @@ -108,7 +220,7 @@ private List getPaimonResult(FileStoreTable paimonTable) throws Exceptio recordReader.forEachRemaining( row -> result.add( - DataFormatTestUtil.internalRowToString( + DataFormatTestUtil.toStringNoRowKind( row, paimonTable.rowType()))); return result; } @@ -133,32 +245,4 @@ private List getIcebergResult( result.close(); return actual; } - - // private void writeData(Table table) { - // File dataDir = new File("/tmp/iceberg_data"); // 本地或远程文件系统路径 - // dataDir.mkdirs(); // 创建目录 - // OutputFile outputFile = Files.localOutput("/tmp/iceberg_data/table.parquet"); - // - // FileAppender appender = - // GenericParquetWriter.buildWriter(table.schema(), table.spec(), outputFile, - // null); - // - // try (appender) { - // // 创建示例数据记录 - // GenericRecord record = GenericRecord.create(table.schema()); - // appender.add(record.copy("id", 1, "name", "Alice")); - // appender.add(record.copy("id", 2, "name", "Bob")); - // } - // - // // 将数据文件追加为一个新的数据文件 - // DataFile dataFile = - // DataFiles.builder(table.spec()) - // .withInputFile(new Files.LocalInputFile(new File(dataDir, - // "table.parquet"))) - // .withRecordCount(2) - // .build(); - // - // // 提交事务 - // table.newAppend().appendFile(dataFile).commit(); - // } } From 4ee8126303805179f9c8c43033a4fa8fdff67802 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 10:00:42 +0800 Subject: [PATCH 06/30] fix bug for type conversion --- .../iceberg/metadata/IcebergDataField.java | 40 ++++++++++++------- 1 file changed, 25 insertions(+), 15 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 6b9d4eaee9be..1b0d72c9479e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -206,51 +206,61 @@ public DataType getDataType() { !simpleType.contains("(") ? simpleType : simpleType.substring(0, simpleType.indexOf("(")); + String delimiter = "("; + if (type.contains("[")) { + delimiter = "["; + } + String typePrefix = + !type.contains(delimiter) ? type : type.substring(0, type.indexOf(delimiter)); switch (typePrefix) { case "boolean": - return new BooleanType(required); + return new BooleanType(!required); case "int": - return new IntType(required); + return new IntType(!required); case "long": - return new BigIntType(required); + return new BigIntType(!required); case "float": - return new FloatType(required); + return new FloatType(!required); case "double": - return new DoubleType(required); + return new DoubleType(!required); case "date": - return new DateType(required); + return new DateType(!required); case "string": - return new VarCharType(required, VarCharType.MAX_LENGTH); + return new VarCharType(!required, VarCharType.MAX_LENGTH); case "binary": - return new VarBinaryType(required, VarBinaryType.MAX_LENGTH); + return new VarBinaryType(!required, VarBinaryType.MAX_LENGTH); case "fixed": int fixedLength = Integer.parseInt( simpleType.substring( simpleType.indexOf("(") + 1, simpleType.indexOf(")"))); return new BinaryType(required, fixedLength); + Integer.parseInt(type.substring(type.indexOf("[") + 1, type.indexOf("]"))); + return new BinaryType(!required, fixedLength); case "uuid": // https://iceberg.apache.org/spec/?h=vector#primitive-types // uuid should use 16-byte fixed - return new BinaryType(required, 16); + return new BinaryType(!required, 16); case "decimal": int precision = Integer.parseInt( simpleType.substring( simpleType.indexOf("(") + 1, simpleType.indexOf(","))); int scale = + Integer.parseInt(type.substring(type.indexOf(",") + 2, type.indexOf(")"))); + return new DecimalType(!required, precision, scale); Integer.parseInt( simpleType.substring( simpleType.indexOf(",") + 1, simpleType.indexOf(")"))); return new DecimalType(required, precision, scale); case "timestamp": - return new TimestampType(required, 6); + return new TimestampType(!required, 6); case "timestamptz": - return new LocalZonedTimestampType(required, 6); - case "timestamp_ns": - return new TimestampType(required, 9); - case "timestamptz_ns": - return new LocalZonedTimestampType(required, 9); + return new LocalZonedTimestampType(!required, 6); + case "timestamp_ns": // iceberg v3 format + return new TimestampType(!required, 9); + case "timestamptz_ns": // iceberg v3 format + return new LocalZonedTimestampType(!required, 9); default: throw new UnsupportedOperationException("Unsupported data type: " + type); } From 1eb789b931469722a7272e1c025e97caeb691170 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 10:01:26 +0800 Subject: [PATCH 07/30] add unit tests --- .../paimon/iceberg/IcebergMigrator.java | 2 + .../paimon/iceberg/IcebergMigrateTest.java | 407 +++++++++++++++++- 2 files changed, 400 insertions(+), 9 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java index d1684a634100..dd084ccfefb3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java @@ -83,6 +83,7 @@ public IcebergMigrator( Path icebergMetaPath, String paimonDatabaseName, String paimonTableNameame, + boolean ignoreDelete, Integer parallelism) { this.paimonCatalog = paimonCatalog; this.paimonFileIO = paimonCatalog.fileIO(); @@ -94,6 +95,7 @@ public IcebergMigrator( this.icebergMetadata = IcebergMetadata.fromPath( paimonFileIO, icebergPathFactory.toMetadataPath(icebergNewestSnapshotId)); + this.ignoreDelete = ignoreDelete; this.executor = createCachedThreadPool(parallelism, "ICEBERG_MIGRATOR"); } diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java index d1d28c25a8d1..334ae935e860 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -40,6 +41,8 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DataWriter; @@ -47,12 +50,20 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.UUID; import java.util.function.Function; @@ -61,6 +72,7 @@ import static org.apache.paimon.options.CatalogOptions.CACHE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** doc. */ public class IcebergMigrateTest { @@ -79,13 +91,69 @@ public class IcebergMigrateTest { Types.NestedField.required(2, "v", Types.IntegerType.get()), Types.NestedField.required(3, "dt", Types.StringType.get()), Types.NestedField.required(4, "hh", Types.StringType.get())); + Schema iceDeleteSchema = + new Schema( + Types.NestedField.required(1, "k", Types.IntegerType.get()), + Types.NestedField.optional(2, "v", Types.IntegerType.get())); PartitionSpec icePartitionSpec = PartitionSpec.builderFor(iceSchema).identity("dt").identity("hh").build(); @ParameterizedTest(name = "isPartitioned = {0}") @ValueSource(booleans = {true, false}) - public void testMigrateOnlyAddedData(boolean isPartitioned) throws Exception { + public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { + Table icebergTable = createIcebergTable(isPartitioned); + List records1 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "00"), + toIcebergRecord(2, 2, "20240101", "00")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, records1); + } + + List records2 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "01"), + toIcebergRecord(2, 2, "20240101", "01")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + } else { + writeRecordsToIceberg(icebergTable, records2); + } + + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + false, + 1); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + Stream.concat(records1.stream(), records2.stream()) + .map(GenericRecord::toString) + .collect(Collectors.toList())); + } + + @ParameterizedTest(name = "isPartitioned = {0}") + @ValueSource(booleans = {true, false}) + public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { Table icebergTable = createIcebergTable(isPartitioned); List records1 = Stream.of( @@ -109,6 +177,10 @@ public void testMigrateOnlyAddedData(boolean isPartitioned) throws Exception { writeRecordsToIceberg(icebergTable, records2); } + // the file written with records2 will be deleted and generate a delete manifest entry, not + // a delete file + icebergTable.newDelete().deleteFromRowFilter(Expressions.equal("hh", "00")).commit(); + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); context.options().set(CACHE_ENABLED, false); Catalog catalog = CatalogFactory.createCatalog(context); @@ -118,9 +190,78 @@ public void testMigrateOnlyAddedData(boolean isPartitioned) throws Exception { new Path(icebergTable.location(), "metadata"), paiDatabase, paiTable, + false, 1); icebergMigrator.executeMigrate(); + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + records2.stream() + .map(GenericRecord::toString) + .collect(Collectors.toList())); + } + + @ParameterizedTest + @CsvSource({"true, true", "true, false", "false, true", "false, false"}) + public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelete) + throws Exception { + Table icebergTable = createIcebergTable(isPartitioned); + List records1 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "00"), + toIcebergRecord(2, 2, "20240101", "00")) + .collect(Collectors.toList()); + List deleteRecords1 = + Collections.singletonList(toIcebergRecord(1, 1, iceDeleteSchema)); + + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + writeEqualityDeleteFile(icebergTable, deleteRecords1, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, records1); + writeEqualityDeleteFile(icebergTable, deleteRecords1); + } + + List records2 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "01"), + toIcebergRecord(2, 2, "20240101", "01")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + } else { + writeRecordsToIceberg(icebergTable, records2); + } + + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + ignoreDelete, + 1); + if (!ignoreDelete) { + assertThatThrownBy(icebergMigrator::executeMigrate) + .rootCause() + .isInstanceOf(RuntimeException.class) + .hasMessage( + "IcebergMigrator don't support analyzing manifest file with 'DELETE' content. " + + "You can set 'ignore-delete' to ignore manifest file with 'DELETE' content."); + return; + } else { + icebergMigrator.executeMigrate(); + } + FileStoreTable paimonTable = (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); List paiResults = getPaimonResult(paimonTable); @@ -134,21 +275,216 @@ public void testMigrateOnlyAddedData(boolean isPartitioned) throws Exception { .collect(Collectors.toList())); } + @ParameterizedTest(name = "isPartitioned = {0}") + @ValueSource(booleans = {true, false}) + public void testMigrateWithSchemaEvolution(boolean isPartitioned) throws Exception { + Table icebergTable = createIcebergTable(isPartitioned); + + // write base data + List records1 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "00"), + toIcebergRecord(2, 2, "20240101", "00")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, records1); + } + + List records2 = + Stream.of( + toIcebergRecord(1, 1, "20240101", "01"), + toIcebergRecord(2, 2, "20240101", "01")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + } else { + writeRecordsToIceberg(icebergTable, records2); + } + + // testDeleteColumn(icebergTable, isPartitioned); + testAddColumn(icebergTable, isPartitioned); + } + + private void testDeleteColumn(Table icebergTable, boolean isPartitioned) throws Exception { + icebergTable.updateSchema().deleteColumn("v").commit(); + Schema newIceSchema = icebergTable.schema(); + List addedRecords = + Stream.of( + toIcebergRecord(3, "20240101", "00", newIceSchema), + toIcebergRecord(4, "20240101", "00", newIceSchema)) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords); + } + + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + false, + 1); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + Stream.of( + "Record(1, 20240101, 00)", + "Record(2, 20240101, 00)", + "Record(1, 20240101, 01)", + "Record(2, 20240101, 01)", + "Record(3, 20240101, 00)", + "Record(4, 20240101, 00)") + .collect(Collectors.toList())); + } + + private void testAddColumn(Table icebergTable, boolean isPartitioned) throws Exception { + icebergTable.updateSchema().addColumn("v2", Types.IntegerType.get()).commit(); + Schema newIceSchema = icebergTable.schema(); + List addedRecords = + Stream.of( + toIcebergRecord(3, 3, "20240101", "00", 3, newIceSchema), + toIcebergRecord(4, 3, "20240101", "00", 3, newIceSchema)) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords); + } + + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + false, + 1); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + Stream.of( + "Record(1, 1, 20240101, 00, null)", + "Record(2, 2, 20240101, 00, null)", + "Record(1, 1, 20240101, 01, null)", + "Record(2, 2, 20240101, 01, null)", + "Record(3, 3, 20240101, 00, 3)", + "Record(4, 4, 20240101, 00, 3)") + .collect(Collectors.toList())); + } + + @Test + public void testAllDataTypes() throws Exception { + Schema iceAllTypesSchema = + new Schema( + Types.NestedField.required(1, "c1", Types.BooleanType.get()), + Types.NestedField.required(2, "c2", Types.IntegerType.get()), + Types.NestedField.required(3, "c3", Types.LongType.get()), + Types.NestedField.required(4, "c4", Types.FloatType.get()), + Types.NestedField.required(5, "c5", Types.DoubleType.get()), + Types.NestedField.required(6, "c6", Types.DateType.get()), + Types.NestedField.required(7, "c7", Types.StringType.get()), + Types.NestedField.required(8, "c9", Types.BinaryType.get()), + Types.NestedField.required(9, "c11", Types.DecimalType.of(10, 2)), + Types.NestedField.required(10, "c13", Types.TimestampType.withoutZone()), + Types.NestedField.required(11, "c14", Types.TimestampType.withZone())); + Table icebergTable = createIcebergTable(false, iceAllTypesSchema); + GenericRecord record = GenericRecord.create(iceAllTypesSchema); + record.set(0, true); + record.set(1, 1); + record.set(2, 1L); + record.set(3, 1.0F); + record.set(4, 1.0D); + record.set(5, LocalDate.of(2023, 10, 18)); + record.set(6, "test"); + record.set(7, ByteBuffer.wrap(new byte[] {1, 2, 3})); + record.set(8, new BigDecimal("122.50")); + record.set(9, LocalDateTime.now()); + record.set(10, OffsetDateTime.now()); + + writeRecordsToIceberg(icebergTable, iceAllTypesSchema, Collections.singletonList(record)); + + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + false, + 1); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat(paiResults.size()).isEqualTo(1); + } + private Table createIcebergTable(boolean isPartitioned) { + return createIcebergTable(isPartitioned, iceSchema); + } + + private Table createIcebergTable(boolean isPartitioned, Schema icebergSchema) { HadoopCatalog catalog = new HadoopCatalog(new Configuration(), iceTempDir.toString()); TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); if (!isPartitioned) { - return catalog.buildTable(icebergIdentifier, iceSchema) + return catalog.buildTable(icebergIdentifier, icebergSchema) .withPartitionSpec(PartitionSpec.unpartitioned()) .create(); } else { - return catalog.buildTable(icebergIdentifier, iceSchema) + return catalog.buildTable(icebergIdentifier, icebergSchema) .withPartitionSpec(icePartitionSpec) .create(); } } + private GenericRecord toIcebergRecord(int k, String dt, String hh, Schema icebergSchema) { + GenericRecord record = GenericRecord.create(icebergSchema); + record.set(0, k); + record.set(1, dt); + record.set(2, hh); + return record; + } + + private GenericRecord toIcebergRecord( + int k, int v, String dt, String hh, int v2, Schema icebergSchema) { + GenericRecord record = GenericRecord.create(icebergSchema); + record.set(0, k); + record.set(1, v); + record.set(2, dt); + record.set(3, hh); + record.set(4, v2); + return record; + } + private GenericRecord toIcebergRecord(int k, int v, String dt, String hh) { GenericRecord record = GenericRecord.create(iceSchema); record.set(0, k); @@ -158,15 +494,31 @@ private GenericRecord toIcebergRecord(int k, int v, String dt, String hh) { return record; } + private GenericRecord toIcebergRecord(int k, int v, Schema icebergSchema) { + GenericRecord record = GenericRecord.create(iceSchema); + record.set(0, k); + record.set(1, v); + return record; + } + private DataFile writeRecordsToIceberg( Table icebergTable, List records, String... partitionValues) throws IOException { + return writeRecordsToIceberg(icebergTable, iceSchema, records, partitionValues); + } + + private DataFile writeRecordsToIceberg( + Table icebergTable, + Schema icebergSchema, + List records, + String... partitionValues) + throws IOException { String filepath = icebergTable.location() + "/" + UUID.randomUUID(); OutputFile file = icebergTable.io().newOutputFile(filepath); DataWriter dataWriter = Parquet.writeData(file) - .schema(iceSchema) + .schema(icebergSchema) .createWriterFunc(GenericParquetWriter::buildWriter) .overwrite() .withSpec(PartitionSpec.unpartitioned()) @@ -175,7 +527,7 @@ private DataFile writeRecordsToIceberg( if (partitionValues.length != 0) { dataWriter = Parquet.writeData(file) - .schema(iceSchema) + .schema(icebergSchema) .createWriterFunc(GenericParquetWriter::buildWriter) .overwrite() .withSpec(icePartitionSpec) @@ -200,6 +552,45 @@ private DataFile writeRecordsToIceberg( return dataFile; } + private void writeEqualityDeleteFile( + Table icebergTable, List deleteRecords, String... partitionValues) + throws IOException { + String filepath = icebergTable.location() + "/" + UUID.randomUUID(); + OutputFile file = icebergTable.io().newOutputFile(filepath); + + EqualityDeleteWriter deleteWriter = + Parquet.writeDeletes(file) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .rowSchema(iceDeleteSchema) + .withSpec(PartitionSpec.unpartitioned()) + .equalityFieldIds(1) + .buildEqualityWriter(); + if (partitionValues.length != 0) { + deleteWriter = + Parquet.writeDeletes(file) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .rowSchema(iceDeleteSchema) + .withSpec(icePartitionSpec) + .withPartition( + partitionKey( + icePartitionSpec, + icebergTable, + partitionValues[0], + partitionValues[1])) + .equalityFieldIds(1) + .buildEqualityWriter(); + } + + try (EqualityDeleteWriter closableWriter = deleteWriter) { + closableWriter.write(deleteRecords); + } + + DeleteFile deleteFile = deleteWriter.toDeleteFile(); + icebergTable.newRowDelta().addDeletes(deleteFile).commit(); + } + private PartitionKey partitionKey( PartitionSpec spec, Table icergTable, String... partitionValues) { Record record = @@ -231,11 +622,9 @@ private List getIcebergResult( Function icebergRecordToString) throws Exception { HadoopCatalog icebergCatalog = - new HadoopCatalog(new Configuration(), "/Users/catyeah/testHome/icebergtest"); - TableIdentifier icebergIdentifier = TableIdentifier.of("mydb.db", "t"); + new HadoopCatalog(new Configuration(), iceTempDir.toString()); + TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); org.apache.iceberg.Table icebergTable = icebergCatalog.loadTable(icebergIdentifier); - List manifestFiles = - icebergTable.currentSnapshot().allManifests(icebergTable.io()); CloseableIterable result = query.apply(icebergTable); List actual = new ArrayList<>(); From f3a9d5bec5900c4498fb55e6d713469a145c00bb Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 11:41:04 +0800 Subject: [PATCH 08/30] [mark] unit tests including add and rename column --- .../paimon/iceberg/IcebergMigrateTest.java | 252 +++++++++++------- 1 file changed, 152 insertions(+), 100 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java index 334ae935e860..21678c6386d3 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -37,9 +37,11 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcWriter; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.expressions.Expressions; @@ -47,6 +49,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; @@ -103,15 +106,16 @@ public class IcebergMigrateTest { @ValueSource(booleans = {true, false}) public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { Table icebergTable = createIcebergTable(isPartitioned); + String format = "parquet"; List records1 = Stream.of( toIcebergRecord(1, 1, "20240101", "00"), toIcebergRecord(2, 2, "20240101", "00")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + writeRecordsToIceberg(icebergTable, format, records1, "20240101", "00"); } else { - writeRecordsToIceberg(icebergTable, records1); + writeRecordsToIceberg(icebergTable, format, records1); } List records2 = @@ -120,9 +124,9 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { toIcebergRecord(2, 2, "20240101", "01")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + writeRecordsToIceberg(icebergTable, format, records2, "20240101", "01"); } else { - writeRecordsToIceberg(icebergTable, records2); + writeRecordsToIceberg(icebergTable, format, records2); } CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); @@ -155,15 +159,16 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { @ValueSource(booleans = {true, false}) public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { Table icebergTable = createIcebergTable(isPartitioned); + String format = "parquet"; List records1 = Stream.of( toIcebergRecord(1, 1, "20240101", "00"), toIcebergRecord(2, 2, "20240101", "00")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + writeRecordsToIceberg(icebergTable, format, records1, "20240101", "00"); } else { - writeRecordsToIceberg(icebergTable, records1); + writeRecordsToIceberg(icebergTable, format, records1); } List records2 = @@ -172,9 +177,9 @@ public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { toIcebergRecord(2, 2, "20240101", "01")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + writeRecordsToIceberg(icebergTable, format, records2, "20240101", "01"); } else { - writeRecordsToIceberg(icebergTable, records2); + writeRecordsToIceberg(icebergTable, format, records2); } // the file written with records2 will be deleted and generate a delete manifest entry, not @@ -211,7 +216,9 @@ public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { @CsvSource({"true, true", "true, false", "false, true", "false, false"}) public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelete) throws Exception { + // only support create delete file with parquet format Table icebergTable = createIcebergTable(isPartitioned); + String format = "parquet"; List records1 = Stream.of( toIcebergRecord(1, 1, "20240101", "00"), @@ -221,10 +228,10 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet Collections.singletonList(toIcebergRecord(1, 1, iceDeleteSchema)); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + writeRecordsToIceberg(icebergTable, format, records1, "20240101", "00"); writeEqualityDeleteFile(icebergTable, deleteRecords1, "20240101", "00"); } else { - writeRecordsToIceberg(icebergTable, records1); + writeRecordsToIceberg(icebergTable, format, records1); writeEqualityDeleteFile(icebergTable, deleteRecords1); } @@ -234,9 +241,9 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet toIcebergRecord(2, 2, "20240101", "01")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + writeRecordsToIceberg(icebergTable, format, records2, "20240101", "01"); } else { - writeRecordsToIceberg(icebergTable, records2); + writeRecordsToIceberg(icebergTable, format, records2); } CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); @@ -279,6 +286,7 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet @ValueSource(booleans = {true, false}) public void testMigrateWithSchemaEvolution(boolean isPartitioned) throws Exception { Table icebergTable = createIcebergTable(isPartitioned); + String format = "parquet"; // write base data List records1 = @@ -287,9 +295,9 @@ public void testMigrateWithSchemaEvolution(boolean isPartitioned) throws Excepti toIcebergRecord(2, 2, "20240101", "00")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records1, "20240101", "00"); + writeRecordsToIceberg(icebergTable, format, records1, "20240101", "00"); } else { - writeRecordsToIceberg(icebergTable, records1); + writeRecordsToIceberg(icebergTable, format, records1); } List records2 = @@ -298,16 +306,18 @@ public void testMigrateWithSchemaEvolution(boolean isPartitioned) throws Excepti toIcebergRecord(2, 2, "20240101", "01")) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, records2, "20240101", "01"); + writeRecordsToIceberg(icebergTable, format, records2, "20240101", "01"); } else { - writeRecordsToIceberg(icebergTable, records2); + writeRecordsToIceberg(icebergTable, format, records2); } - // testDeleteColumn(icebergTable, isPartitioned); - testAddColumn(icebergTable, isPartitioned); + // testDeleteColumn(icebergTable, format, isPartitioned); + // testAddColumn(icebergTable, format, isPartitioned); + testRenameColumn(icebergTable, format, isPartitioned); } - private void testDeleteColumn(Table icebergTable, boolean isPartitioned) throws Exception { + private void testDeleteColumn(Table icebergTable, String format, boolean isPartitioned) + throws Exception { icebergTable.updateSchema().deleteColumn("v").commit(); Schema newIceSchema = icebergTable.schema(); List addedRecords = @@ -316,9 +326,9 @@ private void testDeleteColumn(Table icebergTable, boolean isPartitioned) throws toIcebergRecord(4, "20240101", "00", newIceSchema)) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords, "20240101", "00"); + writeRecordsToIceberg(icebergTable, format, addedRecords, "20240101", "00"); } else { - writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords); + writeRecordsToIceberg(icebergTable, format, addedRecords); } CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); @@ -352,18 +362,19 @@ private void testDeleteColumn(Table icebergTable, boolean isPartitioned) throws .collect(Collectors.toList())); } - private void testAddColumn(Table icebergTable, boolean isPartitioned) throws Exception { + private void testAddColumn(Table icebergTable, String format, boolean isPartitioned) + throws Exception { icebergTable.updateSchema().addColumn("v2", Types.IntegerType.get()).commit(); Schema newIceSchema = icebergTable.schema(); List addedRecords = Stream.of( - toIcebergRecord(3, 3, "20240101", "00", 3, newIceSchema), - toIcebergRecord(4, 3, "20240101", "00", 3, newIceSchema)) + toIcebergRecord(newIceSchema, 3, 3, "20240101", "00", 3), + toIcebergRecord(newIceSchema, 4, 4, "20240101", "00", 3)) .collect(Collectors.toList()); if (isPartitioned) { - writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords, "20240101", "00"); + writeRecordsToIceberg(icebergTable, format, addedRecords, "20240101", "00"); } else { - writeRecordsToIceberg(icebergTable, newIceSchema, addedRecords); + writeRecordsToIceberg(icebergTable, format, addedRecords); } CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); @@ -388,15 +399,61 @@ private void testAddColumn(Table icebergTable, boolean isPartitioned) throws Exc .collect(Collectors.toList())) .hasSameElementsAs( Stream.of( - "Record(1, 1, 20240101, 00, null)", - "Record(2, 2, 20240101, 00, null)", - "Record(1, 1, 20240101, 01, null)", - "Record(2, 2, 20240101, 01, null)", + "Record(1, 1, 20240101, 00, NULL)", + "Record(2, 2, 20240101, 00, NULL)", + "Record(1, 1, 20240101, 01, NULL)", + "Record(2, 2, 20240101, 01, NULL)", "Record(3, 3, 20240101, 00, 3)", "Record(4, 4, 20240101, 00, 3)") .collect(Collectors.toList())); } + private void testRenameColumn(Table icebergTable, String format, boolean isPartitioned) + throws Exception { + icebergTable.updateSchema().renameColumn("v", "v2").commit(); + Schema newIceSchema = icebergTable.schema(); + List addedRecords = + Stream.of( + toIcebergRecord(newIceSchema, 3, 3, "20240101", "00"), + toIcebergRecord(newIceSchema, 4, 4, "20240101", "00")) + .collect(Collectors.toList()); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, format, addedRecords, "20240101", "00"); + } else { + writeRecordsToIceberg(icebergTable, format, addedRecords); + } + + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + Catalog catalog = CatalogFactory.createCatalog(context); + IcebergMigrator icebergMigrator = + new IcebergMigrator( + catalog, + new Path(icebergTable.location(), "metadata"), + paiDatabase, + paiTable, + false, + 1); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + Stream.of( + "Record(1, 1, 20240101, 00)", + "Record(2, 2, 20240101, 00)", + "Record(1, 1, 20240101, 01)", + "Record(2, 2, 20240101, 01)", + "Record(3, 3, 20240101, 00)", + "Record(4, 4, 20240101, 00)") + .collect(Collectors.toList())); + } + @Test public void testAllDataTypes() throws Exception { Schema iceAllTypesSchema = @@ -413,20 +470,23 @@ public void testAllDataTypes() throws Exception { Types.NestedField.required(10, "c13", Types.TimestampType.withoutZone()), Types.NestedField.required(11, "c14", Types.TimestampType.withZone())); Table icebergTable = createIcebergTable(false, iceAllTypesSchema); - GenericRecord record = GenericRecord.create(iceAllTypesSchema); - record.set(0, true); - record.set(1, 1); - record.set(2, 1L); - record.set(3, 1.0F); - record.set(4, 1.0D); - record.set(5, LocalDate.of(2023, 10, 18)); - record.set(6, "test"); - record.set(7, ByteBuffer.wrap(new byte[] {1, 2, 3})); - record.set(8, new BigDecimal("122.50")); - record.set(9, LocalDateTime.now()); - record.set(10, OffsetDateTime.now()); - - writeRecordsToIceberg(icebergTable, iceAllTypesSchema, Collections.singletonList(record)); + String format = "parquet"; + GenericRecord record = + toIcebergRecord( + iceAllTypesSchema, + true, + 1, + 1L, + 1.0F, + 1.0D, + LocalDate.of(2023, 10, 18), + "test", + ByteBuffer.wrap(new byte[] {1, 2, 3}), + new BigDecimal("122.50"), + LocalDateTime.now(), + OffsetDateTime.now()); + + writeRecordsToIceberg(icebergTable, format, Collections.singletonList(record)); CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); context.options().set(CACHE_ENABLED, false); @@ -466,50 +526,64 @@ private Table createIcebergTable(boolean isPartitioned, Schema icebergSchema) { } } - private GenericRecord toIcebergRecord(int k, String dt, String hh, Schema icebergSchema) { + private GenericRecord toIcebergRecord(Schema icebergSchema, Object... values) { GenericRecord record = GenericRecord.create(icebergSchema); - record.set(0, k); - record.set(1, dt); - record.set(2, hh); - return record; - } - - private GenericRecord toIcebergRecord( - int k, int v, String dt, String hh, int v2, Schema icebergSchema) { - GenericRecord record = GenericRecord.create(icebergSchema); - record.set(0, k); - record.set(1, v); - record.set(2, dt); - record.set(3, hh); - record.set(4, v2); - return record; - } - - private GenericRecord toIcebergRecord(int k, int v, String dt, String hh) { - GenericRecord record = GenericRecord.create(iceSchema); - record.set(0, k); - record.set(1, v); - record.set(2, dt); - record.set(3, hh); + for (int i = 0; i < values.length; i++) { + record.set(i, values[i]); + } return record; } - private GenericRecord toIcebergRecord(int k, int v, Schema icebergSchema) { - GenericRecord record = GenericRecord.create(iceSchema); - record.set(0, k); - record.set(1, v); - return record; + private GenericRecord toIcebergRecord(Object... values) { + return toIcebergRecord(iceSchema, values); } - private DataFile writeRecordsToIceberg( - Table icebergTable, List records, String... partitionValues) + private DataWriter createIcebergDataWriter( + Table icebergTable, String format, OutputFile file, String... partitionValues) throws IOException { - return writeRecordsToIceberg(icebergTable, iceSchema, records, partitionValues); + Schema schema = icebergTable.schema(); + PartitionSpec partitionSpec = icebergTable.spec(); + PartitionKey partitionKey = + partitionValues.length == 0 + ? null + : partitionKey( + icePartitionSpec, + icebergTable, + partitionValues[0], + partitionValues[1]); + switch (format) { + case "parquet": + return Parquet.writeData(file) + .schema(schema) + .createWriterFunc(GenericParquetWriter::buildWriter) + .overwrite() + .withSpec(partitionSpec) + .withPartition(partitionKey) + .build(); + case "avro": + return Avro.writeData(file) + .schema(schema) + .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) + .overwrite() + .withSpec(partitionSpec) + .withPartition(partitionKey) + .build(); + case "orc": + return ORC.writeData(file) + .schema(schema) + .createWriterFunc(GenericOrcWriter::buildWriter) + .overwrite() + .withSpec(partitionSpec) + .withPartition(partitionKey) + .build(); + default: + throw new IllegalArgumentException("Unsupported format: " + format); + } } private DataFile writeRecordsToIceberg( Table icebergTable, - Schema icebergSchema, + String format, List records, String... partitionValues) throws IOException { @@ -517,29 +591,7 @@ private DataFile writeRecordsToIceberg( OutputFile file = icebergTable.io().newOutputFile(filepath); DataWriter dataWriter = - Parquet.writeData(file) - .schema(icebergSchema) - .createWriterFunc(GenericParquetWriter::buildWriter) - .overwrite() - .withSpec(PartitionSpec.unpartitioned()) - .build(); - - if (partitionValues.length != 0) { - dataWriter = - Parquet.writeData(file) - .schema(icebergSchema) - .createWriterFunc(GenericParquetWriter::buildWriter) - .overwrite() - .withSpec(icePartitionSpec) - .withPartition( - partitionKey( - icePartitionSpec, - icebergTable, - partitionValues[0], - partitionValues[1])) - .build(); - } - + createIcebergDataWriter(icebergTable, format, file, partitionValues); try { for (GenericRecord r : records) { dataWriter.write(r); From 4bf44e9bc92bcbdf3c2e482b6f0e4e29f4235d64 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 13:49:14 +0800 Subject: [PATCH 09/30] fix test --- .../paimon/iceberg/IcebergMigrator.java | 2 +- .../paimon/iceberg/IcebergMigrateTest.java | 121 +----------------- 2 files changed, 5 insertions(+), 118 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java index dd084ccfefb3..cc9de9be10aa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java @@ -111,7 +111,6 @@ public void executeMigrate() throws Exception { try { FileStoreTable paimonTable = (FileStoreTable) paimonCatalog.getTable(paimonIdentifier); - // TODO: IcebergManifestFile.create() is suitable here? IcebergManifestFile manifestFile = IcebergManifestFile.create(paimonTable, icebergPathFactory); IcebergManifestList manifestList = @@ -186,6 +185,7 @@ public void executeMigrate() throws Exception { } try (BatchTableCommit commit = paimonTable.newBatchWriteBuilder().newCommit()) { commit.commit(new ArrayList<>(commitMessages)); + LOG.info("paimon commit success! Iceberg data files has been migrated to paimon."); } } catch (Exception e) { paimonCatalog.dropTable(paimonIdentifier, true); diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java index 21678c6386d3..c56af5412d28 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -46,7 +46,6 @@ import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.orc.ORC; @@ -69,7 +68,6 @@ import java.util.Collections; import java.util.List; import java.util.UUID; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -311,9 +309,8 @@ public void testMigrateWithSchemaEvolution(boolean isPartitioned) throws Excepti writeRecordsToIceberg(icebergTable, format, records2); } - // testDeleteColumn(icebergTable, format, isPartitioned); - // testAddColumn(icebergTable, format, isPartitioned); - testRenameColumn(icebergTable, format, isPartitioned); + // TODO: currently only support schema evolution of deleting columns + testDeleteColumn(icebergTable, format, isPartitioned); } private void testDeleteColumn(Table icebergTable, String format, boolean isPartitioned) @@ -362,98 +359,6 @@ private void testDeleteColumn(Table icebergTable, String format, boolean isParti .collect(Collectors.toList())); } - private void testAddColumn(Table icebergTable, String format, boolean isPartitioned) - throws Exception { - icebergTable.updateSchema().addColumn("v2", Types.IntegerType.get()).commit(); - Schema newIceSchema = icebergTable.schema(); - List addedRecords = - Stream.of( - toIcebergRecord(newIceSchema, 3, 3, "20240101", "00", 3), - toIcebergRecord(newIceSchema, 4, 4, "20240101", "00", 3)) - .collect(Collectors.toList()); - if (isPartitioned) { - writeRecordsToIceberg(icebergTable, format, addedRecords, "20240101", "00"); - } else { - writeRecordsToIceberg(icebergTable, format, addedRecords); - } - - CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); - IcebergMigrator icebergMigrator = - new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), - paiDatabase, - paiTable, - false, - 1); - icebergMigrator.executeMigrate(); - - FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); - List paiResults = getPaimonResult(paimonTable); - assertThat( - paiResults.stream() - .map(row -> String.format("Record(%s)", row)) - .collect(Collectors.toList())) - .hasSameElementsAs( - Stream.of( - "Record(1, 1, 20240101, 00, NULL)", - "Record(2, 2, 20240101, 00, NULL)", - "Record(1, 1, 20240101, 01, NULL)", - "Record(2, 2, 20240101, 01, NULL)", - "Record(3, 3, 20240101, 00, 3)", - "Record(4, 4, 20240101, 00, 3)") - .collect(Collectors.toList())); - } - - private void testRenameColumn(Table icebergTable, String format, boolean isPartitioned) - throws Exception { - icebergTable.updateSchema().renameColumn("v", "v2").commit(); - Schema newIceSchema = icebergTable.schema(); - List addedRecords = - Stream.of( - toIcebergRecord(newIceSchema, 3, 3, "20240101", "00"), - toIcebergRecord(newIceSchema, 4, 4, "20240101", "00")) - .collect(Collectors.toList()); - if (isPartitioned) { - writeRecordsToIceberg(icebergTable, format, addedRecords, "20240101", "00"); - } else { - writeRecordsToIceberg(icebergTable, format, addedRecords); - } - - CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); - IcebergMigrator icebergMigrator = - new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), - paiDatabase, - paiTable, - false, - 1); - icebergMigrator.executeMigrate(); - - FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); - List paiResults = getPaimonResult(paimonTable); - assertThat( - paiResults.stream() - .map(row -> String.format("Record(%s)", row)) - .collect(Collectors.toList())) - .hasSameElementsAs( - Stream.of( - "Record(1, 1, 20240101, 00)", - "Record(2, 2, 20240101, 00)", - "Record(1, 1, 20240101, 01)", - "Record(2, 2, 20240101, 01)", - "Record(3, 3, 20240101, 00)", - "Record(4, 4, 20240101, 00)") - .collect(Collectors.toList())); - } - @Test public void testAllDataTypes() throws Exception { Schema iceAllTypesSchema = @@ -551,6 +456,7 @@ private DataWriter createIcebergDataWriter( icebergTable, partitionValues[0], partitionValues[1]); + // TODO: currently only support "parquet" format switch (format) { case "parquet": return Parquet.writeData(file) @@ -581,7 +487,7 @@ private DataWriter createIcebergDataWriter( } } - private DataFile writeRecordsToIceberg( + private void writeRecordsToIceberg( Table icebergTable, String format, List records, @@ -601,7 +507,6 @@ private DataFile writeRecordsToIceberg( } DataFile dataFile = dataWriter.toDataFile(); icebergTable.newAppend().appendFile(dataFile).commit(); - return dataFile; } private void writeEqualityDeleteFile( @@ -668,22 +573,4 @@ private List getPaimonResult(FileStoreTable paimonTable) throws Exceptio return result; } } - - private List getIcebergResult( - Function> query, - Function icebergRecordToString) - throws Exception { - HadoopCatalog icebergCatalog = - new HadoopCatalog(new Configuration(), iceTempDir.toString()); - TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); - org.apache.iceberg.Table icebergTable = icebergCatalog.loadTable(icebergIdentifier); - - CloseableIterable result = query.apply(icebergTable); - List actual = new ArrayList<>(); - for (Record record : result) { - actual.add(icebergRecordToString.apply(record)); - } - result.close(); - return actual; - } } From eaf0efe4d20fb2e4154f6141fa8f7fbf717787cc Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 17:33:50 +0800 Subject: [PATCH 10/30] add doc and add dependency in paimon-core/pom.xml --- paimon-core/pom.xml | 14 ++++++++++++++ .../org/apache/paimon/iceberg/IcebergMigrator.java | 2 +- .../apache/paimon/iceberg/IcebergMigrateTest.java | 2 +- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index c5a82b14d60d..697079d5af21 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -225,6 +225,20 @@ under the License. test + + org.apache.iceberg + iceberg-parquet + ${iceberg.version} + test + + + + org.apache.iceberg + iceberg-orc + ${iceberg.version} + test + + diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java index cc9de9be10aa..bc7382f11b98 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java @@ -58,7 +58,7 @@ import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; -/** doc. */ +/** migrate hive table to paimon table. */ public class IcebergMigrator implements Migrator { private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrator.class); private static final String VERSION_HINT_FILENAME = "version-hint.text"; diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java index c56af5412d28..898484c765a2 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -75,7 +75,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** doc. */ +/** Tests for {@link IcebergMigrator}. */ public class IcebergMigrateTest { @TempDir java.nio.file.Path iceTempDir; @TempDir java.nio.file.Path paiTempDir; From ff8db0ad0a3bd4540924f3c0fe349bc76b7c1f69 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 5 Dec 2024 12:56:09 +0800 Subject: [PATCH 11/30] resolve conflicts --- .../iceberg/metadata/IcebergDataField.java | 21 +++++++------------ 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 1b0d72c9479e..c5588b1d3769 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -199,19 +199,16 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { } } - // TODO: two datatype("uuid", "unknown") has not been processed public DataType getDataType() { String simpleType = type.toString(); - String typePrefix = - !simpleType.contains("(") - ? simpleType - : simpleType.substring(0, simpleType.indexOf("(")); String delimiter = "("; - if (type.contains("[")) { + if (simpleType.contains("[")) { delimiter = "["; } String typePrefix = - !type.contains(delimiter) ? type : type.substring(0, type.indexOf(delimiter)); + !simpleType.contains(delimiter) + ? simpleType + : simpleType.substring(0, simpleType.indexOf(delimiter)); switch (typePrefix) { case "boolean": return new BooleanType(!required); @@ -233,9 +230,7 @@ public DataType getDataType() { int fixedLength = Integer.parseInt( simpleType.substring( - simpleType.indexOf("(") + 1, simpleType.indexOf(")"))); - return new BinaryType(required, fixedLength); - Integer.parseInt(type.substring(type.indexOf("[") + 1, type.indexOf("]"))); + simpleType.indexOf("[") + 1, simpleType.indexOf("]"))); return new BinaryType(!required, fixedLength); case "uuid": // https://iceberg.apache.org/spec/?h=vector#primitive-types @@ -247,12 +242,10 @@ public DataType getDataType() { simpleType.substring( simpleType.indexOf("(") + 1, simpleType.indexOf(","))); int scale = - Integer.parseInt(type.substring(type.indexOf(",") + 2, type.indexOf(")"))); - return new DecimalType(!required, precision, scale); Integer.parseInt( simpleType.substring( - simpleType.indexOf(",") + 1, simpleType.indexOf(")"))); - return new DecimalType(required, precision, scale); + simpleType.indexOf(",") + 2, simpleType.indexOf(")"))); + return new DecimalType(!required, precision, scale); case "timestamp": return new TimestampType(!required, 6); case "timestamptz": From 1dd792588fc71ea47f866c20fb9d76cc9a46a462 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 5 Dec 2024 13:12:05 +0800 Subject: [PATCH 12/30] fix check style --- .../org/apache/paimon/iceberg/manifest/IcebergManifestFile.java | 1 - 1 file changed, 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java index c95d7be1a051..4553a1c850a0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java @@ -19,7 +19,6 @@ package org.apache.paimon.iceberg.manifest; import org.apache.paimon.annotation.VisibleForTesting; -import org.apache.paimon.CoreOptions; import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FormatReaderFactory; From 625b67b4f146b2ef4818b8c7450d311f338befb5 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Tue, 10 Dec 2024 14:26:54 +0800 Subject: [PATCH 13/30] [improve] get iceberg metadata from iceberg table instead from a certain metadata path --- .../paimon/iceberg/IcebergMigrator.java | 80 ++++++++++------- .../paimon/iceberg/IcebergPathFactory.java | 4 + .../paimon/iceberg/IcebergMigrateTest.java | 90 ++++++++++++------- 3 files changed, 113 insertions(+), 61 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java index bc7382f11b98..3afaa88ae6bf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java @@ -42,10 +42,10 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.types.DataField; +import org.apache.iceberg.catalog.TableIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -61,18 +61,24 @@ /** migrate hive table to paimon table. */ public class IcebergMigrator implements Migrator { private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrator.class); - private static final String VERSION_HINT_FILENAME = "version-hint.text"; private final ThreadPoolExecutor executor; private final Catalog paimonCatalog; private final FileIO paimonFileIO; private final String paimonDatabaseName; - private final String paimonTableNameame; - - private final int icebergNewestSnapshotId; - // Path factory for iceberg metadata - private final IcebergPathFactory icebergPathFactory; + private final String paimonTableName; + + private final org.apache.iceberg.catalog.Catalog icebergCatalog; + private final String icebergDatabaseName; + private final String icebergTableName; + + // metadata path factory for iceberg metadata + private IcebergPathFactory icebergMetaPathFactory; + // iceberg table latest snapshot id + private long icebergLatestSnapshotId; + // iceberg latest snapshot metadata file name + private String icebergLatestMetaDataFileName; // metadata for newest iceberg snapshot private final IcebergMetadata icebergMetadata; @@ -80,30 +86,56 @@ public class IcebergMigrator implements Migrator { public IcebergMigrator( Catalog paimonCatalog, - Path icebergMetaPath, String paimonDatabaseName, - String paimonTableNameame, + String paimonTableName, + org.apache.iceberg.catalog.Catalog icebergCatalog, + String icebergDatabaseName, + String icebergTableName, boolean ignoreDelete, Integer parallelism) { this.paimonCatalog = paimonCatalog; this.paimonFileIO = paimonCatalog.fileIO(); this.paimonDatabaseName = paimonDatabaseName; - this.paimonTableNameame = paimonTableNameame; + this.paimonTableName = paimonTableName; + + this.icebergCatalog = icebergCatalog; + this.icebergDatabaseName = icebergDatabaseName; + this.icebergTableName = icebergTableName; - this.icebergPathFactory = new IcebergPathFactory(icebergMetaPath); - this.icebergNewestSnapshotId = getIcebergNewestSnapshotId(); + initIcebergInfoForPaimon(); this.icebergMetadata = IcebergMetadata.fromPath( - paimonFileIO, icebergPathFactory.toMetadataPath(icebergNewestSnapshotId)); + paimonFileIO, + icebergMetaPathFactory.toMetadataPath(icebergLatestMetaDataFileName)); this.ignoreDelete = ignoreDelete; this.executor = createCachedThreadPool(parallelism, "ICEBERG_MIGRATOR"); } + public void initIcebergInfoForPaimon() { + try { + LOG.info("Try to load iceberg table to get the metadata path of latest snapshot."); + org.apache.iceberg.Table icebergTable = + icebergCatalog.loadTable( + TableIdentifier.of(icebergDatabaseName, icebergTableName)); + org.apache.iceberg.TableMetadata currentMetadata = + ((org.apache.iceberg.BaseTable) icebergTable).operations().current(); + Path metadataLocation = new Path(currentMetadata.metadataFileLocation()); + LOG.info("iceberg latest snapshot metadata file location: {}", metadataLocation); + this.icebergMetaPathFactory = new IcebergPathFactory(metadataLocation.getParent()); + this.icebergLatestMetaDataFileName = metadataLocation.getName(); + this.icebergLatestSnapshotId = currentMetadata.currentSnapshot().snapshotId(); + LOG.info("get iceberg metadata path successfully."); + } catch (Exception e) { + throw new RuntimeException( + "failed to get iceberg metadata path by loading iceberg table."); + } + } + @Override public void executeMigrate() throws Exception { Schema paimonSchema = icebergSchemaToPaimonSchema(icebergMetadata); - Identifier paimonIdentifier = Identifier.create(paimonDatabaseName, paimonTableNameame); + Identifier paimonIdentifier = Identifier.create(paimonDatabaseName, paimonTableName); paimonCatalog.createDatabase(paimonDatabaseName, false); paimonCatalog.createTable(paimonIdentifier, paimonSchema, false); @@ -112,9 +144,9 @@ public void executeMigrate() throws Exception { FileStoreTable paimonTable = (FileStoreTable) paimonCatalog.getTable(paimonIdentifier); IcebergManifestFile manifestFile = - IcebergManifestFile.create(paimonTable, icebergPathFactory); + IcebergManifestFile.create(paimonTable, icebergMetaPathFactory); IcebergManifestList manifestList = - IcebergManifestList.create(paimonTable, icebergPathFactory); + IcebergManifestList.create(paimonTable, icebergMetaPathFactory); List icebergManifestFileMetas = manifestList.read(icebergMetadata.currentSnapshot().manifestList()); @@ -131,8 +163,8 @@ public void executeMigrate() throws Exception { if (icebergEntries.isEmpty()) { LOG.info( "No live manifest entry in iceberg table for snapshot {}, iceberg table meta path is {}.", - icebergNewestSnapshotId, - icebergPathFactory.toMetadataPath(icebergNewestSnapshotId)); + icebergLatestSnapshotId, + icebergMetaPathFactory.toMetadataPath(icebergLatestSnapshotId)); return; } @@ -199,18 +231,6 @@ public void deleteOriginTable(boolean delete) throws Exception {} @Override public void renameTable(boolean ignoreIfNotExists) throws Exception {} - public int getIcebergNewestSnapshotId() { - Path versionHintPath = - new Path(icebergPathFactory.metadataDirectory(), VERSION_HINT_FILENAME); - try { - return Integer.parseInt(paimonFileIO.readFileUtf8(versionHintPath)); - } catch (IOException e) { - throw new RuntimeException( - "read iceberg version-hint.text failed. Iceberg metadata path: " - + icebergPathFactory.metadataDirectory()); - } - } - public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { // get iceberg current schema IcebergSchema icebergSchema = diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergPathFactory.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergPathFactory.java index 74d2e8e48f1b..9025dbe87ad0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergPathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergPathFactory.java @@ -68,6 +68,10 @@ public Path toMetadataPath(long snapshotId) { return new Path(metadataDirectory(), String.format("v%d.metadata.json", snapshotId)); } + public Path toMetadataPath(String metadataName) { + return new Path(metadataDirectory(), metadataName); + } + public Stream getAllMetadataPathBefore(FileIO fileIO, long snapshotId) throws IOException { return FileUtils.listVersionedFileStatus(fileIO, metadataDirectory, "v") diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java index 898484c765a2..89a63c389f69 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java @@ -31,6 +31,7 @@ import org.apache.paimon.table.source.TableRead; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionKey; @@ -45,13 +46,13 @@ import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -66,20 +67,25 @@ import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; import static org.apache.paimon.options.CatalogOptions.CACHE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; /** Tests for {@link IcebergMigrator}. */ public class IcebergMigrateTest { @TempDir java.nio.file.Path iceTempDir; @TempDir java.nio.file.Path paiTempDir; + Catalog paiCatalog; + + org.apache.iceberg.catalog.Catalog icebergCatalog; String iceDatabase = "ice_db"; String iceTable = "ice_t"; @@ -100,6 +106,12 @@ public class IcebergMigrateTest { PartitionSpec icePartitionSpec = PartitionSpec.builderFor(iceSchema).identity("dt").identity("hh").build(); + @BeforeEach + public void beforeEach() throws Exception { + paiCatalog = createPaimonCatalog(); + icebergCatalog = createIcebergCatalog(); + } + @ParameterizedTest(name = "isPartitioned = {0}") @ValueSource(booleans = {true, false}) public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { @@ -127,21 +139,20 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { writeRecordsToIceberg(icebergTable, format, records2); } - CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); IcebergMigrator icebergMigrator = new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), + paiCatalog, paiDatabase, paiTable, + icebergCatalog, + iceDatabase, + iceTable, false, 1); icebergMigrator.executeMigrate(); FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); List paiResults = getPaimonResult(paimonTable); assertThat( paiResults.stream() @@ -184,21 +195,20 @@ public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { // a delete file icebergTable.newDelete().deleteFromRowFilter(Expressions.equal("hh", "00")).commit(); - CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); IcebergMigrator icebergMigrator = new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), + paiCatalog, paiDatabase, paiTable, + icebergCatalog, + iceDatabase, + iceTable, false, 1); icebergMigrator.executeMigrate(); FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); List paiResults = getPaimonResult(paimonTable); assertThat( paiResults.stream() @@ -244,15 +254,14 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet writeRecordsToIceberg(icebergTable, format, records2); } - CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); IcebergMigrator icebergMigrator = new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), + paiCatalog, paiDatabase, paiTable, + icebergCatalog, + iceDatabase, + iceTable, ignoreDelete, 1); if (!ignoreDelete) { @@ -268,7 +277,7 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet } FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); List paiResults = getPaimonResult(paimonTable); assertThat( paiResults.stream() @@ -328,21 +337,20 @@ private void testDeleteColumn(Table icebergTable, String format, boolean isParti writeRecordsToIceberg(icebergTable, format, addedRecords); } - CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); - context.options().set(CACHE_ENABLED, false); - Catalog catalog = CatalogFactory.createCatalog(context); IcebergMigrator icebergMigrator = new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), + paiCatalog, paiDatabase, paiTable, + icebergCatalog, + iceDatabase, + iceTable, false, 1); icebergMigrator.executeMigrate(); FileStoreTable paimonTable = - (FileStoreTable) catalog.getTable(Identifier.create(paiDatabase, paiTable)); + (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); List paiResults = getPaimonResult(paimonTable); assertThat( paiResults.stream() @@ -398,10 +406,12 @@ public void testAllDataTypes() throws Exception { Catalog catalog = CatalogFactory.createCatalog(context); IcebergMigrator icebergMigrator = new IcebergMigrator( - catalog, - new Path(icebergTable.location(), "metadata"), + paiCatalog, paiDatabase, paiTable, + icebergCatalog, + iceDatabase, + iceTable, false, 1); icebergMigrator.executeMigrate(); @@ -412,20 +422,38 @@ public void testAllDataTypes() throws Exception { assertThat(paiResults.size()).isEqualTo(1); } + private org.apache.iceberg.catalog.Catalog createIcebergCatalog() { + Map icebergCatalogOptions = new HashMap<>(); + icebergCatalogOptions.put("type", "hadoop"); + icebergCatalogOptions.put("warehouse", iceTempDir.toString()); + + return CatalogUtil.buildIcebergCatalog( + "iceberg_catalog", icebergCatalogOptions, new Configuration()); + } + + private Catalog createPaimonCatalog() { + CatalogContext context = CatalogContext.create(new Path(paiTempDir.toString())); + context.options().set(CACHE_ENABLED, false); + return CatalogFactory.createCatalog(context); + } + private Table createIcebergTable(boolean isPartitioned) { return createIcebergTable(isPartitioned, iceSchema); } private Table createIcebergTable(boolean isPartitioned, Schema icebergSchema) { - HadoopCatalog catalog = new HadoopCatalog(new Configuration(), iceTempDir.toString()); + // HadoopCatalog catalog = new HadoopCatalog(new Configuration(), + // iceTempDir.toString()); TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); if (!isPartitioned) { - return catalog.buildTable(icebergIdentifier, icebergSchema) + return icebergCatalog + .buildTable(icebergIdentifier, icebergSchema) .withPartitionSpec(PartitionSpec.unpartitioned()) .create(); } else { - return catalog.buildTable(icebergIdentifier, icebergSchema) + return icebergCatalog + .buildTable(icebergIdentifier, icebergSchema) .withPartitionSpec(icePartitionSpec) .create(); } From 98a4ec3ce7ac5d97bcfa5f669816eb6260756c19 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 11 Dec 2024 15:18:05 +0800 Subject: [PATCH 14/30] [improve] get iceberg metadata from IcebergMigrateMetadataFactory instead from iceberg catalog # Conflicts: # paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory --- .../migrate/IcebergMigrateHadoopMetadata.java | 96 +++++++++++++ .../IcebergMigrateHadoopMetadataFactory.java | 39 +++++ .../migrate/IcebergMigrateMetadata.java | 32 +++++ .../IcebergMigrateMetadataFactory.java | 31 ++++ .../{ => migrate}/IcebergMigrator.java | 81 ++++++----- .../org.apache.paimon.factories.Factory | 1 + .../{ => migrate}/IcebergMigrateTest.java | 21 +-- .../migrate/IcebergMigrateHiveMetadata.java | 136 ++++++++++++++++++ .../IcebergMigrateHiveMetadataFactory.java | 38 +++++ .../org.apache.paimon.factories.Factory | 1 + 10 files changed, 430 insertions(+), 46 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadataFactory.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadataFactory.java rename paimon-core/src/main/java/org/apache/paimon/iceberg/{ => migrate}/IcebergMigrator.java (86%) rename paimon-core/src/test/java/org/apache/paimon/iceberg/{ => migrate}/IcebergMigrateTest.java (97%) create mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java create mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java new file mode 100644 index 000000000000..12789cc6bb89 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg.migrate; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.IcebergPathFactory; +import org.apache.paimon.iceberg.metadata.IcebergMetadata; +import org.apache.paimon.options.Options; +import org.apache.paimon.utils.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** Get iceberg table latest snapshot metadata in hadoop. */ +public class IcebergMigrateHadoopMetadata implements IcebergMigrateMetadata { + private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrateHadoopMetadata.class); + + private static final String VERSION_HINT_FILENAME = "version-hint.text"; + private static final String ICEBERG_WAREHOUSE = "iceberg_warehouse"; + + private final FileIO fileIO; + private final Identifier icebergIdentifier; + private final Options icebergOptions; + + private Path icebergLatestMetaVersionPath; + private IcebergPathFactory icebergMetaPathFactory; + + public IcebergMigrateHadoopMetadata( + Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions) { + this.fileIO = fileIO; + this.icebergIdentifier = icebergIdentifier; + this.icebergOptions = icebergOptions; + } + + @Override + public IcebergMetadata icebergMetadata() { + Preconditions.checkArgument( + icebergOptions.get(ICEBERG_WAREHOUSE) != null, + "'iceberg_warehouse' is null. " + + "In hadoop-catalog, you should explicitly set this argument for finding iceberg metadata."); + this.icebergMetaPathFactory = + new IcebergPathFactory( + new Path( + icebergOptions.get(ICEBERG_WAREHOUSE), + new Path( + String.format( + "%s/%s/metadata", + icebergIdentifier.getDatabaseName(), + icebergIdentifier.getTableName())))); + long icebergLatestMetaVersion = getIcebergLatestMetaVersion(); + + this.icebergLatestMetaVersionPath = + icebergMetaPathFactory.toMetadataPath(icebergLatestMetaVersion); + LOG.info( + "iceberg latest snapshot metadata file location: {}", icebergLatestMetaVersionPath); + + return IcebergMetadata.fromPath(fileIO, icebergLatestMetaVersionPath); + } + + @Override + public String icebergLatestMetadataLocation() { + return icebergLatestMetaVersionPath.toString(); + } + + private long getIcebergLatestMetaVersion() { + Path versionHintPath = + new Path(icebergMetaPathFactory.metadataDirectory(), VERSION_HINT_FILENAME); + try { + return Integer.parseInt(fileIO.readFileUtf8(versionHintPath)); + } catch (IOException e) { + throw new RuntimeException( + "read iceberg version-hint.text failed. Iceberg metadata path: " + + icebergMetaPathFactory.metadataDirectory()); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadataFactory.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadataFactory.java new file mode 100644 index 000000000000..666630101445 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadataFactory.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg.migrate; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.iceberg.IcebergOptions; +import org.apache.paimon.options.Options; + +/** Factory to create {@link IcebergMigrateHadoopMetadata}. */ +public class IcebergMigrateHadoopMetadataFactory implements IcebergMigrateMetadataFactory { + + @Override + public String identifier() { + return IcebergOptions.StorageType.HADOOP_CATALOG.toString() + "_migrate"; + } + + @Override + public IcebergMigrateHadoopMetadata create( + Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions) { + return new IcebergMigrateHadoopMetadata(icebergIdentifier, fileIO, icebergOptions); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java new file mode 100644 index 000000000000..8c754e0a08e3 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg.migrate; + +import org.apache.paimon.iceberg.metadata.IcebergMetadata; + +/** + * Get iceberg metadata for migrate. Each kind of iceberg catalog should have its own + * implementation. + */ +public interface IcebergMigrateMetadata { + + IcebergMetadata icebergMetadata(); + + String icebergLatestMetadataLocation(); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadataFactory.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadataFactory.java new file mode 100644 index 000000000000..f727088f5d11 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadataFactory.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg.migrate; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.factories.Factory; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.options.Options; + +/** Factory to create {@link IcebergMigrateMetadata}. */ +public interface IcebergMigrateMetadataFactory extends Factory { + + IcebergMigrateMetadata create( + Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java similarity index 86% rename from paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java rename to paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index 3afaa88ae6bf..a5e24c4924af 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -16,13 +16,17 @@ * limitations under the License. */ -package org.apache.paimon.iceberg; +package org.apache.paimon.iceberg.migrate; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.factories.FactoryException; +import org.apache.paimon.factories.FactoryUtil; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.IcebergOptions; +import org.apache.paimon.iceberg.IcebergPathFactory; import org.apache.paimon.iceberg.manifest.IcebergDataFileMeta; import org.apache.paimon.iceberg.manifest.IcebergManifestEntry; import org.apache.paimon.iceberg.manifest.IcebergManifestFile; @@ -36,13 +40,14 @@ import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.migrate.FileMetaUtils; import org.apache.paimon.migrate.Migrator; +import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.table.FileStoreTable; 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.Preconditions; -import org.apache.iceberg.catalog.TableIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +63,7 @@ import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; -/** migrate hive table to paimon table. */ +/** migrate iceberg table to paimon table. */ public class IcebergMigrator implements Migrator { private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrator.class); @@ -69,28 +74,26 @@ public class IcebergMigrator implements Migrator { private final String paimonDatabaseName; private final String paimonTableName; - private final org.apache.iceberg.catalog.Catalog icebergCatalog; private final String icebergDatabaseName; private final String icebergTableName; + private final Options icebergOptions; // metadata path factory for iceberg metadata - private IcebergPathFactory icebergMetaPathFactory; - // iceberg table latest snapshot id - private long icebergLatestSnapshotId; - // iceberg latest snapshot metadata file name - private String icebergLatestMetaDataFileName; + private final IcebergPathFactory icebergMetaPathFactory; + // latest metadata file path + private final String icebergLatestMetadataLocation; // metadata for newest iceberg snapshot private final IcebergMetadata icebergMetadata; - private boolean ignoreDelete = false; + private final boolean ignoreDelete; public IcebergMigrator( Catalog paimonCatalog, String paimonDatabaseName, String paimonTableName, - org.apache.iceberg.catalog.Catalog icebergCatalog, String icebergDatabaseName, String icebergTableName, + Options icebergOptions, boolean ignoreDelete, Integer parallelism) { this.paimonCatalog = paimonCatalog; @@ -98,40 +101,42 @@ public IcebergMigrator( this.paimonDatabaseName = paimonDatabaseName; this.paimonTableName = paimonTableName; - this.icebergCatalog = icebergCatalog; this.icebergDatabaseName = icebergDatabaseName; this.icebergTableName = icebergTableName; + this.icebergOptions = icebergOptions; - initIcebergInfoForPaimon(); - this.icebergMetadata = - IcebergMetadata.fromPath( + Preconditions.checkArgument( + icebergOptions.containsKey(IcebergOptions.METADATA_ICEBERG_STORAGE.key()), + "'metadata.iceberg.storage' is required, please make sure it has been set."); + + IcebergMigrateMetadataFactory icebergMigrateMetadataFactory; + try { + icebergMigrateMetadataFactory = + FactoryUtil.discoverFactory( + IcebergMigrator.class.getClassLoader(), + IcebergMigrateMetadataFactory.class, + icebergOptions.get(IcebergOptions.METADATA_ICEBERG_STORAGE).toString() + + "_migrate"); + } catch (FactoryException e) { + throw new RuntimeException("create IcebergMigrateMetadataFactory failed.", e); + } + + IcebergMigrateMetadata icebergMigrateMetadata = + icebergMigrateMetadataFactory.create( + Identifier.create(icebergDatabaseName, icebergTableName), paimonFileIO, - icebergMetaPathFactory.toMetadataPath(icebergLatestMetaDataFileName)); + icebergOptions); + + this.icebergMetadata = icebergMigrateMetadata.icebergMetadata(); + this.icebergLatestMetadataLocation = icebergMigrateMetadata.icebergLatestMetadataLocation(); + this.icebergMetaPathFactory = + new IcebergPathFactory(new Path(icebergLatestMetadataLocation).getParent()); + this.ignoreDelete = ignoreDelete; this.executor = createCachedThreadPool(parallelism, "ICEBERG_MIGRATOR"); } - public void initIcebergInfoForPaimon() { - try { - LOG.info("Try to load iceberg table to get the metadata path of latest snapshot."); - org.apache.iceberg.Table icebergTable = - icebergCatalog.loadTable( - TableIdentifier.of(icebergDatabaseName, icebergTableName)); - org.apache.iceberg.TableMetadata currentMetadata = - ((org.apache.iceberg.BaseTable) icebergTable).operations().current(); - Path metadataLocation = new Path(currentMetadata.metadataFileLocation()); - LOG.info("iceberg latest snapshot metadata file location: {}", metadataLocation); - this.icebergMetaPathFactory = new IcebergPathFactory(metadataLocation.getParent()); - this.icebergLatestMetaDataFileName = metadataLocation.getName(); - this.icebergLatestSnapshotId = currentMetadata.currentSnapshot().snapshotId(); - LOG.info("get iceberg metadata path successfully."); - } catch (Exception e) { - throw new RuntimeException( - "failed to get iceberg metadata path by loading iceberg table."); - } - } - @Override public void executeMigrate() throws Exception { Schema paimonSchema = icebergSchemaToPaimonSchema(icebergMetadata); @@ -163,8 +168,8 @@ public void executeMigrate() throws Exception { if (icebergEntries.isEmpty()) { LOG.info( "No live manifest entry in iceberg table for snapshot {}, iceberg table meta path is {}.", - icebergLatestSnapshotId, - icebergMetaPathFactory.toMetadataPath(icebergLatestSnapshotId)); + icebergMetadata.currentSnapshotId(), + icebergLatestMetadataLocation); return; } diff --git a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 6416edd720f8..ff423bffd8df 100644 --- a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -39,3 +39,4 @@ org.apache.paimon.mergetree.compact.aggregate.factory.FieldThetaSketchAggFactory org.apache.paimon.rest.RESTCatalogFactory org.apache.paimon.rest.auth.BearTokenCredentialsProviderFactory org.apache.paimon.rest.auth.BearTokenFileCredentialsProviderFactory +org.apache.paimon.iceberg.migrate.IcebergMigrateHadoopMetadataFactory diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java similarity index 97% rename from paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java rename to paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java index 89a63c389f69..f3a98a4b6620 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.paimon.iceberg; +package org.apache.paimon.iceberg.migrate; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; @@ -25,6 +25,8 @@ import org.apache.paimon.data.DataFormatTestUtil; import org.apache.paimon.data.InternalRow; import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.IcebergOptions; +import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.Split; @@ -85,7 +87,6 @@ public class IcebergMigrateTest { Catalog paiCatalog; - org.apache.iceberg.catalog.Catalog icebergCatalog; String iceDatabase = "ice_db"; String iceTable = "ice_t"; @@ -106,10 +107,13 @@ public class IcebergMigrateTest { PartitionSpec icePartitionSpec = PartitionSpec.builderFor(iceSchema).identity("dt").identity("hh").build(); + Map icebergProperties = new HashMap<>(); + @BeforeEach public void beforeEach() throws Exception { paiCatalog = createPaimonCatalog(); - icebergCatalog = createIcebergCatalog(); + icebergProperties.put(IcebergOptions.METADATA_ICEBERG_STORAGE.key(), "hadoop-catalog"); + icebergProperties.put("iceberg_warehouse", iceTempDir.toString()); } @ParameterizedTest(name = "isPartitioned = {0}") @@ -144,9 +148,9 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { paiCatalog, paiDatabase, paiTable, - icebergCatalog, iceDatabase, iceTable, + new Options(icebergProperties), false, 1); icebergMigrator.executeMigrate(); @@ -200,9 +204,9 @@ public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { paiCatalog, paiDatabase, paiTable, - icebergCatalog, iceDatabase, iceTable, + new Options(icebergProperties), false, 1); icebergMigrator.executeMigrate(); @@ -259,9 +263,9 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet paiCatalog, paiDatabase, paiTable, - icebergCatalog, iceDatabase, iceTable, + new Options(icebergProperties), ignoreDelete, 1); if (!ignoreDelete) { @@ -342,9 +346,9 @@ private void testDeleteColumn(Table icebergTable, String format, boolean isParti paiCatalog, paiDatabase, paiTable, - icebergCatalog, iceDatabase, iceTable, + new Options(icebergProperties), false, 1); icebergMigrator.executeMigrate(); @@ -409,9 +413,9 @@ public void testAllDataTypes() throws Exception { paiCatalog, paiDatabase, paiTable, - icebergCatalog, iceDatabase, iceTable, + new Options(icebergProperties), false, 1); icebergMigrator.executeMigrate(); @@ -444,6 +448,7 @@ private Table createIcebergTable(boolean isPartitioned) { private Table createIcebergTable(boolean isPartitioned, Schema icebergSchema) { // HadoopCatalog catalog = new HadoopCatalog(new Configuration(), // iceTempDir.toString()); + org.apache.iceberg.catalog.Catalog icebergCatalog = createIcebergCatalog(); TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); if (!isPartitioned) { diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java new file mode 100644 index 000000000000..4b4839809342 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg.migrate; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.client.ClientPool; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.hive.HiveCatalog; +import org.apache.paimon.hive.pool.CachedClientPool; +import org.apache.paimon.iceberg.IcebergOptions; +import org.apache.paimon.iceberg.metadata.IcebergMetadata; +import org.apache.paimon.options.Options; +import org.apache.paimon.utils.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Get iceberg table latest snapshot metadata in hive. */ +public class IcebergMigrateHiveMetadata implements IcebergMigrateMetadata { + private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrateHiveMetadata.class); + + public static final String TABLE_TYPE_PROP = "table_type"; + public static final String ICEBERG_TABLE_TYPE_VALUE = "iceberg"; + private static final String ICEBERG_METADATA_LOCATION = "metadata_location"; + + private final FileIO fileIO; + private final Identifier icebergIdentifier; + + private final ClientPool clients; + + private String metadataLocation = null; + + public IcebergMigrateHiveMetadata( + Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions) { + this.fileIO = fileIO; + this.icebergIdentifier = icebergIdentifier; + + String uri = icebergOptions.get(IcebergOptions.URI); + String hiveConfDir = icebergOptions.get(IcebergOptions.HIVE_CONF_DIR); + String hadoopConfDir = icebergOptions.get(IcebergOptions.HADOOP_CONF_DIR); + Configuration hadoopConf = new Configuration(); + hadoopConf.setClassLoader(IcebergMigrateHiveMetadata.class.getClassLoader()); + HiveConf hiveConf = HiveCatalog.createHiveConf(hiveConfDir, hadoopConfDir, hadoopConf); + + icebergOptions.toMap().forEach(hiveConf::set); + if (uri != null) { + hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri); + } + + if (hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname) == null) { + LOG.error( + "Can't find hive metastore uri to connect: " + + "either set {} in iceberg options or set hive.metastore.uris " + + "in hive-site.xml or hadoop configurations. " + + "Will use empty metastore uris, which means we may use a embedded metastore. " + + "Please make sure hive metastore uri for iceberg table is correctly set as expected.", + IcebergOptions.URI.key()); + } + + this.clients = + new CachedClientPool( + hiveConf, + icebergOptions, + icebergOptions.getString(IcebergOptions.HIVE_CLIENT_CLASS)); + } + + @Override + public IcebergMetadata icebergMetadata() { + try { + boolean isExist = tableExists(icebergIdentifier); + if (!isExist) { + throw new RuntimeException( + String.format( + "iceberg table %s is not existed in hive metastore", + icebergIdentifier)); + } + Table icebergHiveTable = + clients.run( + client -> + client.getTable( + icebergIdentifier.getDatabaseName(), + icebergIdentifier.getTableName())); + // TODO:Is this check necessary? + // check whether it is an iceberg table + String tableType = icebergHiveTable.getParameters().get(TABLE_TYPE_PROP); + Preconditions.checkArgument( + tableType != null && tableType.equalsIgnoreCase(ICEBERG_TABLE_TYPE_VALUE), + "not an iceberg table: %s (table-type=%s)", + icebergIdentifier.toString(), + tableType); + + metadataLocation = icebergHiveTable.getParameters().get(ICEBERG_METADATA_LOCATION); + LOG.info("iceberg latest metadata location: {}", metadataLocation); + + return IcebergMetadata.fromPath(fileIO, new Path(metadataLocation)); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to read Iceberg metadata from path %s", metadataLocation), + e); + } + } + + @Override + public String icebergLatestMetadataLocation() { + return metadataLocation; + } + + private boolean tableExists(Identifier identifier) throws Exception { + return clients.run( + client -> + client.tableExists( + identifier.getDatabaseName(), identifier.getTableName())); + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java new file mode 100644 index 000000000000..184eb3134058 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg.migrate; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.iceberg.IcebergOptions; +import org.apache.paimon.options.Options; + +/** Factory to create {@link IcebergMigrateHiveMetadata}. */ +public class IcebergMigrateHiveMetadataFactory implements IcebergMigrateMetadataFactory { + @Override + public String identifier() { + return IcebergOptions.StorageType.HIVE_CATALOG.toString() + "_migrate"; + } + + @Override + public IcebergMigrateHiveMetadata create( + Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions) { + return new IcebergMigrateHiveMetadata(icebergIdentifier, fileIO, icebergOptions); + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 26f0944d916e..608f034659ca 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -16,3 +16,4 @@ org.apache.paimon.hive.HiveCatalogFactory org.apache.paimon.hive.HiveCatalogLockFactory org.apache.paimon.iceberg.IcebergHiveMetadataCommitterFactory +org.apache.paimon.iceberg.migrate.IcebergMigrateHiveMetadataFactory From 6f2fef80afd6b81a866c356472afdd83395ed484 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 12 Dec 2024 18:26:00 +0800 Subject: [PATCH 15/30] [improve] allow migrate to existing database --- .../java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index a5e24c4924af..9285571fb818 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -142,7 +142,7 @@ public void executeMigrate() throws Exception { Schema paimonSchema = icebergSchemaToPaimonSchema(icebergMetadata); Identifier paimonIdentifier = Identifier.create(paimonDatabaseName, paimonTableName); - paimonCatalog.createDatabase(paimonDatabaseName, false); + paimonCatalog.createDatabase(paimonDatabaseName, true); paimonCatalog.createTable(paimonIdentifier, paimonSchema, false); try { From a6fdc51f89097415976bd610e81d0a4c0751fc70 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Fri, 13 Dec 2024 11:15:58 +0800 Subject: [PATCH 16/30] [improve][core] get datatype directly if datatype in IcebergDataField is not null --- .../org/apache/paimon/iceberg/metadata/IcebergDataField.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index c5588b1d3769..e12ebf9852a9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -200,6 +200,9 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { } public DataType getDataType() { + if (dataType != null) { + return dataType; + } String simpleType = type.toString(); String delimiter = "("; if (simpleType.contains("[")) { From 0c0bb89a337a10a6dc6897a573dc1c82839a3efe Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Fri, 13 Dec 2024 11:44:06 +0800 Subject: [PATCH 17/30] [core] change scope for iceberg dependency in paimon-core to test --- paimon-core/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 697079d5af21..7c0de6da3098 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -203,12 +203,14 @@ under the License. org.apache.iceberg iceberg-core ${iceberg.version} + test org.apache.iceberg iceberg-data ${iceberg.version} + test From f73ecb5b244efe460c27f4eb287f621085df7318 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Fri, 13 Dec 2024 14:45:56 +0800 Subject: [PATCH 18/30] [improve][core] remove ignore-delete conf --- .../iceberg/migrate/IcebergMigrator.java | 50 ++++++------------- .../iceberg/migrate/IcebergMigrateTest.java | 40 +++------------ 2 files changed, 23 insertions(+), 67 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index 9285571fb818..fee1d6d71d43 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -85,8 +85,6 @@ public class IcebergMigrator implements Migrator { // metadata for newest iceberg snapshot private final IcebergMetadata icebergMetadata; - private final boolean ignoreDelete; - public IcebergMigrator( Catalog paimonCatalog, String paimonDatabaseName, @@ -94,7 +92,6 @@ public IcebergMigrator( String icebergDatabaseName, String icebergTableName, Options icebergOptions, - boolean ignoreDelete, Integer parallelism) { this.paimonCatalog = paimonCatalog; this.paimonFileIO = paimonCatalog.fileIO(); @@ -132,8 +129,6 @@ public IcebergMigrator( this.icebergMetaPathFactory = new IcebergPathFactory(new Path(icebergLatestMetadataLocation).getParent()); - this.ignoreDelete = ignoreDelete; - this.executor = createCachedThreadPool(parallelism, "ICEBERG_MIGRATOR"); } @@ -157,7 +152,7 @@ public void executeMigrate() throws Exception { manifestList.read(icebergMetadata.currentSnapshot().manifestList()); // check manifest file with 'DELETE' kind - icebergManifestFileMetas = checkAndFilterManifestFiles(icebergManifestFileMetas); + checkAndFilterManifestFiles(icebergManifestFileMetas); // get all live iceberg entries List icebergEntries = @@ -178,8 +173,8 @@ public void executeMigrate() throws Exception { .map(IcebergManifestEntry::file) .collect(Collectors.toList()); - // Again, check if the file is a Delete File - icebergDataFileMetas = checkAndFilterDataFiles(icebergDataFileMetas); + // Again, check if delete File exists + checkAndFilterDataFiles(icebergDataFileMetas); LOG.info( "Begin to create Migrate Task, the number of iceberg data files is {}", @@ -260,39 +255,24 @@ public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { dataFields, partitionKeys, Collections.emptyList(), Collections.emptyMap(), null); } - private List checkAndFilterManifestFiles( + private void checkAndFilterManifestFiles( List icebergManifestFileMetas) { - if (!ignoreDelete) { - for (IcebergManifestFileMeta meta : icebergManifestFileMetas) { - if (meta.content() == IcebergManifestFileMeta.Content.DELETES) { - throw new RuntimeException( - "IcebergMigrator don't support analyzing manifest file with 'DELETE' content. " - + "You can set 'ignore-delete' to ignore manifest file with 'DELETE' content."); - } + + for (IcebergManifestFileMeta meta : icebergManifestFileMetas) { + if (meta.content() == IcebergManifestFileMeta.Content.DELETES) { + throw new RuntimeException( + "IcebergMigrator don't support analyzing manifest file with 'DELETE' content."); } - return icebergManifestFileMetas; - } else { - return icebergManifestFileMetas.stream() - .filter(meta -> meta.content() != IcebergManifestFileMeta.Content.DELETES) - .collect(Collectors.toList()); } } - private List checkAndFilterDataFiles( - List icebergDataFileMetas) { - if (!ignoreDelete) { - for (IcebergDataFileMeta meta : icebergDataFileMetas) { - if (meta.content() != IcebergDataFileMeta.Content.DATA) { - throw new RuntimeException( - "IcebergMigrator don't support analyzing iceberg delete file. " - + "You can set 'ignore-delete' to ignore iceberg delete files."); - } + private void checkAndFilterDataFiles(List icebergDataFileMetas) { + + for (IcebergDataFileMeta meta : icebergDataFileMetas) { + if (meta.content() != IcebergDataFileMeta.Content.DATA) { + throw new RuntimeException( + "IcebergMigrator don't support analyzing iceberg delete file."); } - return icebergDataFileMetas; - } else { - return icebergDataFileMetas.stream() - .filter(meta -> meta.content() == IcebergDataFileMeta.Content.DATA) - .collect(Collectors.toList()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java index f3a98a4b6620..16b02ce0d79a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java @@ -58,7 +58,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; @@ -151,7 +150,6 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { iceDatabase, iceTable, new Options(icebergProperties), - false, 1); icebergMigrator.executeMigrate(); @@ -207,7 +205,6 @@ public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { iceDatabase, iceTable, new Options(icebergProperties), - false, 1); icebergMigrator.executeMigrate(); @@ -224,10 +221,9 @@ public void testMigrateAddAndDelete(boolean isPartitioned) throws Exception { .collect(Collectors.toList())); } - @ParameterizedTest - @CsvSource({"true, true", "true, false", "false, true", "false, false"}) - public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelete) - throws Exception { + @ParameterizedTest(name = "isPartitioned = {0}") + @ValueSource(booleans = {true, false}) + public void testMigrateWithDeleteFile(boolean isPartitioned) throws Exception { // only support create delete file with parquet format Table icebergTable = createIcebergTable(isPartitioned); String format = "parquet"; @@ -266,31 +262,13 @@ public void testMigrateWithDeleteFile(boolean isPartitioned, boolean ignoreDelet iceDatabase, iceTable, new Options(icebergProperties), - ignoreDelete, 1); - if (!ignoreDelete) { - assertThatThrownBy(icebergMigrator::executeMigrate) - .rootCause() - .isInstanceOf(RuntimeException.class) - .hasMessage( - "IcebergMigrator don't support analyzing manifest file with 'DELETE' content. " - + "You can set 'ignore-delete' to ignore manifest file with 'DELETE' content."); - return; - } else { - icebergMigrator.executeMigrate(); - } - FileStoreTable paimonTable = - (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); - List paiResults = getPaimonResult(paimonTable); - assertThat( - paiResults.stream() - .map(row -> String.format("Record(%s)", row)) - .collect(Collectors.toList())) - .hasSameElementsAs( - Stream.concat(records1.stream(), records2.stream()) - .map(GenericRecord::toString) - .collect(Collectors.toList())); + assertThatThrownBy(icebergMigrator::executeMigrate) + .rootCause() + .isInstanceOf(RuntimeException.class) + .hasMessage( + "IcebergMigrator don't support analyzing manifest file with 'DELETE' content."); } @ParameterizedTest(name = "isPartitioned = {0}") @@ -349,7 +327,6 @@ private void testDeleteColumn(Table icebergTable, String format, boolean isParti iceDatabase, iceTable, new Options(icebergProperties), - false, 1); icebergMigrator.executeMigrate(); @@ -416,7 +393,6 @@ public void testAllDataTypes() throws Exception { iceDatabase, iceTable, new Options(icebergProperties), - false, 1); icebergMigrator.executeMigrate(); From a11769fd8c074deb95a55b36b2ad39659cd85586 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 26 Dec 2024 10:52:45 +0800 Subject: [PATCH 19/30] [core] store the iceberg field datatype when computing over --- .../iceberg/metadata/IcebergDataField.java | 48 ++++++++++++------- 1 file changed, 32 insertions(+), 16 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index e12ebf9852a9..3b4715632167 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -73,7 +73,7 @@ public class IcebergDataField { @JsonProperty(FIELD_TYPE) private final Object type; - @JsonIgnore private final DataType dataType; + @JsonIgnore private DataType dataType; @JsonProperty(FIELD_DOC) private final String doc; @@ -214,31 +214,41 @@ public DataType getDataType() { : simpleType.substring(0, simpleType.indexOf(delimiter)); switch (typePrefix) { case "boolean": - return new BooleanType(!required); + dataType = new BooleanType(!required); + break; case "int": - return new IntType(!required); + dataType = new IntType(!required); + break; case "long": - return new BigIntType(!required); + dataType = new BigIntType(!required); + break; case "float": - return new FloatType(!required); + dataType = new FloatType(!required); + break; case "double": - return new DoubleType(!required); + dataType = new DoubleType(!required); + break; case "date": - return new DateType(!required); + dataType = new DateType(!required); + break; case "string": - return new VarCharType(!required, VarCharType.MAX_LENGTH); + dataType = new VarCharType(!required, VarCharType.MAX_LENGTH); + break; case "binary": - return new VarBinaryType(!required, VarBinaryType.MAX_LENGTH); + dataType = new VarBinaryType(!required, VarBinaryType.MAX_LENGTH); + break; case "fixed": int fixedLength = Integer.parseInt( simpleType.substring( simpleType.indexOf("[") + 1, simpleType.indexOf("]"))); - return new BinaryType(!required, fixedLength); + dataType = new BinaryType(!required, fixedLength); + break; case "uuid": // https://iceberg.apache.org/spec/?h=vector#primitive-types // uuid should use 16-byte fixed - return new BinaryType(!required, 16); + dataType = new BinaryType(!required, 16); + break; case "decimal": int precision = Integer.parseInt( @@ -248,18 +258,24 @@ public DataType getDataType() { Integer.parseInt( simpleType.substring( simpleType.indexOf(",") + 2, simpleType.indexOf(")"))); - return new DecimalType(!required, precision, scale); + dataType = new DecimalType(!required, precision, scale); + break; case "timestamp": - return new TimestampType(!required, 6); + dataType = new TimestampType(!required, 6); + break; case "timestamptz": - return new LocalZonedTimestampType(!required, 6); + dataType = new LocalZonedTimestampType(!required, 6); + break; case "timestamp_ns": // iceberg v3 format - return new TimestampType(!required, 9); + dataType = new TimestampType(!required, 9); + break; case "timestamptz_ns": // iceberg v3 format - return new LocalZonedTimestampType(!required, 9); + dataType = new LocalZonedTimestampType(!required, 9); + break; default: throw new UnsupportedOperationException("Unsupported data type: " + type); } + return dataType(); } public DataField toDatafield() { From 92e283a882141b6d8fa0827d3201faf4c9a95ced Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 26 Dec 2024 13:56:05 +0800 Subject: [PATCH 20/30] [core] remove construct to IcebergMigrator and use Preconditions instead throwing exception manually --- .../iceberg/migrate/IcebergMigrator.java | 51 +++++++++++++++---- .../apache/paimon/migrate/FileMetaUtils.java | 37 ++------------ 2 files changed, 44 insertions(+), 44 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index fee1d6d71d43..7804b70bd070 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -24,6 +24,7 @@ import org.apache.paimon.factories.FactoryException; import org.apache.paimon.factories.FactoryUtil; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; import org.apache.paimon.iceberg.IcebergOptions; import org.apache.paimon.iceberg.IcebergPathFactory; @@ -43,6 +44,7 @@ import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; 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; @@ -51,6 +53,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -259,21 +262,50 @@ private void checkAndFilterManifestFiles( List icebergManifestFileMetas) { for (IcebergManifestFileMeta meta : icebergManifestFileMetas) { - if (meta.content() == IcebergManifestFileMeta.Content.DELETES) { - throw new RuntimeException( - "IcebergMigrator don't support analyzing manifest file with 'DELETE' content."); - } + Preconditions.checkArgument( + meta.content() != IcebergManifestFileMeta.Content.DELETES, + "IcebergMigrator don't support analyzing manifest file with 'DELETE' content."); } } private void checkAndFilterDataFiles(List icebergDataFileMetas) { for (IcebergDataFileMeta meta : icebergDataFileMetas) { - if (meta.content() != IcebergDataFileMeta.Content.DATA) { - throw new RuntimeException( - "IcebergMigrator don't support analyzing iceberg delete file."); - } + Preconditions.checkArgument( + meta.content() == IcebergDataFileMeta.Content.DATA, + "IcebergMigrator don't support analyzing iceberg delete file."); + } + } + + private static List construct( + List icebergDataFileMetas, + FileIO fileIO, + Table paimonTable, + Path newDir, + Map rollback) { + return icebergDataFileMetas.stream() + .map( + icebergDataFileMeta -> + constructFileMeta( + icebergDataFileMeta, fileIO, paimonTable, newDir, rollback)) + .collect(Collectors.toList()); + } + + private static DataFileMeta constructFileMeta( + IcebergDataFileMeta icebergDataFileMeta, + FileIO fileIO, + Table table, + Path dir, + Map rollback) { + FileStatus status; + try { + status = fileIO.getFileStatus(new Path(icebergDataFileMeta.filePath())); + } catch (IOException e) { + throw new RuntimeException( + "error when get file status. file path is " + icebergDataFileMeta.filePath()); } + String format = icebergDataFileMeta.fileFormat(); + return FileMetaUtils.constructFileMeta(format, status, fileIO, table, dir, rollback); } private MigrateTask importUnPartitionedTable( @@ -341,8 +373,7 @@ public CommitMessage call() throws Exception { fileIO.mkdirs(newDir); } List fileMetas = - FileMetaUtils.construct( - icebergDataFileMetas, fileIO, paimonTable, newDir, rollback); + construct(icebergDataFileMetas, fileIO, paimonTable, newDir, rollback); return FileMetaUtils.commitFile(partitionRow, fileMetas); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java index dc87bd1fc4b3..c34afc76442e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java @@ -27,7 +27,6 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; -import org.apache.paimon.iceberg.manifest.IcebergDataFileMeta; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; @@ -61,20 +60,6 @@ public class FileMetaUtils { private static final Logger LOG = LoggerFactory.getLogger(FileMetaUtils.class); - public static List construct( - List icebergDataFileMetas, - FileIO fileIO, - Table paimonTable, - Path newDir, - Map rollback) { - return icebergDataFileMetas.stream() - .map( - icebergDataFileMeta -> - constructFileMeta( - icebergDataFileMeta, fileIO, paimonTable, newDir, rollback)) - .collect(Collectors.toList()); - } - public static List construct( FileIO fileIO, String format, @@ -107,25 +92,7 @@ public static CommitMessage commitFile(BinaryRow partition, List d Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); } - // -----------------------------private method--------------------------------------------- - private static DataFileMeta constructFileMeta( - IcebergDataFileMeta icebergDataFileMeta, - FileIO fileIO, - Table table, - Path dir, - Map rollback) { - FileStatus status; - try { - status = fileIO.getFileStatus(new Path(icebergDataFileMeta.filePath())); - } catch (IOException e) { - throw new RuntimeException( - "error when get file status. file path is " + icebergDataFileMeta.filePath()); - } - String format = icebergDataFileMeta.fileFormat(); - return constructFileMeta(format, status, fileIO, table, dir, rollback); - } - - private static DataFileMeta constructFileMeta( + public static DataFileMeta constructFileMeta( String format, FileStatus fileStatus, FileIO fileIO, @@ -162,6 +129,8 @@ private static DataFileMeta constructFileMeta( } } + // -----------------------------private method--------------------------------------------- + private static Path renameFile( FileIO fileIO, Path originPath, Path newDir, String format, Map rollback) throws IOException { From ab74c4ab58bd0934b0a57077658ecdb7ae547327 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 26 Dec 2024 15:31:10 +0800 Subject: [PATCH 21/30] [core] add deleteOriginTable in IcebergMigrateMetadata and implement it in subclass --- .../migrate/IcebergMigrateHadoopMetadata.java | 15 ++++++++++++++ .../migrate/IcebergMigrateMetadata.java | 2 ++ .../iceberg/migrate/IcebergMigrator.java | 14 +++++++++++-- .../migrate/IcebergMigrateHiveMetadata.java | 20 +++++++++++++++++++ 4 files changed, 49 insertions(+), 2 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java index 12789cc6bb89..4d0069e4d228 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java @@ -82,6 +82,21 @@ public String icebergLatestMetadataLocation() { return icebergLatestMetaVersionPath.toString(); } + @Override + public void deleteOriginTable() { + Path tablePath = icebergMetaPathFactory.metadataDirectory().getParent(); + LOG.info("Iceberg table path to be deleted:{}", tablePath); + try { + if (fileIO.isDir(tablePath)) { + fileIO.deleteDirectoryQuietly(tablePath); + } + } catch (IOException e) { + LOG.warn( + "exception occurred when deleting origin table, exception message:{}", + e.getMessage()); + } + } + private long getIcebergLatestMetaVersion() { Path versionHintPath = new Path(icebergMetaPathFactory.metadataDirectory(), VERSION_HINT_FILENAME); diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java index 8c754e0a08e3..58648c537f90 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateMetadata.java @@ -29,4 +29,6 @@ public interface IcebergMigrateMetadata { IcebergMetadata icebergMetadata(); String icebergLatestMetadataLocation(); + + void deleteOriginTable(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index 7804b70bd070..44d28e6f8685 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -81,6 +81,7 @@ public class IcebergMigrator implements Migrator { private final String icebergTableName; private final Options icebergOptions; + private final IcebergMigrateMetadata icebergMigrateMetadata; // metadata path factory for iceberg metadata private final IcebergPathFactory icebergMetaPathFactory; // latest metadata file path @@ -88,6 +89,8 @@ public class IcebergMigrator implements Migrator { // metadata for newest iceberg snapshot private final IcebergMetadata icebergMetadata; + private Boolean deleteOriginTable = true; + public IcebergMigrator( Catalog paimonCatalog, String paimonDatabaseName, @@ -121,7 +124,7 @@ public IcebergMigrator( throw new RuntimeException("create IcebergMigrateMetadataFactory failed.", e); } - IcebergMigrateMetadata icebergMigrateMetadata = + icebergMigrateMetadata = icebergMigrateMetadataFactory.create( Identifier.create(icebergDatabaseName, icebergTableName), paimonFileIO, @@ -226,10 +229,17 @@ public void executeMigrate() throws Exception { paimonCatalog.dropTable(paimonIdentifier, true); throw new RuntimeException("Migrating failed", e); } + + // if all success, drop the origin table according the delete field + if (deleteOriginTable) { + icebergMigrateMetadata.deleteOriginTable(); + } } @Override - public void deleteOriginTable(boolean delete) throws Exception {} + public void deleteOriginTable(boolean delete) throws Exception { + this.deleteOriginTable = delete; + } @Override public void renameTable(boolean ignoreIfNotExists) throws Exception {} diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java index 4b4839809342..10669f09a0f5 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java @@ -127,6 +127,26 @@ public String icebergLatestMetadataLocation() { return metadataLocation; } + @Override + public void deleteOriginTable() { + LOG.info("Iceberg table in hive to be deleted:{}", icebergIdentifier.toString()); + try { + clients.run( + client -> { + client.dropTable( + icebergIdentifier.getDatabaseName(), + icebergIdentifier.getTableName(), + true, + true); + return null; + }); + } catch (Exception e) { + LOG.warn( + "exception occurred when deleting origin table, exception message:{}", + e.getMessage()); + } + } + private boolean tableExists(Identifier identifier) throws Exception { return clients.run( client -> From 0fd196f08a54b3fe7b52f3853bd7d5bcee1aeccd Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 26 Dec 2024 15:39:28 +0800 Subject: [PATCH 22/30] [core] implement 'renameTable' again for a independent commit --- .../org/apache/paimon/iceberg/migrate/IcebergMigrator.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index 44d28e6f8685..69d87cbb00a4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -242,7 +242,10 @@ public void deleteOriginTable(boolean delete) throws Exception { } @Override - public void renameTable(boolean ignoreIfNotExists) throws Exception {} + public void renameTable(boolean ignoreIfNotExists) throws Exception { + LOG.info("Last step: rename."); + LOG.info("Iceberg migrator do not rename table now."); + } public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { // get iceberg current schema From 67ee959b41e554c36506358659bab466cf56c946 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 26 Dec 2024 17:04:32 +0800 Subject: [PATCH 23/30] [core][test] add a test which using random data --- .../iceberg/migrate/IcebergMigrateTest.java | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java index 16b02ce0d79a..d230208cc260 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java @@ -72,6 +72,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -271,6 +272,55 @@ public void testMigrateWithDeleteFile(boolean isPartitioned) throws Exception { "IcebergMigrator don't support analyzing manifest file with 'DELETE' content."); } + @ParameterizedTest(name = "isPartitioned = {0}") + @ValueSource(booleans = {true, false}) + public void testMigrateWithRandomIcebergData(boolean isPartitioned) throws Exception { + Table icebergTable = createIcebergTable(isPartitioned); + String format = "parquet"; + + int numRounds = 100; + int numRecords = 50; + ThreadLocalRandom random = ThreadLocalRandom.current(); + List expectRecords = new ArrayList<>(); + for (int i = 0; i < numRounds; i++) { + List records = new ArrayList<>(); + String dt = Integer.toString(random.nextInt(20240101, 20240104)); + String hh = Integer.toString(random.nextInt(3)); + for (int j = 0; j < numRecords; j++) { + records.add(toIcebergRecord(random.nextInt(100), random.nextInt(100), dt, hh)); + } + expectRecords.addAll(records); + if (isPartitioned) { + writeRecordsToIceberg(icebergTable, format, records, dt, hh); + } else { + writeRecordsToIceberg(icebergTable, format, records); + } + } + + IcebergMigrator icebergMigrator = + new IcebergMigrator( + paiCatalog, + paiDatabase, + paiTable, + iceDatabase, + iceTable, + new Options(icebergProperties), + 1); + icebergMigrator.executeMigrate(); + + FileStoreTable paimonTable = + (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); + List paiResults = getPaimonResult(paimonTable); + assertThat( + paiResults.stream() + .map(row -> String.format("Record(%s)", row)) + .collect(Collectors.toList())) + .hasSameElementsAs( + expectRecords.stream() + .map(GenericRecord::toString) + .collect(Collectors.toList())); + } + @ParameterizedTest(name = "isPartitioned = {0}") @ValueSource(booleans = {true, false}) public void testMigrateWithSchemaEvolution(boolean isPartitioned) throws Exception { From 3911175a25882b377632f9f92fbea35c54047fa4 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 26 Dec 2024 17:34:49 +0800 Subject: [PATCH 24/30] [core][hive] remove contents about iceberg hive-catalog which will be contained in other pr --- .../migrate/IcebergMigrateHiveMetadata.java | 156 ------------------ .../IcebergMigrateHiveMetadataFactory.java | 38 ----- .../org.apache.paimon.factories.Factory | 1 - 3 files changed, 195 deletions(-) delete mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java delete mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java deleted file mode 100644 index 10669f09a0f5..000000000000 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadata.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.iceberg.migrate; - -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.client.ClientPool; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; -import org.apache.paimon.hive.HiveCatalog; -import org.apache.paimon.hive.pool.CachedClientPool; -import org.apache.paimon.iceberg.IcebergOptions; -import org.apache.paimon.iceberg.metadata.IcebergMetadata; -import org.apache.paimon.options.Options; -import org.apache.paimon.utils.Preconditions; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Get iceberg table latest snapshot metadata in hive. */ -public class IcebergMigrateHiveMetadata implements IcebergMigrateMetadata { - private static final Logger LOG = LoggerFactory.getLogger(IcebergMigrateHiveMetadata.class); - - public static final String TABLE_TYPE_PROP = "table_type"; - public static final String ICEBERG_TABLE_TYPE_VALUE = "iceberg"; - private static final String ICEBERG_METADATA_LOCATION = "metadata_location"; - - private final FileIO fileIO; - private final Identifier icebergIdentifier; - - private final ClientPool clients; - - private String metadataLocation = null; - - public IcebergMigrateHiveMetadata( - Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions) { - this.fileIO = fileIO; - this.icebergIdentifier = icebergIdentifier; - - String uri = icebergOptions.get(IcebergOptions.URI); - String hiveConfDir = icebergOptions.get(IcebergOptions.HIVE_CONF_DIR); - String hadoopConfDir = icebergOptions.get(IcebergOptions.HADOOP_CONF_DIR); - Configuration hadoopConf = new Configuration(); - hadoopConf.setClassLoader(IcebergMigrateHiveMetadata.class.getClassLoader()); - HiveConf hiveConf = HiveCatalog.createHiveConf(hiveConfDir, hadoopConfDir, hadoopConf); - - icebergOptions.toMap().forEach(hiveConf::set); - if (uri != null) { - hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri); - } - - if (hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname) == null) { - LOG.error( - "Can't find hive metastore uri to connect: " - + "either set {} in iceberg options or set hive.metastore.uris " - + "in hive-site.xml or hadoop configurations. " - + "Will use empty metastore uris, which means we may use a embedded metastore. " - + "Please make sure hive metastore uri for iceberg table is correctly set as expected.", - IcebergOptions.URI.key()); - } - - this.clients = - new CachedClientPool( - hiveConf, - icebergOptions, - icebergOptions.getString(IcebergOptions.HIVE_CLIENT_CLASS)); - } - - @Override - public IcebergMetadata icebergMetadata() { - try { - boolean isExist = tableExists(icebergIdentifier); - if (!isExist) { - throw new RuntimeException( - String.format( - "iceberg table %s is not existed in hive metastore", - icebergIdentifier)); - } - Table icebergHiveTable = - clients.run( - client -> - client.getTable( - icebergIdentifier.getDatabaseName(), - icebergIdentifier.getTableName())); - // TODO:Is this check necessary? - // check whether it is an iceberg table - String tableType = icebergHiveTable.getParameters().get(TABLE_TYPE_PROP); - Preconditions.checkArgument( - tableType != null && tableType.equalsIgnoreCase(ICEBERG_TABLE_TYPE_VALUE), - "not an iceberg table: %s (table-type=%s)", - icebergIdentifier.toString(), - tableType); - - metadataLocation = icebergHiveTable.getParameters().get(ICEBERG_METADATA_LOCATION); - LOG.info("iceberg latest metadata location: {}", metadataLocation); - - return IcebergMetadata.fromPath(fileIO, new Path(metadataLocation)); - } catch (Exception e) { - throw new RuntimeException( - String.format("Failed to read Iceberg metadata from path %s", metadataLocation), - e); - } - } - - @Override - public String icebergLatestMetadataLocation() { - return metadataLocation; - } - - @Override - public void deleteOriginTable() { - LOG.info("Iceberg table in hive to be deleted:{}", icebergIdentifier.toString()); - try { - clients.run( - client -> { - client.dropTable( - icebergIdentifier.getDatabaseName(), - icebergIdentifier.getTableName(), - true, - true); - return null; - }); - } catch (Exception e) { - LOG.warn( - "exception occurred when deleting origin table, exception message:{}", - e.getMessage()); - } - } - - private boolean tableExists(Identifier identifier) throws Exception { - return clients.run( - client -> - client.tableExists( - identifier.getDatabaseName(), identifier.getTableName())); - } -} diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java deleted file mode 100644 index 184eb3134058..000000000000 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHiveMetadataFactory.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.iceberg.migrate; - -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.iceberg.IcebergOptions; -import org.apache.paimon.options.Options; - -/** Factory to create {@link IcebergMigrateHiveMetadata}. */ -public class IcebergMigrateHiveMetadataFactory implements IcebergMigrateMetadataFactory { - @Override - public String identifier() { - return IcebergOptions.StorageType.HIVE_CATALOG.toString() + "_migrate"; - } - - @Override - public IcebergMigrateHiveMetadata create( - Identifier icebergIdentifier, FileIO fileIO, Options icebergOptions) { - return new IcebergMigrateHiveMetadata(icebergIdentifier, fileIO, icebergOptions); - } -} diff --git a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 608f034659ca..26f0944d916e 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -16,4 +16,3 @@ org.apache.paimon.hive.HiveCatalogFactory org.apache.paimon.hive.HiveCatalogLockFactory org.apache.paimon.iceberg.IcebergHiveMetadataCommitterFactory -org.apache.paimon.iceberg.migrate.IcebergMigrateHiveMetadataFactory From ffe2a027d7ada2f6b160999f601f0257b28473a4 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 6 Jan 2025 15:35:01 +0800 Subject: [PATCH 25/30] [core] throwing exception and logging better. optimize code about getting iceberg datatype. --- .../IcebergManifestEntrySerializer.java | 2 +- .../iceberg/metadata/IcebergDataField.java | 51 ++++++++----------- .../migrate/IcebergMigrateHadoopMetadata.java | 7 ++- .../iceberg/migrate/IcebergMigrator.java | 3 +- 4 files changed, 26 insertions(+), 37 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java index cf55f9749228..b9d2c271b509 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestEntrySerializer.java @@ -57,7 +57,7 @@ public IcebergManifestEntry fromRow(InternalRow row) { public IcebergManifestEntry fromRow(InternalRow row, IcebergManifestFileMeta meta) { IcebergManifestEntry.Status status = IcebergManifestEntry.Status.fromId(row.getInt(0)); - long snapshotId = !row.isNullAt(1) ? row.getLong(1) : meta.addedSnapshotId(); + long snapshotId = row.isNullAt(1) ? meta.addedSnapshotId() : row.getLong(1); long sequenceNumber = getOrInherit(row, meta, 2, status); long fileSequenceNumber = getOrInherit(row, meta, 3, status); diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 3b4715632167..2cfd6bfa80b7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -203,6 +203,11 @@ public DataType getDataType() { if (dataType != null) { return dataType; } + dataType = getDataTypeFromType(); + return dataType(); + } + + public DataType getDataTypeFromType() { String simpleType = type.toString(); String delimiter = "("; if (simpleType.contains("[")) { @@ -214,41 +219,31 @@ public DataType getDataType() { : simpleType.substring(0, simpleType.indexOf(delimiter)); switch (typePrefix) { case "boolean": - dataType = new BooleanType(!required); - break; + return new BooleanType(!required); case "int": - dataType = new IntType(!required); - break; + return new IntType(!required); case "long": - dataType = new BigIntType(!required); - break; + return new BigIntType(!required); case "float": - dataType = new FloatType(!required); - break; + return new FloatType(!required); case "double": - dataType = new DoubleType(!required); - break; + return new DoubleType(!required); case "date": - dataType = new DateType(!required); - break; + return new DateType(!required); case "string": - dataType = new VarCharType(!required, VarCharType.MAX_LENGTH); - break; + return new VarCharType(!required, VarCharType.MAX_LENGTH); case "binary": - dataType = new VarBinaryType(!required, VarBinaryType.MAX_LENGTH); - break; + return new VarBinaryType(!required, VarBinaryType.MAX_LENGTH); case "fixed": int fixedLength = Integer.parseInt( simpleType.substring( simpleType.indexOf("[") + 1, simpleType.indexOf("]"))); - dataType = new BinaryType(!required, fixedLength); - break; + return new BinaryType(!required, fixedLength); case "uuid": // https://iceberg.apache.org/spec/?h=vector#primitive-types // uuid should use 16-byte fixed - dataType = new BinaryType(!required, 16); - break; + return new BinaryType(!required, 16); case "decimal": int precision = Integer.parseInt( @@ -258,24 +253,18 @@ public DataType getDataType() { Integer.parseInt( simpleType.substring( simpleType.indexOf(",") + 2, simpleType.indexOf(")"))); - dataType = new DecimalType(!required, precision, scale); - break; + return new DecimalType(!required, precision, scale); case "timestamp": - dataType = new TimestampType(!required, 6); - break; + return new TimestampType(!required, 6); case "timestamptz": - dataType = new LocalZonedTimestampType(!required, 6); - break; + return new LocalZonedTimestampType(!required, 6); case "timestamp_ns": // iceberg v3 format - dataType = new TimestampType(!required, 9); - break; + return new TimestampType(!required, 9); case "timestamptz_ns": // iceberg v3 format - dataType = new LocalZonedTimestampType(!required, 9); - break; + return new LocalZonedTimestampType(!required, 9); default: throw new UnsupportedOperationException("Unsupported data type: " + type); } - return dataType(); } public DataField toDatafield() { diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java index 4d0069e4d228..a6c5fb027ba2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrateHadoopMetadata.java @@ -91,9 +91,7 @@ public void deleteOriginTable() { fileIO.deleteDirectoryQuietly(tablePath); } } catch (IOException e) { - LOG.warn( - "exception occurred when deleting origin table, exception message:{}", - e.getMessage()); + LOG.warn("exception occurred when deleting origin table.", e); } } @@ -105,7 +103,8 @@ private long getIcebergLatestMetaVersion() { } catch (IOException e) { throw new RuntimeException( "read iceberg version-hint.text failed. Iceberg metadata path: " - + icebergMetaPathFactory.metadataDirectory()); + + icebergMetaPathFactory.metadataDirectory(), + e); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index 69d87cbb00a4..ded5d2353b6a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -315,7 +315,8 @@ private static DataFileMeta constructFileMeta( status = fileIO.getFileStatus(new Path(icebergDataFileMeta.filePath())); } catch (IOException e) { throw new RuntimeException( - "error when get file status. file path is " + icebergDataFileMeta.filePath()); + "error when get file status. file path is " + icebergDataFileMeta.filePath(), + e); } String format = icebergDataFileMeta.fileFormat(); return FileMetaUtils.constructFileMeta(format, status, fileIO, table, dir, rollback); From 41ded21c6bfea893fd1b2b7468d3ac83aa08897a Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Tue, 7 Jan 2025 15:19:16 +0800 Subject: [PATCH 26/30] [core] implement rename table --- .../apache/paimon/iceberg/migrate/IcebergMigrator.java | 8 +++++--- .../paimon/iceberg/migrate/IcebergMigrateTest.java | 10 +++++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java index ded5d2353b6a..44162dea7fc3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/migrate/IcebergMigrator.java @@ -223,7 +223,7 @@ public void executeMigrate() throws Exception { } try (BatchTableCommit commit = paimonTable.newBatchWriteBuilder().newCommit()) { commit.commit(new ArrayList<>(commitMessages)); - LOG.info("paimon commit success! Iceberg data files has been migrated to paimon."); + LOG.info("paimon commit success! Iceberg data files have been migrated to paimon."); } } catch (Exception e) { paimonCatalog.dropTable(paimonIdentifier, true); @@ -243,8 +243,10 @@ public void deleteOriginTable(boolean delete) throws Exception { @Override public void renameTable(boolean ignoreIfNotExists) throws Exception { - LOG.info("Last step: rename."); - LOG.info("Iceberg migrator do not rename table now."); + Identifier targetTableId = Identifier.create(paimonDatabaseName, paimonTableName); + Identifier sourceTableId = Identifier.create(icebergDatabaseName, icebergTableName); + LOG.info("Last step: rename {} to {}.", targetTableId, sourceTableId); + paimonCatalog.renameTable(targetTableId, sourceTableId, ignoreIfNotExists); } public Schema icebergSchemaToPaimonSchema(IcebergMetadata icebergMetadata) { diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java index d230208cc260..8ce494e66da5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java @@ -153,9 +153,10 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { new Options(icebergProperties), 1); icebergMigrator.executeMigrate(); + icebergMigrator.renameTable(false); FileStoreTable paimonTable = - (FileStoreTable) paiCatalog.getTable(Identifier.create(paiDatabase, paiTable)); + (FileStoreTable) paiCatalog.getTable(Identifier.create(iceDatabase, iceTable)); List paiResults = getPaimonResult(paimonTable); assertThat( paiResults.stream() @@ -165,6 +166,9 @@ public void testMigrateOnlyAdd(boolean isPartitioned) throws Exception { Stream.concat(records1.stream(), records2.stream()) .map(GenericRecord::toString) .collect(Collectors.toList())); + + // verify iceberg table has been deleted + assertThat(paimonTable.fileIO().exists(new Path(icebergTable.location()))).isFalse(); } @ParameterizedTest(name = "isPartitioned = {0}") @@ -278,8 +282,8 @@ public void testMigrateWithRandomIcebergData(boolean isPartitioned) throws Excep Table icebergTable = createIcebergTable(isPartitioned); String format = "parquet"; - int numRounds = 100; - int numRecords = 50; + int numRounds = 50; + int numRecords = 20; ThreadLocalRandom random = ThreadLocalRandom.current(); List expectRecords = new ArrayList<>(); for (int i = 0; i < numRounds; i++) { From 1f74505e28164dfd800a7a2c1a851e22fedf46ec Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Tue, 7 Jan 2025 15:58:17 +0800 Subject: [PATCH 27/30] [core][fix] avoid json serializer using wrong getters --- .../org/apache/paimon/iceberg/metadata/IcebergDataField.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 2cfd6bfa80b7..b17283b6fa53 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -199,6 +199,7 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { } } + @JsonIgnore public DataType getDataType() { if (dataType != null) { return dataType; @@ -207,6 +208,7 @@ public DataType getDataType() { return dataType(); } + @JsonIgnore public DataType getDataTypeFromType() { String simpleType = type.toString(); String delimiter = "("; From 90cfcaee1a4f75004bda27b4ad6ba38e916785d0 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 9 Jan 2025 14:13:03 +0800 Subject: [PATCH 28/30] [core] modify dataType() in IcebergDataField --- .../paimon/iceberg/metadata/IcebergDataField.java | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index b17283b6fa53..33221e76b174 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -135,6 +135,10 @@ public String doc() { @JsonIgnore public DataType dataType() { + if (dataType != null) { + return dataType; + } + dataType = getDataTypeFromType(); return Preconditions.checkNotNull(dataType); } @@ -199,15 +203,6 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { } } - @JsonIgnore - public DataType getDataType() { - if (dataType != null) { - return dataType; - } - dataType = getDataTypeFromType(); - return dataType(); - } - @JsonIgnore public DataType getDataTypeFromType() { String simpleType = type.toString(); @@ -270,7 +265,7 @@ public DataType getDataTypeFromType() { } public DataField toDatafield() { - return new DataField(id, name, getDataType(), doc); + return new DataField(id, name, dataType(), doc); } @Override From 39461657efc36592daf9b87fddd313e12c3de273 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 9 Jan 2025 14:24:01 +0800 Subject: [PATCH 29/30] [core][test] remove content about orc and avro format --- paimon-core/pom.xml | 7 ------ .../iceberg/migrate/IcebergMigrateTest.java | 24 ++----------------- 2 files changed, 2 insertions(+), 29 deletions(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 7c0de6da3098..1531d9f511b3 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -234,13 +234,6 @@ under the License. test - - org.apache.iceberg - iceberg-orc - ${iceberg.version} - test - - diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java index 8ce494e66da5..aadaca0c3854 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/migrate/IcebergMigrateTest.java @@ -40,17 +40,14 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.orc.GenericOrcWriter; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; @@ -476,8 +473,7 @@ private Table createIcebergTable(boolean isPartitioned) { } private Table createIcebergTable(boolean isPartitioned, Schema icebergSchema) { - // HadoopCatalog catalog = new HadoopCatalog(new Configuration(), - // iceTempDir.toString()); + org.apache.iceberg.catalog.Catalog icebergCatalog = createIcebergCatalog(); TableIdentifier icebergIdentifier = TableIdentifier.of(iceDatabase, iceTable); @@ -519,7 +515,7 @@ private DataWriter createIcebergDataWriter( icebergTable, partitionValues[0], partitionValues[1]); - // TODO: currently only support "parquet" format + // currently only support "parquet" format switch (format) { case "parquet": return Parquet.writeData(file) @@ -529,22 +525,6 @@ private DataWriter createIcebergDataWriter( .withSpec(partitionSpec) .withPartition(partitionKey) .build(); - case "avro": - return Avro.writeData(file) - .schema(schema) - .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) - .overwrite() - .withSpec(partitionSpec) - .withPartition(partitionKey) - .build(); - case "orc": - return ORC.writeData(file) - .schema(schema) - .createWriterFunc(GenericOrcWriter::buildWriter) - .overwrite() - .withSpec(partitionSpec) - .withPartition(partitionKey) - .build(); default: throw new IllegalArgumentException("Unsupported format: " + format); } From 927a602013a62f82172357aeecd810b7e4fa96b5 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 9 Jan 2025 16:18:24 +0800 Subject: [PATCH 30/30] [core] alter access permisssion of getDataTypeFromType() --- .../org/apache/paimon/iceberg/metadata/IcebergDataField.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 33221e76b174..a310e64f6474 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -203,8 +203,7 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { } } - @JsonIgnore - public DataType getDataTypeFromType() { + private DataType getDataTypeFromType() { String simpleType = type.toString(); String delimiter = "("; if (simpleType.contains("[")) {