From d10b584604544f27dfc5f065491c9498a207613b Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 21 Nov 2024 15:11:50 +0800 Subject: [PATCH 01/17] 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 029e55dde77610d0b18f53214b6ba02f43b04214 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 28 Nov 2024 17:19:41 +0800 Subject: [PATCH 02/17] [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 2b4813dfc18e0d19db6fdd73d76f4f727ac004b7 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 28 Nov 2024 17:20:43 +0800 Subject: [PATCH 03/17] [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 cca1d079df711cca6577aa58255e66a89c48355e Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 2 Dec 2024 18:20:01 +0800 Subject: [PATCH 04/17] 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 d553122cb8c664451d375438fd06031d46c1b0e4 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 2 Dec 2024 18:20:36 +0800 Subject: [PATCH 05/17] 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 92280c433907ceab04febd47210d64032f1f9381 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 10:00:42 +0800 Subject: [PATCH 06/17] 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 b466e4997c25104bc5faee3eace18c233a2974bb Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 10:01:26 +0800 Subject: [PATCH 07/17] 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 dd13791d434358b6d01d3a23e6e5a4f7f9ea417c Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 11:41:04 +0800 Subject: [PATCH 08/17] [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 6116f541465a12542495dd8364427cc7b58a891d Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 13:49:14 +0800 Subject: [PATCH 09/17] 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 cfb9109b4ee447d8b16cb9d92b6dffb26acffbcf Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Wed, 4 Dec 2024 17:33:50 +0800 Subject: [PATCH 10/17] 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 898eed1c6b96bf452dcacad1383f8f373f8df9bf Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 5 Dec 2024 12:56:09 +0800 Subject: [PATCH 11/17] 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 147606fa847ab3cc093c07a6ee7ac102f973e178 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 5 Dec 2024 13:12:05 +0800 Subject: [PATCH 12/17] 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 7020df61dff1425d2e11a0fb88065dd5b2ad0a25 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Thu, 5 Dec 2024 16:20:11 +0800 Subject: [PATCH 13/17] add flink procedure and action for migrating iceberg table --- .../flink/procedure/MigrateFileProcedure.java | 9 ++- .../procedure/MigrateTableProcedure.java | 63 ++++++++++--------- .../flink/action/MigrateFileAction.java | 6 +- .../flink/action/MigrateTableAction.java | 8 ++- .../action/MigrateTableActionFactory.java | 5 +- .../flink/procedure/MigrateFileProcedure.java | 9 ++- .../procedure/MigrateTableProcedure.java | 30 ++++----- .../flink/utils/TableMigrationUtils.java | 23 +++++-- .../MigrateTableProcedureITCase.java | 3 +- 9 files changed, 94 insertions(+), 62 deletions(-) diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java index 1e581c38cb97..b64ff3056c2c 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java @@ -86,7 +86,10 @@ public void migrateHandle( boolean deleteOrigin, Integer parallelism) throws Exception { - Identifier sourceTableId = Identifier.fromString(sourceTablePath); + if (!connector.equals("hive")) { + throw new IllegalArgumentException("MigrateFile only support hive connector now."); + } + Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); try { @@ -100,11 +103,11 @@ public void migrateHandle( TableMigrationUtils.getImporter( connector, catalog, - sourceTableId.getDatabaseName(), - sourceTableId.getObjectName(), + sourceTablePath, targetTableId.getDatabaseName(), targetTableId.getObjectName(), parallelism, + Collections.emptyMap(), Collections.emptyMap()); importer.deleteOriginTable(deleteOrigin); importer.executeMigrate(); diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java index 196528d31c78..12007669dda6 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java @@ -20,17 +20,14 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.utils.TableMigrationUtils; +import org.apache.paimon.migrate.Migrator; import org.apache.paimon.utils.ParameterUtils; import org.apache.flink.table.procedure.ProcedureContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Migrate procedure to migrate hive table to paimon table. */ public class MigrateTableProcedure extends ProcedureBase { - private static final Logger LOG = LoggerFactory.getLogger(MigrateTableProcedure.class); - private static final String PAIMON_SUFFIX = "_paimon_"; @Override @@ -50,25 +47,14 @@ public String[] call( String sourceTablePath, String properties) throws Exception { - String targetPaimonTablePath = sourceTablePath + PAIMON_SUFFIX; - - Identifier sourceTableId = Identifier.fromString(sourceTablePath); - Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); - - TableMigrationUtils.getImporter( - connector, - catalog, - sourceTableId.getDatabaseName(), - sourceTableId.getObjectName(), - targetTableId.getDatabaseName(), - targetTableId.getObjectName(), - Runtime.getRuntime().availableProcessors(), - ParameterUtils.parseCommaSeparatedKeyValues(properties)) - .executeMigrate(); - LOG.info("Last step: rename " + targetTableId + " to " + sourceTableId); - catalog.renameTable(targetTableId, sourceTableId, false); - return new String[] {"Success"}; + return call( + procedureContext, + connector, + sourceTablePath, + properties, + Runtime.getRuntime().availableProcessors(), + ""); } public String[] call( @@ -78,24 +64,41 @@ public String[] call( String properties, Integer parallelism) throws Exception { + return call( + procedureContext, + connector, + sourceTablePath, + properties, + Runtime.getRuntime().availableProcessors(), + ""); + } + + public String[] call( + ProcedureContext procedureContext, + String connector, + String sourceTablePath, + String properties, + Integer parallelism, + String icebergProperties) + throws Exception { String targetPaimonTablePath = sourceTablePath + PAIMON_SUFFIX; - Identifier sourceTableId = Identifier.fromString(sourceTablePath); Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); - TableMigrationUtils.getImporter( + Migrator migrator = + TableMigrationUtils.getImporter( connector, catalog, - sourceTableId.getDatabaseName(), - sourceTableId.getObjectName(), + sourceTablePath, targetTableId.getDatabaseName(), targetTableId.getObjectName(), parallelism, - ParameterUtils.parseCommaSeparatedKeyValues(properties)) - .executeMigrate(); + ParameterUtils.parseCommaSeparatedKeyValues(properties), + ParameterUtils.parseCommaSeparatedKeyValues(icebergProperties)); + + migrator.executeMigrate(); - LOG.info("Last step: rename " + targetTableId + " to " + sourceTableId); - catalog.renameTable(targetTableId, sourceTableId, false); + migrator.renameTable(false); return new String[] {"Success"}; } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileAction.java index 798d1d347732..bd8a177c4597 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileAction.java @@ -54,9 +54,9 @@ public MigrateFileAction( @Override public void run() throws Exception { - MigrateFileProcedure migrateTableProcedure = new MigrateFileProcedure(); - migrateTableProcedure.withCatalog(catalog); - migrateTableProcedure.call( + MigrateFileProcedure migrateFileProcedure = new MigrateFileProcedure(); + migrateFileProcedure.withCatalog(catalog); + migrateFileProcedure.call( new DefaultProcedureContext(env), connector, sourceTable, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableAction.java index 8a4efdfc710d..b140a4dd6a64 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableAction.java @@ -31,6 +31,7 @@ public class MigrateTableAction extends ActionBase { private final String hiveTableFullName; private final String tableProperties; private final Integer parallelism; + private final String icebergOptions; public MigrateTableAction( String connector, @@ -38,12 +39,14 @@ public MigrateTableAction( String hiveTableFullName, Map catalogConfig, String tableProperties, - Integer parallelism) { + Integer parallelism, + String icebergOptions) { super(warehouse, catalogConfig); this.connector = connector; this.hiveTableFullName = hiveTableFullName; this.tableProperties = tableProperties; this.parallelism = parallelism; + this.icebergOptions = icebergOptions; } @Override @@ -55,6 +58,7 @@ public void run() throws Exception { connector, hiveTableFullName, tableProperties, - parallelism); + parallelism, + icebergOptions); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableActionFactory.java index a1a93bc91163..92a8aec00730 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateTableActionFactory.java @@ -29,6 +29,7 @@ public class MigrateTableActionFactory implements ActionFactory { private static final String SOURCE_TYPE = "source_type"; private static final String OPTIONS = "options"; private static final String PARALLELISM = "parallelism"; + private static final String ICEBERG_OPTIONS = "iceberg_options"; @Override public String identifier() { @@ -43,6 +44,7 @@ public Optional create(MultipleParameterToolAdapter params) { Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); String tableConf = params.get(OPTIONS); Integer parallelism = Integer.parseInt(params.get(PARALLELISM)); + String icebergOptions = params.get(ICEBERG_OPTIONS); MigrateTableAction migrateTableAction = new MigrateTableAction( @@ -51,7 +53,8 @@ public Optional create(MultipleParameterToolAdapter params) { sourceHiveTable, catalogConfig, tableConf, - parallelism); + parallelism, + icebergOptions); return Optional.of(migrateTableAction); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java index f2f10d087406..76cc999cce5d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java @@ -75,7 +75,10 @@ public void migrateHandle( boolean deleteOrigin, Integer parallelism) throws Exception { - Identifier sourceTableId = Identifier.fromString(sourceTablePath); + if (!connector.equals("hive")) { + throw new IllegalArgumentException("MigrateFile only support hive connector now."); + } + Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); try { @@ -89,11 +92,11 @@ public void migrateHandle( TableMigrationUtils.getImporter( connector, catalog, - sourceTableId.getDatabaseName(), - sourceTableId.getObjectName(), + sourceTablePath, targetTableId.getDatabaseName(), targetTableId.getObjectName(), parallelism, + Collections.emptyMap(), Collections.emptyMap()); importer.deleteOriginTable(deleteOrigin); importer.executeMigrate(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java index fff05a1a8555..264e97100806 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java @@ -20,20 +20,17 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.utils.TableMigrationUtils; +import org.apache.paimon.migrate.Migrator; import org.apache.paimon.utils.ParameterUtils; import org.apache.flink.table.annotation.ArgumentHint; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.annotation.ProcedureHint; import org.apache.flink.table.procedure.ProcedureContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Migrate procedure to migrate hive table to paimon table. */ public class MigrateTableProcedure extends ProcedureBase { - private static final Logger LOG = LoggerFactory.getLogger(MigrateTableProcedure.class); - private static final String PAIMON_SUFFIX = "_paimon_"; @Override @@ -49,37 +46,42 @@ public String identifier() { @ArgumentHint( name = "parallelism", type = @DataTypeHint("Integer"), - isOptional = true) + isOptional = true), + @ArgumentHint( + name = "iceberg_options", + type = @DataTypeHint("STRING"), + isOptional = true), }) public String[] call( ProcedureContext procedureContext, String connector, String sourceTablePath, String properties, - Integer parallelism) + Integer parallelism, + String icebergOptions) throws Exception { properties = notnull(properties); + icebergOptions = notnull(icebergOptions); String targetPaimonTablePath = sourceTablePath + PAIMON_SUFFIX; - Identifier sourceTableId = Identifier.fromString(sourceTablePath); Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); Integer p = parallelism == null ? Runtime.getRuntime().availableProcessors() : parallelism; - TableMigrationUtils.getImporter( + Migrator migrator = + TableMigrationUtils.getImporter( connector, catalog, - sourceTableId.getDatabaseName(), - sourceTableId.getObjectName(), + sourceTablePath, targetTableId.getDatabaseName(), targetTableId.getObjectName(), p, - ParameterUtils.parseCommaSeparatedKeyValues(properties)) - .executeMigrate(); + ParameterUtils.parseCommaSeparatedKeyValues(properties), + ParameterUtils.parseCommaSeparatedKeyValues(icebergOptions)); + migrator.executeMigrate(); - LOG.info("Last step: rename " + targetTableId + " to " + sourceTableId); - catalog.renameTable(targetTableId, sourceTableId, false); + migrator.renameTable(false); return new String[] {"Success"}; } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java index b59c3592a97d..59fd20117fe0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java @@ -20,9 +20,13 @@ import org.apache.paimon.catalog.CachingCatalog; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.Path; import org.apache.paimon.hive.HiveCatalog; import org.apache.paimon.hive.migrate.HiveMigrator; +import org.apache.paimon.iceberg.IcebergMigrator; import org.apache.paimon.migrate.Migrator; +import org.apache.paimon.options.Options; import java.util.List; import java.util.Map; @@ -33,14 +37,15 @@ public class TableMigrationUtils { public static Migrator getImporter( String connector, Catalog catalog, - String sourceDatabase, - String sourceTableName, + String sourceIdentifier, String targetDatabase, String targetTableName, Integer parallelism, - Map options) { + Map options, + Map icebergOptions) { switch (connector) { case "hive": + Identifier identifier = Identifier.fromString(sourceIdentifier); if (catalog instanceof CachingCatalog) { catalog = ((CachingCatalog) catalog).wrapped(); } @@ -49,12 +54,20 @@ public static Migrator getImporter( } return new HiveMigrator( (HiveCatalog) catalog, - sourceDatabase, - sourceTableName, + identifier.getDatabaseName(), + identifier.getTableName(), targetDatabase, targetTableName, parallelism, options); + case "iceberg": + return new IcebergMigrator( + catalog, + new Path(sourceIdentifier), + targetDatabase, + targetTableName, + new Options(icebergOptions).getBoolean("ignore-delete-file", false), + parallelism); default: throw new UnsupportedOperationException("Don't support connector " + connector); } diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java index ca3b6c82e7d3..adc8aff918b1 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java @@ -192,7 +192,8 @@ public void testMigrateAction(String format) throws Exception { "default.hivetable", catalogConf, "", - 6); + 6, + ""); migrateTableAction.run(); tEnv.executeSql( From 5b49dc56f528607f5a1e0fbfe83a87ecb5034260 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Fri, 6 Dec 2024 12:04:59 +0800 Subject: [PATCH 14/17] adjust arguments for procedure --- .../paimon/iceberg/IcebergMigrator.java | 5 +- .../flink/procedure/MigrateFileProcedure.java | 1 - .../procedure/MigrateTableProcedure.java | 47 +++++++++--------- .../flink/procedure/MigrateFileProcedure.java | 1 - .../procedure/MigrateTableProcedure.java | 49 +++++++++++++------ .../flink/utils/TableMigrationUtils.java | 39 +++++++++++---- 6 files changed, 92 insertions(+), 50 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..54b74c3edcc4 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 @@ -197,7 +197,10 @@ public void executeMigrate() throws Exception { 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 int getIcebergNewestSnapshotId() { Path versionHintPath = diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java index b64ff3056c2c..785e41383a48 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java @@ -107,7 +107,6 @@ public void migrateHandle( targetTableId.getDatabaseName(), targetTableId.getObjectName(), parallelism, - Collections.emptyMap(), Collections.emptyMap()); importer.deleteOriginTable(deleteOrigin); importer.executeMigrate(); diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java index 12007669dda6..53be1e3c0581 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java @@ -22,6 +22,7 @@ import org.apache.paimon.flink.utils.TableMigrationUtils; import org.apache.paimon.migrate.Migrator; import org.apache.paimon.utils.ParameterUtils; +import org.apache.paimon.utils.Preconditions; import org.apache.flink.table.procedure.ProcedureContext; @@ -47,14 +48,12 @@ public String[] call( String sourceTablePath, String properties) throws Exception { - return call( procedureContext, connector, sourceTablePath, properties, - Runtime.getRuntime().availableProcessors(), - ""); + Runtime.getRuntime().availableProcessors()); } public String[] call( @@ -64,23 +63,7 @@ public String[] call( String properties, Integer parallelism) throws Exception { - return call( - procedureContext, - connector, - sourceTablePath, - properties, - Runtime.getRuntime().availableProcessors(), - ""); - } - - public String[] call( - ProcedureContext procedureContext, - String connector, - String sourceTablePath, - String properties, - Integer parallelism, - String icebergProperties) - throws Exception { + Preconditions.checkArgument(connector.equals("hive")); String targetPaimonTablePath = sourceTablePath + PAIMON_SUFFIX; Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); @@ -93,12 +76,32 @@ public String[] call( targetTableId.getDatabaseName(), targetTableId.getObjectName(), parallelism, - ParameterUtils.parseCommaSeparatedKeyValues(properties), - ParameterUtils.parseCommaSeparatedKeyValues(icebergProperties)); + ParameterUtils.parseCommaSeparatedKeyValues(properties)); migrator.executeMigrate(); migrator.renameTable(false); return new String[] {"Success"}; } + + public String[] call( + ProcedureContext procedureContext, + String connector, + String sourceTablePath, + String properties, + Integer parallelism, + String icebrgConf) + throws Exception { + Preconditions.checkArgument(connector.equals("iceberg")); + Migrator migrator = + TableMigrationUtils.getIcebergImporter( + catalog, + parallelism, + ParameterUtils.parseCommaSeparatedKeyValues(properties), + ParameterUtils.parseCommaSeparatedKeyValues(icebrgConf)); + migrator.executeMigrate(); + + migrator.renameTable(false); + return new String[] {"Success"}; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java index 76cc999cce5d..1712f4ca8926 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java @@ -96,7 +96,6 @@ public void migrateHandle( targetTableId.getDatabaseName(), targetTableId.getObjectName(), parallelism, - Collections.emptyMap(), Collections.emptyMap()); importer.deleteOriginTable(deleteOrigin); importer.executeMigrate(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java index 264e97100806..835b568758c3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateTableProcedure.java @@ -22,6 +22,7 @@ import org.apache.paimon.flink.utils.TableMigrationUtils; import org.apache.paimon.migrate.Migrator; import org.apache.paimon.utils.ParameterUtils; +import org.apache.paimon.utils.Preconditions; import org.apache.flink.table.annotation.ArgumentHint; import org.apache.flink.table.annotation.DataTypeHint; @@ -41,7 +42,10 @@ public String identifier() { @ProcedureHint( argument = { @ArgumentHint(name = "connector", type = @DataTypeHint("STRING")), - @ArgumentHint(name = "source_table", type = @DataTypeHint("STRING")), + @ArgumentHint( + name = "source_table", + type = @DataTypeHint("STRING"), + isOptional = true), @ArgumentHint(name = "options", type = @DataTypeHint("STRING"), isOptional = true), @ArgumentHint( name = "parallelism", @@ -63,22 +67,37 @@ public String[] call( properties = notnull(properties); icebergOptions = notnull(icebergOptions); - String targetPaimonTablePath = sourceTablePath + PAIMON_SUFFIX; - - Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); - Integer p = parallelism == null ? Runtime.getRuntime().availableProcessors() : parallelism; - Migrator migrator = - TableMigrationUtils.getImporter( - connector, - catalog, - sourceTablePath, - targetTableId.getDatabaseName(), - targetTableId.getObjectName(), - p, - ParameterUtils.parseCommaSeparatedKeyValues(properties), - ParameterUtils.parseCommaSeparatedKeyValues(icebergOptions)); + Migrator migrator; + switch (connector) { + case "hive": + Preconditions.checkArgument( + sourceTablePath != null, "please set 'source_table' for hive migrator"); + String targetPaimonTablePath = sourceTablePath + PAIMON_SUFFIX; + Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); + migrator = + TableMigrationUtils.getImporter( + connector, + catalog, + sourceTablePath, + targetTableId.getDatabaseName(), + targetTableId.getObjectName(), + p, + ParameterUtils.parseCommaSeparatedKeyValues(properties)); + break; + case "iceberg": + migrator = + TableMigrationUtils.getIcebergImporter( + catalog, + p, + ParameterUtils.parseCommaSeparatedKeyValues(properties), + ParameterUtils.parseCommaSeparatedKeyValues(icebergOptions)); + break; + default: + throw new UnsupportedOperationException("Don't support connector " + connector); + } + migrator.executeMigrate(); migrator.renameTable(false); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java index 59fd20117fe0..86c763bab7cc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java @@ -27,6 +27,7 @@ import org.apache.paimon.iceberg.IcebergMigrator; import org.apache.paimon.migrate.Migrator; import org.apache.paimon.options.Options; +import org.apache.paimon.utils.Preconditions; import java.util.List; import java.util.Map; @@ -41,8 +42,7 @@ public static Migrator getImporter( String targetDatabase, String targetTableName, Integer parallelism, - Map options, - Map icebergOptions) { + Map options) { switch (connector) { case "hive": Identifier identifier = Identifier.fromString(sourceIdentifier); @@ -60,19 +60,26 @@ public static Migrator getImporter( targetTableName, parallelism, options); - case "iceberg": - return new IcebergMigrator( - catalog, - new Path(sourceIdentifier), - targetDatabase, - targetTableName, - new Options(icebergOptions).getBoolean("ignore-delete-file", false), - parallelism); default: throw new UnsupportedOperationException("Don't support connector " + connector); } } + public static Migrator getIcebergImporter( + Catalog catalog, + Integer parallelism, + Map options, + Map icebergConf) { + checkIcebergRequiredConf(icebergConf); + return new IcebergMigrator( + catalog, + new Path(icebergConf.get("iceberg-meta-path")), + icebergConf.get("target-database"), + icebergConf.get("target-table"), + new Options(icebergConf).getBoolean("ignore-delete-file", false), + parallelism); + } + public static List getImporters( String connector, Catalog catalog, @@ -93,4 +100,16 @@ public static List getImporters( throw new UnsupportedOperationException("Don't support connector " + connector); } } + + private static void checkIcebergRequiredConf(Map icebergConf) { + Preconditions.checkArgument( + icebergConf.containsKey("iceberg-meta-path"), + "please set required iceberg argument 'iceberg-meta-path'."); + Preconditions.checkArgument( + icebergConf.containsKey("target-database"), + "please set required iceberg argument 'target-database'."); + Preconditions.checkArgument( + icebergConf.containsKey("target-table"), + "please set required iceberg argument 'target-table'."); + } } From 95b24b3709ecdb2e920ea54ff0f6e2d817a2ece8 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Fri, 6 Dec 2024 17:10:54 +0800 Subject: [PATCH 15/17] add IT cases for migrate iceberg table --- .../MigrateTableProcedureITCase.java | 88 +++++++++++++++++-- paimon-hive/pom.xml | 15 ++++ 2 files changed, 97 insertions(+), 6 deletions(-) diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java index adc8aff918b1..95826ac04782 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java @@ -32,11 +32,13 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -50,6 +52,9 @@ public class MigrateTableProcedureITCase extends ActionITCaseBase { private static final int PORT = 9084; + @TempDir private java.nio.file.Path iceTempDir; + @TempDir private java.nio.file.Path paiTempDir; + @BeforeEach public void beforeEach() { TEST_HIVE_METASTORE.start(PORT); @@ -70,12 +75,17 @@ private static Stream testArguments() { Arguments.of("parquet", false)); } + private static Stream testIcebergArguments() { + return Stream.of(Arguments.of(true, false), Arguments.of(false, false)); + } + @ParameterizedTest @MethodSource("testArguments") - public void testMigrateProcedure(String format, boolean isNamedArgument) throws Exception { - testUpgradeNonPartitionTable(format, isNamedArgument); + public void testMigrateProcedureForHive(String format, boolean isNamedArgument) + throws Exception { + testUpgradeHiveNonPartitionTable(format, isNamedArgument); resetMetastore(); - testUpgradePartitionTable(format, isNamedArgument); + testUpgradeHivePartitionTable(format, isNamedArgument); } private void resetMetastore() throws Exception { @@ -84,7 +94,8 @@ private void resetMetastore() throws Exception { TEST_HIVE_METASTORE.start(PORT); } - public void testUpgradePartitionTable(String format, boolean isNamedArgument) throws Exception { + public void testUpgradeHivePartitionTable(String format, boolean isNamedArgument) + throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')"); tEnv.useCatalog("HIVE"); @@ -125,7 +136,7 @@ public void testUpgradePartitionTable(String format, boolean isNamedArgument) th Assertions.assertThatList(r1).containsExactlyInAnyOrderElementsOf(r2); } - public void testUpgradeNonPartitionTable(String format, boolean isNamedArgument) + public void testUpgradeHiveNonPartitionTable(String format, boolean isNamedArgument) throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')"); @@ -167,7 +178,7 @@ public void testUpgradeNonPartitionTable(String format, boolean isNamedArgument) @ParameterizedTest @ValueSource(strings = {"orc", "parquet", "avro"}) - public void testMigrateAction(String format) throws Exception { + public void testMigrateActionForHive(String format) throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')"); tEnv.useCatalog("HIVE"); @@ -208,6 +219,71 @@ public void testMigrateAction(String format) throws Exception { Assertions.assertThatList(r1).containsExactlyInAnyOrderElementsOf(r2); } + @ParameterizedTest + @MethodSource("testIcebergArguments") + public void testMigrateIcebergUnPartitionedTable(boolean isPartitioned, boolean isHive) + throws Exception { + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); + + // create iceberg catalog, database, table, and insert some data to iceberg table + tEnv.executeSql(icebergCatalogDdl(isHive)); + tEnv.executeSql("USE CATALOG my_iceberg"); + tEnv.executeSql("CREATE DATABASE iceberg_db;"); + if (isPartitioned) { + tEnv.executeSql( + "CREATE TABLE iceberg_db.iceberg_table (id string, id2 int, id3 int) PARTITIONED BY (id3)" + + " WITH ('format-version'='2')"); + } else { + tEnv.executeSql( + "CREATE TABLE iceberg_db.iceberg_table (id string, id2 int, id3 int) WITH ('format-version'='2')"); + } + tEnv.executeSql("INSERT INTO iceberg_db.iceberg_table VALUES ('a',1,1),('b',2,2),('c',3,3)") + .await(); + + tEnv.executeSql(paimonCatalogDdl(isHive)); + tEnv.executeSql("USE CATALOG my_paimon"); + tEnv.executeSql( + String.format( + "CALL sys.migrate_table(connector => 'iceberg', " + + "iceberg_options => 'iceberg-meta-path=%s,target-database=%s,target-table=%s')", + iceTempDir + "/iceberg_db/iceberg_table/metadata", + "paimon_db", + "paimon_table")) + .await(); + + Assertions.assertThatList( + Arrays.asList(Row.of("a", 1, 1), Row.of("b", 2, 2), Row.of("c", 3, 3))) + .containsExactlyInAnyOrderElementsOf( + ImmutableList.copyOf( + tEnv.executeSql("SELECT * FROM paimon_db.paimon_table").collect())); + } + + private String icebergCatalogDdl(boolean isHive) { + return isHive + ? String.format( + "CREATE CATALOG my_iceberg WITH " + + "( 'type' = 'iceberg', 'catalog-type' = 'hive', 'uri' = 'thrift://localhost:%s', " + + "'warehouse' = '%s', 'cache-enabled' = 'false' )", + PORT, iceTempDir) + : String.format( + "CREATE CATALOG my_iceberg WITH " + + "( 'type' = 'iceberg', 'catalog-type' = 'hadoop'," + + "'warehouse' = '%s', 'cache-enabled' = 'false' )", + iceTempDir); + } + + private String paimonCatalogDdl(boolean isHive) { + return isHive + ? String.format( + "CREATE CATALOG my_paimon WITH " + + "( 'type' = 'paimon', 'metastore' = 'hive', 'uri' = 'thrift://localhost:%s', " + + "'warehouse' = '%s', 'cache-enabled' = 'false' )", + PORT, iceTempDir) + : String.format( + "CREATE CATALOG my_paimon WITH ('type' = 'paimon', 'warehouse' = '%s')", + paiTempDir); + } + protected static String data(int i) { Random random = new Random(); StringBuilder stringBuilder = new StringBuilder(); diff --git a/paimon-hive/pom.xml b/paimon-hive/pom.xml index 7d1d0f2c499c..20314b350832 100644 --- a/paimon-hive/pom.xml +++ b/paimon-hive/pom.xml @@ -50,6 +50,7 @@ under the License. 0.9.8 1.12.319 1.19 + 1.19.0 @@ -129,6 +130,20 @@ under the License. ${project.version} test + + + org.apache.iceberg + iceberg-flink-${iceberg.flink.version} + ${iceberg.version} + test + + + + org.apache.flink + flink-metrics-dropwizard + ${iceberg.flink.dropwizard.version} + test + From 2367f6105d0c638bf99ff15a879925928f179274 Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 9 Dec 2024 12:47:16 +0800 Subject: [PATCH 16/17] remove test to 2.3 and 3.1 module --- paimon-hive/paimon-hive-connector-2.3/pom.xml | 6 + .../MigrateIcebergTableProcedure23ITCase.java | 24 ++++ paimon-hive/paimon-hive-connector-3.1/pom.xml | 6 + .../MigrateIcebergTableProcedure31ITCase.java | 24 ++++ .../MigrateIcebergTableProcedureITCase.java | 121 ++++++++++++++++++ .../MigrateTableProcedureITCase.java | 88 +------------ paimon-hive/pom.xml | 14 -- 7 files changed, 187 insertions(+), 96 deletions(-) create mode 100644 paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure23ITCase.java create mode 100644 paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure31ITCase.java create mode 100644 paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java diff --git a/paimon-hive/paimon-hive-connector-2.3/pom.xml b/paimon-hive/paimon-hive-connector-2.3/pom.xml index a0f509b53375..daeb87475f3f 100644 --- a/paimon-hive/paimon-hive-connector-2.3/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.3/pom.xml @@ -569,6 +569,12 @@ under the License. ${iceberg.version} test + + org.apache.flink + flink-metrics-dropwizard + ${iceberg.flink.dropwizard.version} + test + diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure23ITCase.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure23ITCase.java new file mode 100644 index 000000000000..638a803eb75c --- /dev/null +++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure23ITCase.java @@ -0,0 +1,24 @@ +/* + * 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.procedure; + +import org.apache.paimon.hive.procedure.MigrateIcebergTableProcedureITCase; + +/** IT cases for migrating iceberg table to paimon table in hive 2.3. */ +public class MigrateIcebergTableProcedure23ITCase extends MigrateIcebergTableProcedureITCase {} diff --git a/paimon-hive/paimon-hive-connector-3.1/pom.xml b/paimon-hive/paimon-hive-connector-3.1/pom.xml index 5383af90c3e5..9ea1718205b2 100644 --- a/paimon-hive/paimon-hive-connector-3.1/pom.xml +++ b/paimon-hive/paimon-hive-connector-3.1/pom.xml @@ -599,6 +599,12 @@ under the License. ${iceberg.version} test + + org.apache.flink + flink-metrics-dropwizard + ${iceberg.flink.dropwizard.version} + test + diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure31ITCase.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure31ITCase.java new file mode 100644 index 000000000000..fa8ac6dc6503 --- /dev/null +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/procedure/MigrateIcebergTableProcedure31ITCase.java @@ -0,0 +1,24 @@ +/* + * 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.procedure; + +import org.apache.paimon.hive.procedure.MigrateIcebergTableProcedureITCase; + +/** IT cases for migrating iceberg table to paimon table in hive 3.1. */ +public class MigrateIcebergTableProcedure31ITCase extends MigrateIcebergTableProcedureITCase {} diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java new file mode 100644 index 000000000000..11d461b0c2a6 --- /dev/null +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java @@ -0,0 +1,121 @@ +/* + * 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.hive.procedure; + +import org.apache.paimon.hive.runner.PaimonEmbeddedHiveRunner; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; + +import com.klarna.hiverunner.HiveShell; +import com.klarna.hiverunner.annotations.HiveSQL; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.types.Row; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.runner.RunWith; + +import java.util.Arrays; +import java.util.stream.Stream; + +/** IT cases for migrating iceberg table to paimon table. */ +@RunWith(PaimonEmbeddedHiveRunner.class) +public abstract class MigrateIcebergTableProcedureITCase { + + @TempDir private java.nio.file.Path iceTempDir; + @TempDir private java.nio.file.Path paiTempDir; + + @HiveSQL(files = {}) + protected static HiveShell hiveShell; + + private static Stream testIcebergArguments() { + return Stream.of(Arguments.of(true, false), Arguments.of(false, false)); + } + + @ParameterizedTest + @MethodSource("testIcebergArguments") + public void testMigrateIcebergUnPartitionedTable(boolean isPartitioned, boolean isHive) + throws Exception { + TableEnvironment tEnv = + TableEnvironmentImpl.create( + EnvironmentSettings.newInstance().inBatchMode().build()); + + // create iceberg catalog, database, table, and insert some data to iceberg table + tEnv.executeSql(icebergCatalogDdl(isHive)); + tEnv.executeSql("USE CATALOG my_iceberg"); + tEnv.executeSql("CREATE DATABASE iceberg_db;"); + if (isPartitioned) { + tEnv.executeSql( + "CREATE TABLE iceberg_db.iceberg_table (id string, id2 int, id3 int) PARTITIONED BY (id3)" + + " WITH ('format-version'='2')"); + } else { + tEnv.executeSql( + "CREATE TABLE iceberg_db.iceberg_table (id string, id2 int, id3 int) WITH ('format-version'='2')"); + } + tEnv.executeSql("INSERT INTO iceberg_db.iceberg_table VALUES ('a',1,1),('b',2,2),('c',3,3)") + .await(); + + tEnv.executeSql(paimonCatalogDdl(isHive)); + tEnv.executeSql("USE CATALOG my_paimon"); + tEnv.executeSql( + String.format( + "CALL sys.migrate_table(connector => 'iceberg', " + + "iceberg_options => 'iceberg-meta-path=%s,target-database=%s,target-table=%s')", + iceTempDir + "/iceberg_db/iceberg_table/metadata", + "paimon_db", + "paimon_table")) + .await(); + + Assertions.assertThatList( + Arrays.asList(Row.of("a", 1, 1), Row.of("b", 2, 2), Row.of("c", 3, 3))) + .containsExactlyInAnyOrderElementsOf( + ImmutableList.copyOf( + tEnv.executeSql("SELECT * FROM paimon_db.paimon_table").collect())); + } + + private String icebergCatalogDdl(boolean isHive) { + return isHive + ? String.format( + "CREATE CATALOG my_iceberg WITH " + + "( 'type' = 'iceberg', 'catalog-type' = 'hive', 'uri' = '', " + + "'warehouse' = '%s', 'cache-enabled' = 'false' )", + iceTempDir) + : String.format( + "CREATE CATALOG my_iceberg WITH " + + "( 'type' = 'iceberg', 'catalog-type' = 'hadoop'," + + "'warehouse' = '%s', 'cache-enabled' = 'false' )", + iceTempDir); + } + + private String paimonCatalogDdl(boolean isHive) { + return isHive + ? String.format( + "CREATE CATALOG my_paimon WITH " + + "( 'type' = 'paimon', 'metastore' = 'hive', 'uri' = '', " + + "'warehouse' = '%s', 'cache-enabled' = 'false' )", + iceTempDir) + : String.format( + "CREATE CATALOG my_paimon WITH ('type' = 'paimon', 'warehouse' = '%s')", + paiTempDir); + } +} diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java index 95826ac04782..adc8aff918b1 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateTableProcedureITCase.java @@ -32,13 +32,11 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -52,9 +50,6 @@ public class MigrateTableProcedureITCase extends ActionITCaseBase { private static final int PORT = 9084; - @TempDir private java.nio.file.Path iceTempDir; - @TempDir private java.nio.file.Path paiTempDir; - @BeforeEach public void beforeEach() { TEST_HIVE_METASTORE.start(PORT); @@ -75,17 +70,12 @@ private static Stream testArguments() { Arguments.of("parquet", false)); } - private static Stream testIcebergArguments() { - return Stream.of(Arguments.of(true, false), Arguments.of(false, false)); - } - @ParameterizedTest @MethodSource("testArguments") - public void testMigrateProcedureForHive(String format, boolean isNamedArgument) - throws Exception { - testUpgradeHiveNonPartitionTable(format, isNamedArgument); + public void testMigrateProcedure(String format, boolean isNamedArgument) throws Exception { + testUpgradeNonPartitionTable(format, isNamedArgument); resetMetastore(); - testUpgradeHivePartitionTable(format, isNamedArgument); + testUpgradePartitionTable(format, isNamedArgument); } private void resetMetastore() throws Exception { @@ -94,8 +84,7 @@ private void resetMetastore() throws Exception { TEST_HIVE_METASTORE.start(PORT); } - public void testUpgradeHivePartitionTable(String format, boolean isNamedArgument) - throws Exception { + public void testUpgradePartitionTable(String format, boolean isNamedArgument) throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')"); tEnv.useCatalog("HIVE"); @@ -136,7 +125,7 @@ public void testUpgradeHivePartitionTable(String format, boolean isNamedArgument Assertions.assertThatList(r1).containsExactlyInAnyOrderElementsOf(r2); } - public void testUpgradeHiveNonPartitionTable(String format, boolean isNamedArgument) + public void testUpgradeNonPartitionTable(String format, boolean isNamedArgument) throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')"); @@ -178,7 +167,7 @@ public void testUpgradeHiveNonPartitionTable(String format, boolean isNamedArgum @ParameterizedTest @ValueSource(strings = {"orc", "parquet", "avro"}) - public void testMigrateActionForHive(String format) throws Exception { + public void testMigrateAction(String format) throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')"); tEnv.useCatalog("HIVE"); @@ -219,71 +208,6 @@ public void testMigrateActionForHive(String format) throws Exception { Assertions.assertThatList(r1).containsExactlyInAnyOrderElementsOf(r2); } - @ParameterizedTest - @MethodSource("testIcebergArguments") - public void testMigrateIcebergUnPartitionedTable(boolean isPartitioned, boolean isHive) - throws Exception { - TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build(); - - // create iceberg catalog, database, table, and insert some data to iceberg table - tEnv.executeSql(icebergCatalogDdl(isHive)); - tEnv.executeSql("USE CATALOG my_iceberg"); - tEnv.executeSql("CREATE DATABASE iceberg_db;"); - if (isPartitioned) { - tEnv.executeSql( - "CREATE TABLE iceberg_db.iceberg_table (id string, id2 int, id3 int) PARTITIONED BY (id3)" - + " WITH ('format-version'='2')"); - } else { - tEnv.executeSql( - "CREATE TABLE iceberg_db.iceberg_table (id string, id2 int, id3 int) WITH ('format-version'='2')"); - } - tEnv.executeSql("INSERT INTO iceberg_db.iceberg_table VALUES ('a',1,1),('b',2,2),('c',3,3)") - .await(); - - tEnv.executeSql(paimonCatalogDdl(isHive)); - tEnv.executeSql("USE CATALOG my_paimon"); - tEnv.executeSql( - String.format( - "CALL sys.migrate_table(connector => 'iceberg', " - + "iceberg_options => 'iceberg-meta-path=%s,target-database=%s,target-table=%s')", - iceTempDir + "/iceberg_db/iceberg_table/metadata", - "paimon_db", - "paimon_table")) - .await(); - - Assertions.assertThatList( - Arrays.asList(Row.of("a", 1, 1), Row.of("b", 2, 2), Row.of("c", 3, 3))) - .containsExactlyInAnyOrderElementsOf( - ImmutableList.copyOf( - tEnv.executeSql("SELECT * FROM paimon_db.paimon_table").collect())); - } - - private String icebergCatalogDdl(boolean isHive) { - return isHive - ? String.format( - "CREATE CATALOG my_iceberg WITH " - + "( 'type' = 'iceberg', 'catalog-type' = 'hive', 'uri' = 'thrift://localhost:%s', " - + "'warehouse' = '%s', 'cache-enabled' = 'false' )", - PORT, iceTempDir) - : String.format( - "CREATE CATALOG my_iceberg WITH " - + "( 'type' = 'iceberg', 'catalog-type' = 'hadoop'," - + "'warehouse' = '%s', 'cache-enabled' = 'false' )", - iceTempDir); - } - - private String paimonCatalogDdl(boolean isHive) { - return isHive - ? String.format( - "CREATE CATALOG my_paimon WITH " - + "( 'type' = 'paimon', 'metastore' = 'hive', 'uri' = 'thrift://localhost:%s', " - + "'warehouse' = '%s', 'cache-enabled' = 'false' )", - PORT, iceTempDir) - : String.format( - "CREATE CATALOG my_paimon WITH ('type' = 'paimon', 'warehouse' = '%s')", - paiTempDir); - } - protected static String data(int i) { Random random = new Random(); StringBuilder stringBuilder = new StringBuilder(); diff --git a/paimon-hive/pom.xml b/paimon-hive/pom.xml index 20314b350832..43bb7f5a2211 100644 --- a/paimon-hive/pom.xml +++ b/paimon-hive/pom.xml @@ -130,20 +130,6 @@ under the License. ${project.version} test - - - org.apache.iceberg - iceberg-flink-${iceberg.flink.version} - ${iceberg.version} - test - - - - org.apache.flink - flink-metrics-dropwizard - ${iceberg.flink.dropwizard.version} - test - From ec9dadc8fb1f007b6815d95f1e57df375fedea1f Mon Sep 17 00:00:00 2001 From: LsomeYeah Date: Mon, 9 Dec 2024 13:46:25 +0800 Subject: [PATCH 17/17] fix test --- .../hive/procedure/MigrateIcebergTableProcedureITCase.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java index 11d461b0c2a6..f7de3a954450 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateIcebergTableProcedureITCase.java @@ -29,10 +29,13 @@ import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.types.Row; import org.assertj.core.api.Assertions; +import org.junit.Rule; +import org.junit.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import java.util.Arrays; @@ -41,6 +44,7 @@ /** IT cases for migrating iceberg table to paimon table. */ @RunWith(PaimonEmbeddedHiveRunner.class) public abstract class MigrateIcebergTableProcedureITCase { + @Rule public TemporaryFolder folder = new TemporaryFolder(); @TempDir private java.nio.file.Path iceTempDir; @TempDir private java.nio.file.Path paiTempDir; @@ -52,6 +56,9 @@ private static Stream testIcebergArguments() { return Stream.of(Arguments.of(true, false), Arguments.of(false, false)); } + @Test + public void initTest() {} + @ParameterizedTest @MethodSource("testIcebergArguments") public void testMigrateIcebergUnPartitionedTable(boolean isPartitioned, boolean isHive)