From 77670204c98501f9e86f04bc16d651ba52f9594a Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 16 Dec 2024 19:41:24 +0800 Subject: [PATCH 01/17] [core] Remove Catalog.getTableLocation interface (#4718) --- .../paimon/catalog/AbstractCatalog.java | 1 - .../org/apache/paimon/catalog/Catalog.java | 9 ---- .../paimon/catalog/DelegateCatalog.java | 6 --- .../org/apache/paimon/rest/RESTCatalog.java | 6 --- .../org/apache/paimon/flink/FlinkCatalog.java | 15 ------- .../paimon/flink/clone/CopyFileOperator.java | 41 ++++++++++++++++--- .../apache/paimon/flink/FlinkCatalogTest.java | 8 +--- .../sink/partition/PartitionMarkDoneTest.java | 2 +- 8 files changed, 38 insertions(+), 50 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index b56fec279ab1..a1b41e3b8a41 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -496,7 +496,6 @@ public Optional metastoreClientFactory( return Optional.empty(); } - @Override public Path getTableLocation(Identifier identifier) { return new Path(newDatabasePath(identifier.getDatabaseName()), identifier.getTableName()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index d919c5978297..c3808caa135a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -20,7 +20,6 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -136,14 +135,6 @@ void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) */ Table getTable(Identifier identifier) throws TableNotExistException; - /** - * Get the table location in this catalog. If the table exists, return the location of the - * table; If the table does not exist, construct the location for table. - * - * @return the table location - */ - Path getTableLocation(Identifier identifier); - /** * Get names of all tables under this database. An empty list is returned if none exists. * diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index ec14d53a2b03..2298626b0e48 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -19,7 +19,6 @@ package org.apache.paimon.catalog; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -147,11 +146,6 @@ public void renameView(Identifier fromView, Identifier toView, boolean ignoreIfN wrapped.renameView(fromView, toView, ignoreIfNotExists); } - @Override - public Path getTableLocation(Identifier identifier) { - return wrapped.getTableLocation(identifier); - } - @Override public void createPartition(Identifier identifier, Map partitions) throws TableNotExistException { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index 03b257efbf86..86b87e25e832 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -22,7 +22,6 @@ import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; @@ -177,11 +176,6 @@ public Table getTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); } - @Override - public Path getTableLocation(Identifier identifier) { - throw new UnsupportedOperationException(); - } - @Override public List listTables(String databaseName) throws DatabaseNotExistException { return new ArrayList(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 3a7f9790ccca..dd95c48af8d1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -25,7 +25,6 @@ import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.paimon.flink.utils.FlinkDescriptorProperties; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.operation.FileStoreCommit; import org.apache.paimon.options.Options; @@ -525,20 +524,6 @@ protected Schema buildPaimonSchema( registerLogSystem(catalog, identifier, options, classLoader); } - // remove table path - String path = options.remove(PATH.key()); - if (path != null) { - Path expectedPath = catalog.getTableLocation(identifier); - if (!new Path(path).equals(expectedPath)) { - throw new CatalogException( - String.format( - "You specified the Path when creating the table, " - + "but the Path '%s' is different from where it should be '%s'. " - + "Please remove the Path.", - path, expectedPath)); - } - } - if (catalogTable instanceof CatalogTable) { return fromCatalogTable(((CatalogTable) catalogTable).copy(options)); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java index d3554242994b..8bcaa2a2071f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java @@ -18,12 +18,14 @@ package org.apache.paimon.flink.clone; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; +import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; @@ -32,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; import java.util.Map; /** A Operator to copy files. */ @@ -43,8 +46,11 @@ public class CopyFileOperator extends AbstractStreamOperator private final Map sourceCatalogConfig; private final Map targetCatalogConfig; - private Catalog sourceCatalog; - private Catalog targetCatalog; + private transient Catalog sourceCatalog; + private transient Catalog targetCatalog; + + private transient Map srcLocations; + private transient Map targetLocations; public CopyFileOperator( Map sourceCatalogConfig, Map targetCatalogConfig) { @@ -58,6 +64,8 @@ public void open() throws Exception { FlinkCatalogFactory.createPaimonCatalog(Options.fromMap(sourceCatalogConfig)); targetCatalog = FlinkCatalogFactory.createPaimonCatalog(Options.fromMap(targetCatalogConfig)); + srcLocations = new HashMap<>(); + targetLocations = new HashMap<>(); } @Override @@ -66,12 +74,29 @@ public void processElement(StreamRecord streamRecord) throws Exce FileIO sourceTableFileIO = sourceCatalog.fileIO(); FileIO targetTableFileIO = targetCatalog.fileIO(); + Path sourceTableRootPath = - sourceCatalog.getTableLocation( - Identifier.fromString(cloneFileInfo.getSourceIdentifier())); + srcLocations.computeIfAbsent( + cloneFileInfo.getSourceIdentifier(), + key -> { + try { + return pathOfTable( + sourceCatalog.getTable(Identifier.fromString(key))); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException(e); + } + }); Path targetTableRootPath = - targetCatalog.getTableLocation( - Identifier.fromString(cloneFileInfo.getTargetIdentifier())); + targetLocations.computeIfAbsent( + cloneFileInfo.getTargetIdentifier(), + key -> { + try { + return pathOfTable( + targetCatalog.getTable(Identifier.fromString(key))); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException(e); + } + }); String filePathExcludeTableRoot = cloneFileInfo.getFilePathExcludeTableRoot(); Path sourcePath = new Path(sourceTableRootPath + filePathExcludeTableRoot); @@ -110,6 +135,10 @@ public void processElement(StreamRecord streamRecord) throws Exce output.collect(streamRecord); } + private Path pathOfTable(Table table) { + return new Path(table.options().get(CoreOptions.PATH.key())); + } + @Override public void close() throws Exception { if (sourceCatalog != null) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index e4286eb18172..734a47dead06 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -102,6 +102,7 @@ /** Test for {@link FlinkCatalog}. */ public class FlinkCatalogTest { + private static final String TESTING_LOG_STORE = "testing"; private final ObjectPath path1 = new ObjectPath("db1", "t1"); @@ -348,12 +349,7 @@ public void testCreateFlinkTableWithPath() throws Exception { CatalogTable table1 = createTable(options); assertThatThrownBy(() -> catalog.createTable(this.path1, table1, false)) .hasMessageContaining( - "You specified the Path when creating the table, " - + "but the Path '/unknown/path' is different from where it should be"); - - options.put(PATH.key(), warehouse + "/db1.db/t1"); - CatalogTable table2 = createTable(options); - catalog.createTable(this.path1, table2, false); + "The current catalog FileSystemCatalog does not support specifying the table path when creating a table."); } @ParameterizedTest diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java index f0f4596c61bb..9e5fe7ff9ff7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java @@ -83,7 +83,7 @@ private void innerTest(boolean deletionVectors) throws Exception { .build(); catalog.createTable(identifier, schema, true); FileStoreTable table = (FileStoreTable) catalog.getTable(identifier); - Path location = catalog.getTableLocation(identifier); + Path location = table.location(); Path successFile = new Path(location, "a=0/_SUCCESS"); PartitionMarkDone markDone = PartitionMarkDone.create(false, false, new MockOperatorStateStore(), table).get(); From c9eafb61216114c1e6dfed3493293b5485dbc3a4 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Mon, 16 Dec 2024 19:42:52 +0800 Subject: [PATCH 02/17] [core][spark] Fix create external table with schema evolution (#4719) --- .../org/apache/paimon/types/ArrayType.java | 15 +++++ .../org/apache/paimon/types/DataField.java | 26 ++++++--- .../org/apache/paimon/types/DataType.java | 4 ++ .../java/org/apache/paimon/types/MapType.java | 16 ++++++ .../java/org/apache/paimon/types/RowType.java | 24 ++++++-- .../java/org/apache/paimon/schema/Schema.java | 1 - .../apache/paimon/schema/SchemaManager.java | 15 ++--- .../org/apache/paimon/schema/TableSchema.java | 4 ++ .../sql/DDLWithHiveCatalogTestBase.scala | 57 +++++++++++++++++++ 9 files changed, 141 insertions(+), 21 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java b/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java index 00cf0c072886..62fb9ce65b69 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java @@ -100,6 +100,21 @@ public boolean equals(Object o) { return elementType.equals(arrayType.elementType); } + @Override + public boolean equalsIgnoreFieldId(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ArrayType arrayType = (ArrayType) o; + return elementType.equalsIgnoreFieldId(arrayType.elementType); + } + @Override public boolean isPrunedFrom(Object o) { if (this == o) { diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java index e9052684f33f..209118023ba5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java @@ -148,17 +148,29 @@ public boolean equals(Object o) { && Objects.equals(description, field.description); } - public boolean isPrunedFrom(DataField field) { - if (this == field) { + public boolean equalsIgnoreFieldId(DataField other) { + if (this == other) { return true; } - if (field == null) { + if (other == null) { return false; } - return Objects.equals(id, field.id) - && Objects.equals(name, field.name) - && type.isPrunedFrom(field.type) - && Objects.equals(description, field.description); + return Objects.equals(name, other.name) + && type.equalsIgnoreFieldId(other.type) + && Objects.equals(description, other.description); + } + + public boolean isPrunedFrom(DataField other) { + if (this == other) { + return true; + } + if (other == null) { + return false; + } + return Objects.equals(id, other.id) + && Objects.equals(name, other.name) + && type.isPrunedFrom(other.type) + && Objects.equals(description, other.description); } @Override diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java index aff150090e63..dd9a4685ef5b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java @@ -145,6 +145,10 @@ public boolean equals(Object o) { return isNullable == that.isNullable && typeRoot == that.typeRoot; } + public boolean equalsIgnoreFieldId(Object o) { + return equals(o); + } + /** * Determine whether the current type is the result of the target type after pruning (e.g. * select some fields from a nested type) or just the same. diff --git a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java b/paimon-common/src/main/java/org/apache/paimon/types/MapType.java index 57cb1a9724b3..b715d49284fa 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/MapType.java @@ -109,6 +109,22 @@ public boolean equals(Object o) { return keyType.equals(mapType.keyType) && valueType.equals(mapType.valueType); } + @Override + public boolean equalsIgnoreFieldId(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + MapType mapType = (MapType) o; + return keyType.equalsIgnoreFieldId(mapType.keyType) + && valueType.equalsIgnoreFieldId(mapType.valueType); + } + @Override public boolean isPrunedFrom(Object o) { if (this == o) { diff --git a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java b/paimon-common/src/main/java/org/apache/paimon/types/RowType.java index f3fce0db6df1..625a4634b320 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/RowType.java @@ -210,13 +210,25 @@ public boolean equals(Object o) { return false; } RowType rowType = (RowType) o; - // For nested RowTypes e.g. DataField.dataType = RowType we need to ignoreIds as they can be - // different - if (fields.size() != rowType.fields.size()) { + return fields.equals(rowType.fields); + } + + public boolean equalsIgnoreFieldId(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { return false; } - for (int i = 0; i < fields.size(); ++i) { - if (!DataField.dataFieldEqualsIgnoreId(fields.get(i), rowType.fields.get(i))) { + if (!super.equals(o)) { + return false; + } + RowType other = (RowType) o; + if (fields.size() != other.fields.size()) { + return false; + } + for (int i = 0; i < fields.size(); i++) { + if (!fields.get(i).equalsIgnoreFieldId(other.fields.get(i))) { return false; } } @@ -236,7 +248,7 @@ public boolean isPrunedFrom(Object o) { } RowType rowType = (RowType) o; for (DataField field : fields) { - if (!field.isPrunedFrom(rowType.getField(field.name()))) { + if (!field.isPrunedFrom(rowType.getField(field.id()))) { return false; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java index 33309a7cecc9..a3b30d81a3dd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java @@ -68,7 +68,6 @@ public Schema( this.partitionKeys = normalizePartitionKeys(partitionKeys); this.primaryKeys = normalizePrimaryKeys(primaryKeys); this.fields = normalizeFields(fields, this.primaryKeys, this.partitionKeys); - this.comment = comment; } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 2139dca4a990..325e0c392d12 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -214,7 +214,7 @@ public TableSchema createTable(Schema schema, boolean externalTable) throws Exce if (latest.isPresent()) { TableSchema latestSchema = latest.get(); if (externalTable) { - checkSchemaForExternalTable(latestSchema, schema); + checkSchemaForExternalTable(latestSchema.toSchema(), schema); return latestSchema; } else { throw new IllegalStateException( @@ -248,14 +248,15 @@ public TableSchema createTable(Schema schema, boolean externalTable) throws Exce } } - private void checkSchemaForExternalTable(TableSchema existsSchema, Schema newSchema) { + private void checkSchemaForExternalTable(Schema existsSchema, Schema newSchema) { // When creating an external table, if the table already exists in the location, we can // choose not to specify the fields. - if (newSchema.fields().isEmpty() - // When the fields are explicitly specified, we need check for consistency. - || (Objects.equals(existsSchema.fields(), newSchema.fields()) - && Objects.equals(existsSchema.partitionKeys(), newSchema.partitionKeys()) - && Objects.equals(existsSchema.primaryKeys(), newSchema.primaryKeys()))) { + if ((newSchema.fields().isEmpty() + || newSchema.rowType().equalsIgnoreFieldId(existsSchema.rowType())) + && (newSchema.partitionKeys().isEmpty() + || Objects.equals(newSchema.partitionKeys(), existsSchema.partitionKeys())) + && (newSchema.primaryKeys().isEmpty() + || Objects.equals(newSchema.primaryKeys(), existsSchema.primaryKeys()))) { // check for options Map existsOptions = existsSchema.options(); Map newOptions = newSchema.options(); diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java index a0a149d1ae9b..791269dc73b5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java @@ -331,6 +331,10 @@ public static List newFields(RowType rowType) { return rowType.getFields(); } + public Schema toSchema() { + return new Schema(fields, partitionKeys, primaryKeys, options, comment); + } + // =================== Utils for reading ========================= public static TableSchema fromJson(String json) { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 1189f1f2906b..4ba079ea0bb2 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -434,6 +434,63 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: create external table with schema evolution") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + tbLocation => + withDatabase("paimon_db") { + spark.sql(s"CREATE DATABASE IF NOT EXISTS paimon_db") + spark.sql(s"USE paimon_db") + withTable("t1", "t2") { + val expertTbLocation = tbLocation.getCanonicalPath + spark.sql( + s""" + |CREATE TABLE t1 (a INT, b INT, c STRUCT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + spark.sql("INSERT INTO t1 VALUES (1, 1, STRUCT(1, 1, 1))") + spark.sql("ALTER TABLE t1 DROP COLUMN b") + spark.sql("ALTER TABLE t1 ADD COLUMN b INT") + spark.sql("ALTER TABLE t1 DROP COLUMN c.f2") + spark.sql("ALTER TABLE t1 ADD COLUMN c.f2 INT") + spark.sql("INSERT INTO t1 VALUES (2, STRUCT(1, 1, 1), 1)") + checkAnswer( + spark.sql("SELECT * FROM t1 ORDER by a"), + Seq(Row(1, Row(1, 1, null), null), Row(2, Row(1, 1, 1), 1))) + + spark.sql( + s""" + |CREATE TABLE t2 (a INT, c STRUCT, b INT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + checkAnswer( + spark.sql("SELECT * FROM t2 ORDER by a"), + Seq(Row(1, Row(1, 1, null), null), Row(2, Row(1, 1, 1), 1))) + + // create table with wrong schema + intercept[Exception] { + spark.sql( + s""" + |CREATE TABLE t3 (a INT, b INT, c STRUCT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + } + + intercept[Exception] { + spark.sql( + s""" + |CREATE TABLE t4 (a INT, c STRUCT, b INT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + } + } + } + } + } + } + def getDatabaseProp(dbName: String, propertyName: String): String = { spark .sql(s"DESC DATABASE EXTENDED $dbName") From 683fa199e270459b435b58a2a22a4d8d5c6120ca Mon Sep 17 00:00:00 2001 From: zyz33 <35164637+zhangyazhe@users.noreply.github.com> Date: Mon, 16 Dec 2024 21:18:58 +0800 Subject: [PATCH 03/17] [orc] Optimize configuration creating in orc file format (#4716) --- .../paimon/format/orc/OrcFileFormat.java | 4 +- .../paimon/format/orc/OrcFileFormatTest.java | 64 +++++++++++++++++++ 2 files changed, 66 insertions(+), 2 deletions(-) diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java index c3521c6f1a37..9acea56ab393 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java @@ -74,9 +74,9 @@ public class OrcFileFormat extends FileFormat { public OrcFileFormat(FormatContext formatContext) { super(IDENTIFIER); this.orcProperties = getOrcProperties(formatContext.options(), formatContext); - this.readerConf = new org.apache.hadoop.conf.Configuration(); + this.readerConf = new org.apache.hadoop.conf.Configuration(false); this.orcProperties.forEach((k, v) -> readerConf.set(k.toString(), v.toString())); - this.writerConf = new org.apache.hadoop.conf.Configuration(); + this.writerConf = new org.apache.hadoop.conf.Configuration(false); this.orcProperties.forEach((k, v) -> writerConf.set(k.toString(), v.toString())); this.readBatchSize = formatContext.readBatchSize(); this.writeBatchSize = formatContext.writeBatchSize(); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java index 46bf6afe6613..9e5769595c32 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java @@ -83,4 +83,68 @@ public void testSupportedDataTypes() { dataFields.add(new DataField(index++, "decimal_type", DataTypes.DECIMAL(10, 3))); orc.validateDataFields(new RowType(dataFields)); } + + @Test + public void testCreateCost() { + double createConfCost = createConfigCost(); + for (int i = 0; i < 1000; i++) { + create(); + } + int times = 10_000; + long start = System.nanoTime(); + for (int i = 0; i < times; i++) { + create(); + } + double cost = ((double) (System.nanoTime() - start)) / 1000_000 / times; + assertThat(cost * 500 < createConfCost).isTrue(); + } + + @Test + public void testCreateCostWithRandomConfig() { + double createConfCost = createConfigCost(); + for (int i = 0; i < 1000; i++) { + createRandomConfig(); + } + int times = 10_000; + long start = System.nanoTime(); + for (int i = 0; i < times; i++) { + createRandomConfig(); + } + double cost = ((double) (System.nanoTime() - start)) / 1000_000 / times; + assertThat(cost * 10 < createConfCost).isTrue(); + } + + private double createConfigCost() { + for (int i = 0; i < 1000; i++) { + createConfig(); + } + int times = 10_000; + long start = System.nanoTime(); + for (int i = 0; i < times; i++) { + createConfig(); + } + return ((double) (System.nanoTime() - start)) / 1000_000 / times; + } + + private void createConfig() { + org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(); + conf.set("a", "a"); + } + + private void create() { + Options options = new Options(); + options.setString("haha", "1"); + options.setString("compress", "zlib"); + OrcFileFormat orcFileFormat = + new OrcFileFormatFactory().create(new FormatContext(options, 1024, 1024)); + } + + private void createRandomConfig() { + Options options = new Options(); + options.setString("haha", "1"); + options.setString("compress", "zlib"); + options.setString("a", Math.random() + ""); + OrcFileFormat orcFileFormat = + new OrcFileFormatFactory().create(new FormatContext(options, 1024, 1024)); + } } From cdd5bb72f706901f6978a71832e4ee1c78934e08 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 16 Dec 2024 22:33:22 +0800 Subject: [PATCH 04/17] [core] Clean constants, caseSensitive, loader in Catalog (#4721) --- docs/content/maintenance/configurations.md | 6 -- docs/content/program-api/flink-api.md | 2 +- .../generated/catalog_configuration.html | 12 +-- .../generated/core_configuration.html | 12 +-- .../file_system_catalog_configuration.html | 36 --------- .../paimon/arrow/ArrowBundleRecords.java | 8 +- .../org/apache/paimon/arrow/ArrowUtils.java | 10 ++- .../paimon/arrow/reader/ArrowBatchReader.java | 11 +-- .../arrow/vector/ArrowFormatCWriter.java | 4 +- .../arrow/vector/ArrowFormatWriter.java | 4 +- .../apache/paimon/options/CatalogOptions.java | 9 +-- .../org/apache/paimon/utils/StringUtils.java | 4 +- .../paimon/catalog/AbstractCatalog.java | 15 ++-- .../org/apache/paimon/catalog/Catalog.java | 74 +++++++------------ ...CatalogOptions.java => CatalogLoader.java} | 18 ++--- .../apache/paimon/catalog/CatalogUtils.java | 34 +++++++++ .../paimon/catalog/DelegateCatalog.java | 4 +- .../paimon/catalog/FileSystemCatalog.java | 6 +- .../org/apache/paimon/jdbc/JdbcCatalog.java | 2 +- .../org/apache/paimon/rest/RESTCatalog.java | 11 +-- .../paimon/catalog/FileSystemCatalogTest.java | 4 +- .../action/cdc/CdcActionCommonUtils.java | 8 +- .../paimon/flink/action/cdc/Expression.java | 2 +- .../cdc/MessageQueueSyncTableActionBase.java | 2 +- .../action/cdc/SyncDatabaseActionBase.java | 11 +-- .../flink/action/cdc/SyncTableActionBase.java | 11 +-- .../action/cdc/SynchronizationActionBase.java | 4 +- .../flink/action/cdc/TableNameConverter.java | 6 +- .../cdc/mysql/MySqlSyncDatabaseAction.java | 4 +- .../flink/sink/cdc/CaseSensitiveUtils.java | 13 ++-- ...CdcDynamicTableParsingProcessFunction.java | 5 +- .../CdcMultiplexRecordChannelComputer.java | 5 +- .../cdc/CdcRecordStoreMultiWriteOperator.java | 9 ++- .../paimon/flink/sink/cdc/CdcSinkBuilder.java | 6 +- .../sink/cdc/FlinkCdcMultiTableSink.java | 6 +- .../cdc/FlinkCdcSyncDatabaseSinkBuilder.java | 6 +- ...TableUpdatedDataFieldsProcessFunction.java | 3 +- .../flink/sink/cdc/RichCdcSinkBuilder.java | 8 +- .../cdc/UpdatedDataFieldsProcessFunction.java | 4 +- .../UpdatedDataFieldsProcessFunctionBase.java | 12 +-- .../flink/action/cdc/SchemaEvolutionTest.java | 4 +- .../KafkaCanalSyncDatabaseActionITCase.java | 4 +- .../KafkaCanalSyncTableActionITCase.java | 4 +- .../kafka/KafkaSyncDatabaseActionITCase.java | 4 +- .../MongoDBSyncDatabaseActionITCase.java | 4 +- .../mongodb/MongoDBSyncTableActionITCase.java | 4 +- .../mysql/MySqlSyncDatabaseActionITCase.java | 5 +- .../cdc/mysql/MySqlSyncTableActionITCase.java | 5 +- ...CdcMultiplexRecordChannelComputerTest.java | 3 +- .../CdcRecordStoreMultiWriteOperatorTest.java | 7 +- .../cdc/FlinkCdcSyncDatabaseSinkITCase.java | 5 +- .../sink/cdc/FlinkCdcSyncTableSinkITCase.java | 5 +- .../org/apache/paimon/flink/FlinkCatalog.java | 3 +- .../paimon/flink/action/ActionBase.java | 3 +- .../compact/MultiAwareBucketTableScan.java | 4 +- .../flink/compact/MultiTableScanBase.java | 3 +- .../compact/MultiUnawareBucketTableScan.java | 4 +- ...nlyMultiTableCompactionWorkerOperator.java | 9 ++- .../sink/CombinedTableCompactorSink.java | 6 +- .../sink/MultiTablesStoreCompactOperator.java | 9 ++- .../flink/sink/StoreMultiCommitter.java | 5 +- .../sink/partition/PartitionListeners.java | 5 +- ....java => PartitionStatisticsReporter.java} | 17 +++-- ...ener.java => ReportPartStatsListener.java} | 22 +++--- .../CombinedTableCompactorSourceBuilder.java | 7 +- .../operator/CombinedAwareBatchSource.java | 6 +- .../CombinedAwareStreamingSource.java | 6 +- .../operator/CombinedCompactorSource.java | 9 ++- .../operator/CombinedUnawareBatchSource.java | 5 +- .../CombinedUnawareStreamingSource.java | 6 +- .../operator/MultiTablesReadOperator.java | 7 +- .../MultiUnawareTablesReadOperator.java | 6 +- .../flink/sink/CompactorSinkITCase.java | 3 +- .../flink/sink/StoreMultiCommitterTest.java | 5 +- ...a => PartitionStatisticsReporterTest.java} | 8 +- ...ltiTablesCompactorSourceBuilderITCase.java | 3 +- .../org/apache/paimon/hive/HiveCatalog.java | 7 +- .../org/apache/paimon/spark/SparkCatalog.java | 6 +- .../paimon/spark/SparkGenericCatalog.java | 6 +- 79 files changed, 320 insertions(+), 325 deletions(-) delete mode 100644 docs/layouts/shortcodes/generated/file_system_catalog_configuration.html rename paimon-core/src/main/java/org/apache/paimon/catalog/{FileSystemCatalogOptions.java => CatalogLoader.java} (55%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/{HmsReporter.java => PartitionStatisticsReporter.java} (85%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/{ReportHmsListener.java => ReportPartStatsListener.java} (91%) rename paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/{HmsReporterTest.java => PartitionStatisticsReporterTest.java} (95%) diff --git a/docs/content/maintenance/configurations.md b/docs/content/maintenance/configurations.md index 3849d70a5862..99f797e68fa6 100644 --- a/docs/content/maintenance/configurations.md +++ b/docs/content/maintenance/configurations.md @@ -38,12 +38,6 @@ Options for paimon catalog. {{< generated/catalog_configuration >}} -### FilesystemCatalogOptions - -Options for Filesystem catalog. - -{{< generated/file_system_catalog_configuration >}} - ### HiveCatalogOptions Options for Hive catalog. diff --git a/docs/content/program-api/flink-api.md b/docs/content/program-api/flink-api.md index 6ecac3909ced..3451b40d5880 100644 --- a/docs/content/program-api/flink-api.md +++ b/docs/content/program-api/flink-api.md @@ -221,7 +221,7 @@ public class WriteCdcToTable { Identifier identifier = Identifier.create("my_db", "T"); Options catalogOptions = new Options(); catalogOptions.set("warehouse", "/path/to/warehouse"); - Catalog.Loader catalogLoader = + CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); Table table = catalogLoader.load().getTable(identifier); diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html index 63f7adda1e0d..6355c9558653 100644 --- a/docs/layouts/shortcodes/generated/catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/catalog_configuration.html @@ -26,12 +26,6 @@ - -
allow-upper-case
- (none) - Boolean - Indicates whether this catalog allow upper case, its default value depends on the implementation of the specific catalog. -
cache-enabled
true @@ -74,6 +68,12 @@ Integer Controls the max number for snapshots per table in the catalog are cached. + +
case-sensitive
+ (none) + Boolean + Indicates whether this catalog is case-sensitive. +
client-pool-size
2 diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 15b1aac93543..1133de289fa3 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -230,6 +230,12 @@ String Specify the file name prefix of data files. + +
data-file.thin-mode
+ false + Boolean + Enable data file thin mode to avoid duplicate columns storage. +
delete-file.thread-num
(none) @@ -864,12 +870,6 @@ Integer Default spill compression zstd level. For higher compression rates, it can be configured to 9, but the read and write speed will significantly decrease. - -
data-file.thin-mode
- false - Boolean - Enable data file thin mode to avoid duplicate columns storage. -
streaming-read-mode
(none) diff --git a/docs/layouts/shortcodes/generated/file_system_catalog_configuration.html b/docs/layouts/shortcodes/generated/file_system_catalog_configuration.html deleted file mode 100644 index c416ed6da557..000000000000 --- a/docs/layouts/shortcodes/generated/file_system_catalog_configuration.html +++ /dev/null @@ -1,36 +0,0 @@ -{{/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/}} - - - - - - - - - - - - - - - - - -
KeyDefaultTypeDescription
case-sensitive
trueBooleanIs case sensitive. If case insensitive, you need to set this option to false, and the table name and fields be converted to lowercase.
diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java index 9627bbd85d8c..25f6603ec22e 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java @@ -32,13 +32,13 @@ public class ArrowBundleRecords implements BundleRecords { private final VectorSchemaRoot vectorSchemaRoot; private final RowType rowType; - private final boolean allowUpperCase; + private final boolean caseSensitive; public ArrowBundleRecords( - VectorSchemaRoot vectorSchemaRoot, RowType rowType, boolean allowUpperCase) { + VectorSchemaRoot vectorSchemaRoot, RowType rowType, boolean caseSensitive) { this.vectorSchemaRoot = vectorSchemaRoot; this.rowType = rowType; - this.allowUpperCase = allowUpperCase; + this.caseSensitive = caseSensitive; } public VectorSchemaRoot getVectorSchemaRoot() { @@ -52,7 +52,7 @@ public long rowCount() { @Override public Iterator iterator() { - ArrowBatchReader arrowBatchReader = new ArrowBatchReader(rowType, allowUpperCase); + ArrowBatchReader arrowBatchReader = new ArrowBatchReader(rowType, caseSensitive); return arrowBatchReader.readBatch(vectorSchemaRoot).iterator(); } } diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java index b3925a0a769e..0f6a98b7a2e0 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java @@ -55,6 +55,8 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + /** Utilities for creating Arrow objects. */ public class ArrowUtils { @@ -66,13 +68,13 @@ public static VectorSchemaRoot createVectorSchemaRoot( } public static VectorSchemaRoot createVectorSchemaRoot( - RowType rowType, BufferAllocator allocator, boolean allowUpperCase) { + RowType rowType, BufferAllocator allocator, boolean caseSensitive) { List fields = rowType.getFields().stream() .map( f -> toArrowField( - allowUpperCase ? f.name() : f.name().toLowerCase(), + toLowerCaseIfNeed(f.name(), caseSensitive), f.id(), f.type(), 0)) @@ -81,9 +83,9 @@ public static VectorSchemaRoot createVectorSchemaRoot( } public static FieldVector createVector( - DataField dataField, BufferAllocator allocator, boolean allowUpperCase) { + DataField dataField, BufferAllocator allocator, boolean caseSensitive) { return toArrowField( - allowUpperCase ? dataField.name() : dataField.name().toLowerCase(), + toLowerCaseIfNeed(dataField.name(), caseSensitive), dataField.id(), dataField.type(), 0) diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java index 9d20062b437b..b626758dedfc 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java @@ -34,6 +34,8 @@ import java.util.Iterator; import java.util.List; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + /** Reader from a {@link VectorSchemaRoot} to paimon rows. */ public class ArrowBatchReader { @@ -41,9 +43,9 @@ public class ArrowBatchReader { private final VectorizedColumnBatch batch; private final Arrow2PaimonVectorConverter[] convertors; private final RowType projectedRowType; - private final boolean allowUpperCase; + private final boolean caseSensitive; - public ArrowBatchReader(RowType rowType, boolean allowUpperCase) { + public ArrowBatchReader(RowType rowType, boolean caseSensitive) { this.internalRowSerializer = new InternalRowSerializer(rowType); ColumnVector[] columnVectors = new ColumnVector[rowType.getFieldCount()]; this.convertors = new Arrow2PaimonVectorConverter[rowType.getFieldCount()]; @@ -53,7 +55,7 @@ public ArrowBatchReader(RowType rowType, boolean allowUpperCase) { for (int i = 0; i < columnVectors.length; i++) { this.convertors[i] = Arrow2PaimonVectorConverter.construct(rowType.getTypeAt(i)); } - this.allowUpperCase = allowUpperCase; + this.caseSensitive = caseSensitive; } public Iterable readBatch(VectorSchemaRoot vsr) { @@ -63,8 +65,7 @@ public Iterable readBatch(VectorSchemaRoot vsr) { for (int i = 0; i < dataFields.size(); ++i) { try { String fieldName = dataFields.get(i).name(); - Field field = - arrowSchema.findField(allowUpperCase ? fieldName : fieldName.toLowerCase()); + Field field = arrowSchema.findField(toLowerCaseIfNeed(fieldName, caseSensitive)); int idx = arrowSchema.getFields().indexOf(field); mapping[i] = idx; } catch (IllegalArgumentException e) { diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java index 10afcaf6917a..442457813ace 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java @@ -37,8 +37,8 @@ public class ArrowFormatCWriter implements AutoCloseable { private final ArrowSchema schema; private final ArrowFormatWriter realWriter; - public ArrowFormatCWriter(RowType rowType, int writeBatchSize, boolean allowUpperCase) { - this.realWriter = new ArrowFormatWriter(rowType, writeBatchSize, allowUpperCase); + public ArrowFormatCWriter(RowType rowType, int writeBatchSize, boolean caseSensitive) { + this.realWriter = new ArrowFormatWriter(rowType, writeBatchSize, caseSensitive); RootAllocator allocator = realWriter.getAllocator(); array = ArrowArray.allocateNew(allocator); schema = ArrowSchema.allocateNew(allocator); diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java index acdb5d0dcb1d..9f557921979b 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java @@ -43,10 +43,10 @@ public class ArrowFormatWriter implements AutoCloseable { private final RootAllocator allocator; private int rowId; - public ArrowFormatWriter(RowType rowType, int writeBatchSize, boolean allowUpperCase) { + public ArrowFormatWriter(RowType rowType, int writeBatchSize, boolean caseSensitive) { allocator = new RootAllocator(); - vectorSchemaRoot = ArrowUtils.createVectorSchemaRoot(rowType, allocator, allowUpperCase); + vectorSchemaRoot = ArrowUtils.createVectorSchemaRoot(rowType, allocator, caseSensitive); fieldWriters = new ArrowFieldWriter[rowType.getFieldCount()]; diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java index bb8cfae68284..b22274e011fb 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java @@ -128,13 +128,12 @@ public class CatalogOptions { .withDescription( "Controls the max number for snapshots per table in the catalog are cached."); - public static final ConfigOption ALLOW_UPPER_CASE = - ConfigOptions.key("allow-upper-case") + public static final ConfigOption CASE_SENSITIVE = + ConfigOptions.key("case-sensitive") .booleanType() .noDefaultValue() - .withDescription( - "Indicates whether this catalog allow upper case, " - + "its default value depends on the implementation of the specific catalog."); + .withFallbackKeys("allow-upper-case") + .withDescription("Indicates whether this catalog is case-sensitive."); public static final ConfigOption SYNC_ALL_PROPERTIES = ConfigOptions.key("sync-all-properties") diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java index e184624c0bbf..c4e07e0a6972 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java @@ -542,8 +542,8 @@ public static String quote(String str) { return "`" + str + "`"; } - public static String caseSensitiveConversion(String str, boolean allowUpperCase) { - return allowUpperCase ? str : str.toLowerCase(); + public static String toLowerCaseIfNeed(String str, boolean caseSensitive) { + return caseSensitive ? str : str.toLowerCase(); } public static boolean isNumeric(final CharSequence cs) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index a1b41e3b8a41..db6909295556 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -60,7 +60,6 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.CoreOptions.createCommitUser; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; @@ -82,7 +81,7 @@ protected AbstractCatalog(FileIO fileIO) { protected AbstractCatalog(FileIO fileIO, Options options) { this.fileIO = fileIO; - this.tableDefaultOptions = Catalog.tableDefaultOptions(options.toMap()); + this.tableDefaultOptions = CatalogUtils.tableDefaultOptions(options.toMap()); this.catalogOptions = options; } @@ -123,11 +122,6 @@ protected boolean lockEnabled() { return catalogOptions.getOptional(LOCK_ENABLED).orElse(fileIO.isObjectStore()); } - @Override - public boolean allowUpperCase() { - return catalogOptions.getOptional(ALLOW_UPPER_CASE).orElse(true); - } - protected boolean allowCustomTablePath() { return false; } @@ -559,8 +553,9 @@ protected void checkNotSystemDatabase(String database) { } protected void validateIdentifierNameCaseInsensitive(Identifier identifier) { - Catalog.validateCaseInsensitive(allowUpperCase(), "Database", identifier.getDatabaseName()); - Catalog.validateCaseInsensitive(allowUpperCase(), "Table", identifier.getObjectName()); + CatalogUtils.validateCaseInsensitive( + caseSensitive(), "Database", identifier.getDatabaseName()); + CatalogUtils.validateCaseInsensitive(caseSensitive(), "Table", identifier.getObjectName()); } private void validateFieldNameCaseInsensitiveInSchemaChange(List changes) { @@ -578,7 +573,7 @@ private void validateFieldNameCaseInsensitiveInSchemaChange(List c } protected void validateFieldNameCaseInsensitive(List fieldNames) { - Catalog.validateCaseInsensitive(allowUpperCase(), "Field", fieldNames); + CatalogUtils.validateCaseInsensitive(caseSensitive(), "Field", fieldNames); } private void validateAutoCreateClose(Map options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index c3808caa135a..7b1fe0ea072e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -26,15 +26,9 @@ import org.apache.paimon.table.Table; import org.apache.paimon.view.View; -import java.io.Serializable; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; -import static org.apache.paimon.utils.Preconditions.checkArgument; /** * This interface is responsible for reading and writing metadata such as database/table from a @@ -46,30 +40,38 @@ @Public public interface Catalog extends AutoCloseable { - String DEFAULT_DATABASE = "default"; - + // constants for system table and database String SYSTEM_TABLE_SPLITTER = "$"; String SYSTEM_DATABASE_NAME = "sys"; String SYSTEM_BRANCH_PREFIX = "branch_"; - String TABLE_DEFAULT_OPTION_PREFIX = "table-default."; - String DB_SUFFIX = ".db"; + // constants for table and database String COMMENT_PROP = "comment"; String OWNER_PROP = "owner"; + + // constants for database + String DEFAULT_DATABASE = "default"; + String DB_SUFFIX = ".db"; String DB_LOCATION_PROP = "location"; + + // constants for table + String TABLE_DEFAULT_OPTION_PREFIX = "table-default."; String NUM_ROWS_PROP = "numRows"; String NUM_FILES_PROP = "numFiles"; String TOTAL_SIZE_PROP = "totalSize"; String LAST_UPDATE_TIME_PROP = "lastUpdateTime"; - String HIVE_LAST_UPDATE_TIME_PROP = "transient_lastDdlTime"; - /** Warehouse root path containing all database directories in this catalog. */ + /** Warehouse root path for creating new databases. */ String warehouse(); - /** Catalog options. */ + /** {@link FileIO} of this catalog. It can access {@link #warehouse()} path. */ + FileIO fileIO(); + + /** Catalog options for re-creating this catalog. */ Map options(); - FileIO fileIO(); + /** Return a boolean that indicates whether this catalog is case-sensitive. */ + boolean caseSensitive(); /** * Get the names of all databases in this catalog. @@ -325,44 +327,30 @@ default void renameView(Identifier fromView, Identifier toView, boolean ignoreIf throw new UnsupportedOperationException(); } - /** Return a boolean that indicates whether this catalog allow upper case. */ - boolean allowUpperCase(); - + /** + * Repair the entire Catalog, repair the metadata in the metastore consistent with the metadata + * in the filesystem, register missing tables in the metastore. + */ default void repairCatalog() { throw new UnsupportedOperationException(); } + /** + * Repair the entire database, repair the metadata in the metastore consistent with the metadata + * in the filesystem, register missing tables in the metastore. + */ default void repairDatabase(String databaseName) { throw new UnsupportedOperationException(); } + /** + * Repair the table, repair the metadata in the metastore consistent with the metadata in the + * filesystem. + */ default void repairTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); } - static Map tableDefaultOptions(Map options) { - return convertToPropertiesPrefixKey(options, TABLE_DEFAULT_OPTION_PREFIX); - } - - /** Validate database, table and field names must be lowercase when not case-sensitive. */ - static void validateCaseInsensitive(boolean caseSensitive, String type, String... names) { - validateCaseInsensitive(caseSensitive, type, Arrays.asList(names)); - } - - /** Validate database, table and field names must be lowercase when not case-sensitive. */ - static void validateCaseInsensitive(boolean caseSensitive, String type, List names) { - if (caseSensitive) { - return; - } - List illegalNames = - names.stream().filter(f -> !f.equals(f.toLowerCase())).collect(Collectors.toList()); - checkArgument( - illegalNames.isEmpty(), - String.format( - "%s name %s cannot contain upper case in the catalog.", - type, illegalNames)); - } - /** Exception for trying to drop on a database that is not empty. */ class DatabaseNotEmptyException extends Exception { private static final String MSG = "Database %s is not empty."; @@ -599,10 +587,4 @@ public Identifier identifier() { return identifier; } } - - /** Loader of {@link Catalog}. */ - @FunctionalInterface - interface Loader extends Serializable { - Catalog load(); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLoader.java similarity index 55% rename from paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java rename to paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLoader.java index e656742b42e9..c8de08139cb7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLoader.java @@ -18,19 +18,11 @@ package org.apache.paimon.catalog; -import org.apache.paimon.options.ConfigOption; -import org.apache.paimon.options.ConfigOptions; +import java.io.Serializable; -/** Options for filesystem catalog. */ -public final class FileSystemCatalogOptions { +/** Loader for creating a {@link Catalog}. */ +@FunctionalInterface +public interface CatalogLoader extends Serializable { - public static final ConfigOption CASE_SENSITIVE = - ConfigOptions.key("case-sensitive") - .booleanType() - .defaultValue(true) - .withFallbackKeys("allow-upper-case") - .withDescription( - "Is case sensitive. If case insensitive, you need to set this option to false, and the table name and fields be converted to lowercase."); - - private FileSystemCatalogOptions() {} + Catalog load(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java index 39f81833a9eb..bae23c627607 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java @@ -21,6 +21,15 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.schema.SchemaManager; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.paimon.catalog.Catalog.TABLE_DEFAULT_OPTION_PREFIX; +import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; +import static org.apache.paimon.utils.Preconditions.checkArgument; + /** Utils for {@link Catalog}. */ public class CatalogUtils { @@ -51,4 +60,29 @@ public static String table(Path path) { public static String table(String path) { return SchemaManager.identifierFromPath(path, false).getObjectName(); } + + public static Map tableDefaultOptions(Map options) { + return convertToPropertiesPrefixKey(options, TABLE_DEFAULT_OPTION_PREFIX); + } + + /** Validate database, table and field names must be lowercase when not case-sensitive. */ + public static void validateCaseInsensitive( + boolean caseSensitive, String type, String... names) { + validateCaseInsensitive(caseSensitive, type, Arrays.asList(names)); + } + + /** Validate database, table and field names must be lowercase when not case-sensitive. */ + public static void validateCaseInsensitive( + boolean caseSensitive, String type, List names) { + if (caseSensitive) { + return; + } + List illegalNames = + names.stream().filter(f -> !f.equals(f.toLowerCase())).collect(Collectors.toList()); + checkArgument( + illegalNames.isEmpty(), + String.format( + "%s name %s cannot contain upper case in the catalog.", + type, illegalNames)); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index 2298626b0e48..93e8ce2581ad 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -42,8 +42,8 @@ public Catalog wrapped() { } @Override - public boolean allowUpperCase() { - return wrapped.allowUpperCase(); + public boolean caseSensitive() { + return wrapped.caseSensitive(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index 9264a54647b1..279ddb26ee53 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -34,7 +34,7 @@ import java.util.Map; import java.util.concurrent.Callable; -import static org.apache.paimon.catalog.FileSystemCatalogOptions.CASE_SENSITIVE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; /** A catalog implementation for {@link FileIO}. */ public class FileSystemCatalog extends AbstractCatalog { @@ -158,7 +158,7 @@ public String warehouse() { } @Override - public boolean allowUpperCase() { - return catalogOptions.get(CASE_SENSITIVE); + public boolean caseSensitive() { + return catalogOptions.getOptional(CASE_SENSITIVE).orElse(true); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 778bc591fe89..551b2d8fc910 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -320,7 +320,7 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } @Override - public boolean allowUpperCase() { + public boolean caseSensitive() { return false; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index 86b87e25e832..c30e1109e2ec 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -52,6 +52,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; /** A catalog implementation for REST. */ @@ -61,7 +62,7 @@ public class RESTCatalog implements Catalog { private final RESTClient client; private final ResourcePaths resourcePaths; - private final Map options; + private final Options options; private final Map baseHeader; private final AuthSession catalogAuth; @@ -99,7 +100,7 @@ public RESTCatalog(Options options) { } Map initHeaders = RESTUtil.merge(configHeaders(options.toMap()), this.catalogAuth.getHeaders()); - this.options = fetchOptionsFromServer(initHeaders, options.toMap()); + this.options = new Options(fetchOptionsFromServer(initHeaders, options.toMap())); this.resourcePaths = ResourcePaths.forCatalogProperties( this.options.get(RESTCatalogInternalOptions.PREFIX)); @@ -112,7 +113,7 @@ public String warehouse() { @Override public Map options() { - return this.options; + return this.options.toMap(); } @Override @@ -223,8 +224,8 @@ public List listPartitions(Identifier identifier) } @Override - public boolean allowUpperCase() { - return false; + public boolean caseSensitive() { + return options.getOptional(CASE_SENSITIVE).orElse(true); } @Override diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index 303a9d8733d4..65ea6721c220 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -36,12 +36,12 @@ public class FileSystemCatalogTest extends CatalogTestBase { public void setUp() throws Exception { super.setUp(); Options catalogOptions = new Options(); - catalogOptions.set(CatalogOptions.ALLOW_UPPER_CASE, false); + catalogOptions.set(CatalogOptions.CASE_SENSITIVE, false); catalog = new FileSystemCatalog(fileIO, new Path(warehouse), catalogOptions); } @Test - public void testCreateTableAllowUpperCase() throws Exception { + public void testCreateTableCaseSensitive() throws Exception { catalog.createDatabase("test_db", false); Identifier identifier = Identifier.create("test_db", "new_table"); Schema schema = diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java index c8af6f91c420..6482a625f4c7 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java @@ -42,7 +42,7 @@ import static org.apache.paimon.flink.action.MultiTablesSinkMode.DIVIDED; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkState; -import static org.apache.paimon.utils.StringUtils.caseSensitiveConversion; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; /** Common utils for CDC Action. */ public class CdcActionCommonUtils { @@ -129,21 +129,21 @@ public static Schema buildPaimonSchema( List allFieldNames = new ArrayList<>(); for (DataField field : sourceSchema.fields()) { - String fieldName = caseSensitiveConversion(field.name(), caseSensitive); + String fieldName = toLowerCaseIfNeed(field.name(), caseSensitive); allFieldNames.add(fieldName); builder.column(fieldName, field.type(), field.description()); } for (ComputedColumn computedColumn : computedColumns) { String computedColumnName = - caseSensitiveConversion(computedColumn.columnName(), caseSensitive); + toLowerCaseIfNeed(computedColumn.columnName(), caseSensitive); allFieldNames.add(computedColumnName); builder.column(computedColumnName, computedColumn.columnType()); } for (CdcMetadataConverter metadataConverter : metadataConverters) { String metadataColumnName = - caseSensitiveConversion(metadataConverter.columnName(), caseSensitive); + toLowerCaseIfNeed(metadataConverter.columnName(), caseSensitive); allFieldNames.add(metadataColumnName); builder.column(metadataColumnName, metadataConverter.dataType()); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java index 2e0a1319293f..3290ec18291f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java @@ -189,7 +189,7 @@ public static ReferencedField checkArgument( String[] literals = Arrays.stream(args).skip(1).map(String::trim).toArray(String[]::new); String referencedFieldCheckForm = - StringUtils.caseSensitiveConversion(referencedField, caseSensitive); + StringUtils.toLowerCaseIfNeed(referencedField, caseSensitive); DataType fieldType = checkNotNull( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java index 9c629b5a516f..3af0957ce6da 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java @@ -86,7 +86,7 @@ protected Schema buildPaimonSchema(Schema retrievedSchema) { tableConfig, retrievedSchema, metadataConverters, - allowUpperCase, + caseSensitive, true, false); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java index 4fb1339c5193..56334c1e7bff 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.action.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.flink.action.Action; import org.apache.paimon.flink.action.MultiTablesSinkMode; import org.apache.paimon.flink.sink.cdc.EventParser; @@ -155,9 +156,9 @@ public SyncDatabaseActionBase withPrimaryKeys(String... primaryKeys) { @Override protected void validateCaseSensitivity() { - Catalog.validateCaseInsensitive(allowUpperCase, "Database", database); - Catalog.validateCaseInsensitive(allowUpperCase, "Table prefix", tablePrefix); - Catalog.validateCaseInsensitive(allowUpperCase, "Table suffix", tableSuffix); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Database", database); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Table prefix", tablePrefix); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Table suffix", tableSuffix); } @Override @@ -179,7 +180,7 @@ protected EventParser.Factory buildEventParserFactory() NewTableSchemaBuilder schemaBuilder = new NewTableSchemaBuilder( tableConfig, - allowUpperCase, + caseSensitive, partitionKeys, primaryKeys, requirePrimaryKeys(), @@ -190,7 +191,7 @@ protected EventParser.Factory buildEventParserFactory() excludingTables == null ? null : Pattern.compile(excludingTables); TableNameConverter tableNameConverter = new TableNameConverter( - allowUpperCase, + caseSensitive, mergeShards, dbPrefix, dbSuffix, diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index 87efeb2a19cf..6fcdbd44bca2 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.action.Action; @@ -107,15 +108,15 @@ protected Schema buildPaimonSchema(Schema retrievedSchema) { tableConfig, retrievedSchema, metadataConverters, - allowUpperCase, + caseSensitive, true, true); } @Override protected void validateCaseSensitivity() { - Catalog.validateCaseInsensitive(allowUpperCase, "Database", database); - Catalog.validateCaseInsensitive(allowUpperCase, "Table", table); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Database", database); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Table", table); } @Override @@ -142,7 +143,7 @@ protected void beforeBuildingSourceSink() throws Exception { buildComputedColumns( computedColumnArgs, fileStoreTable.schema().fields(), - allowUpperCase); + caseSensitive); // check partition keys and primary keys in case that user specified them checkConstraints(); } @@ -162,7 +163,7 @@ protected FlatMapFunction recordParse() @Override protected EventParser.Factory buildEventParserFactory() { - boolean caseSensitive = this.allowUpperCase; + boolean caseSensitive = this.caseSensitive; return () -> new RichCdcMultiplexRecordEventParser(caseSensitive); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index a7c770347410..d755b200a957 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -64,7 +64,7 @@ public abstract class SynchronizationActionBase extends ActionBase { protected final String database; protected final Configuration cdcSourceConfig; protected final SyncJobHandler syncJobHandler; - protected final boolean allowUpperCase; + protected final boolean caseSensitive; protected Map tableConfig = new HashMap<>(); protected TypeMapping typeMapping = TypeMapping.defaultMapping(); @@ -80,7 +80,7 @@ public SynchronizationActionBase( this.database = database; this.cdcSourceConfig = Configuration.fromMap(cdcSourceConfig); this.syncJobHandler = syncJobHandler; - this.allowUpperCase = catalog.allowUpperCase(); + this.caseSensitive = catalog.caseSensitive(); this.syncJobHandler.registerJdbcDriver(); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java index 15fc3507ce2d..7dd63ed2273e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java @@ -24,6 +24,8 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + /** Used to convert a MySQL source table name to corresponding Paimon table name. */ public class TableNameConverter implements Serializable { @@ -78,7 +80,7 @@ public String convert(String originDbName, String originTblName) { // top priority: table mapping if (tableMapping.containsKey(originTblName.toLowerCase())) { String mappedName = tableMapping.get(originTblName.toLowerCase()); - return caseSensitive ? mappedName : mappedName.toLowerCase(); + return toLowerCaseIfNeed(mappedName, caseSensitive); } String tblPrefix = prefix; @@ -93,7 +95,7 @@ public String convert(String originDbName, String originTblName) { } // third priority: normal prefix and suffix - String tableName = caseSensitive ? originTblName : originTblName.toLowerCase(); + String tableName = toLowerCaseIfNeed(originTblName, caseSensitive); return tblPrefix + tableName + tblSuffix; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index ce2e9124a664..0f452e2834be 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -139,7 +139,7 @@ protected void beforeBuildingSourceSink() throws Exception { TableNameConverter tableNameConverter = new TableNameConverter( - allowUpperCase, mergeShards, tablePrefix, tableSuffix, tableMapping); + caseSensitive, mergeShards, tablePrefix, tableSuffix, tableMapping); for (JdbcTableInfo tableInfo : jdbcTableInfos) { Identifier identifier = Identifier.create( @@ -155,7 +155,7 @@ protected void beforeBuildingSourceSink() throws Exception { tableConfig, tableInfo.schema(), metadataConverters, - allowUpperCase, + caseSensitive, false, true); try { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java index 4892aee03024..e80692ed22f5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -28,8 +29,8 @@ public class CaseSensitiveUtils { public static DataStream cdcRecordConvert( - Catalog.Loader catalogLoader, DataStream input) { - if (allowUpperCase(catalogLoader)) { + CatalogLoader catalogLoader, DataStream input) { + if (caseSensitive(catalogLoader)) { return input; } @@ -46,8 +47,8 @@ public void processElement( } public static DataStream cdcMultiplexRecordConvert( - Catalog.Loader catalogLoader, DataStream input) { - if (allowUpperCase(catalogLoader)) { + CatalogLoader catalogLoader, DataStream input) { + if (caseSensitive(catalogLoader)) { return input; } @@ -65,9 +66,9 @@ public void processElement( .name("Case-insensitive Convert"); } - private static boolean allowUpperCase(Catalog.Loader catalogLoader) { + private static boolean caseSensitive(CatalogLoader catalogLoader) { try (Catalog catalog = catalogLoader.load()) { - return catalog.allowUpperCase(); + return catalog.caseSensitive(); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java index 886e33e2046a..4efcf1207e9f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.types.DataField; @@ -62,13 +63,13 @@ public class CdcDynamicTableParsingProcessFunction extends ProcessFunction parserFactory; private final String database; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private transient EventParser parser; private transient Catalog catalog; public CdcDynamicTableParsingProcessFunction( - String database, Catalog.Loader catalogLoader, EventParser.Factory parserFactory) { + String database, CatalogLoader catalogLoader, EventParser.Factory parserFactory) { // for now, only support single database this.database = database; this.catalogLoader = catalogLoader; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java index fdad3a921d63..2858b2d4eb6b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -38,13 +39,13 @@ public class CdcMultiplexRecordChannelComputer implements ChannelComputer channelComputers; - public CdcMultiplexRecordChannelComputer(Catalog.Loader catalogLoader) { + public CdcMultiplexRecordChannelComputer(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java index 5db111a30047..9387a8293874 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.GenericRow; import org.apache.paimon.flink.sink.MultiTableCommittable; @@ -67,7 +68,7 @@ public class CdcRecordStoreMultiWriteOperator private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; private final String initialCommitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private MemoryPoolFactory memoryPoolFactory; private Catalog catalog; @@ -79,7 +80,7 @@ public class CdcRecordStoreMultiWriteOperator private CdcRecordStoreMultiWriteOperator( StreamOperatorParameters parameters, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, String initialCommitUser, Options options) { @@ -264,10 +265,10 @@ public static class Factory extends PrepareCommitOperator.Factory { private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; private final String initialCommitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; public Factory( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, String initialCommitUser, Options options) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java index 28b68fedc3e6..5c27db6ddf1b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.annotation.Experimental; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.utils.SingleOutputStreamOperatorUtils; import org.apache.paimon.schema.SchemaManager; @@ -48,7 +48,7 @@ public class CdcSinkBuilder { private EventParser.Factory parserFactory = null; private Table table = null; private Identifier identifier = null; - private Catalog.Loader catalogLoader = null; + private CatalogLoader catalogLoader = null; @Nullable private Integer parallelism; @@ -77,7 +77,7 @@ public CdcSinkBuilder withIdentifier(Identifier identifier) { return this; } - public CdcSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) { + public CdcSinkBuilder withCatalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index 1688d4deb088..4cd9235cb58a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.sink.cdc; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.Committer; import org.apache.paimon.flink.sink.CommitterOperatorFactory; @@ -60,13 +60,13 @@ public class FlinkCdcMultiTableSink implements Serializable { private static final String GLOBAL_COMMITTER_NAME = "Multiplex Global Committer"; private final boolean isOverwrite = false; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final double commitCpuCores; @Nullable private final MemorySize commitHeapMemory; private final String commitUser; public FlinkCdcMultiTableSink( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, double commitCpuCores, @Nullable MemorySize commitHeapMemory, String commitUser) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java index a9ad66847b4b..bd18c7e7ad82 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.sink.cdc; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.action.MultiTablesSinkMode; @@ -72,7 +72,7 @@ public class FlinkCdcSyncDatabaseSinkBuilder { // it will check newly added tables and create the corresponding // Paimon tables. 2) in multiplex sink where it is used to // initialize different writers to multiple tables. - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; // database to sync, currently only support single database private String database; private MultiTablesSinkMode mode; @@ -111,7 +111,7 @@ public FlinkCdcSyncDatabaseSinkBuilder withDatabase(String database) { return this; } - public FlinkCdcSyncDatabaseSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) { + public FlinkCdcSyncDatabaseSinkBuilder withCatalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java index 0ad412e47d34..dd612a52c2eb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -51,7 +52,7 @@ public class MultiTableUpdatedDataFieldsProcessFunction private final Map schemaManagers = new HashMap<>(); - public MultiTableUpdatedDataFieldsProcessFunction(Catalog.Loader catalogLoader) { + public MultiTableUpdatedDataFieldsProcessFunction(CatalogLoader catalogLoader) { super(catalogLoader); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java index 610856d3af54..43f63854bb22 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.annotation.Public; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.Table; @@ -39,7 +39,7 @@ public class RichCdcSinkBuilder { private DataStream input = null; private Table table = null; private Identifier identifier = null; - private Catalog.Loader catalogLoader = null; + private CatalogLoader catalogLoader = null; @Nullable private Integer parallelism; @@ -62,7 +62,7 @@ public RichCdcSinkBuilder parallelism(@Nullable Integer parallelism) { return this; } - public RichCdcSinkBuilder catalogLoader(Catalog.Loader catalogLoader) { + public RichCdcSinkBuilder catalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } @@ -114,7 +114,7 @@ public RichCdcSinkBuilder withIdentifier(Identifier identifier) { /** @deprecated Use {@link #catalogLoader}. */ @Deprecated - public RichCdcSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) { + public RichCdcSinkBuilder withCatalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java index 64f00d96b0f5..504f63105801 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.sink.cdc; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -53,7 +53,7 @@ public class UpdatedDataFieldsProcessFunction private Set latestFields; public UpdatedDataFieldsProcessFunction( - SchemaManager schemaManager, Identifier identifier, Catalog.Loader catalogLoader) { + SchemaManager schemaManager, Identifier identifier, CatalogLoader catalogLoader) { super(catalogLoader); this.schemaManager = schemaManager; this.identifier = identifier; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index 4f02b784c2ba..d50df23742aa 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -48,9 +49,9 @@ public abstract class UpdatedDataFieldsProcessFunctionBase extends Process private static final Logger LOG = LoggerFactory.getLogger(UpdatedDataFieldsProcessFunctionBase.class); - protected final Catalog.Loader catalogLoader; + protected final CatalogLoader catalogLoader; protected Catalog catalog; - private boolean allowUpperCase; + private boolean caseSensitive; private static final List STRING_TYPES = Arrays.asList(DataTypeRoot.CHAR, DataTypeRoot.VARCHAR); @@ -70,7 +71,7 @@ public abstract class UpdatedDataFieldsProcessFunctionBase extends Process private static final List TIMESTAMP_TYPES = Arrays.asList(DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); - protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader) { + protected UpdatedDataFieldsProcessFunctionBase(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; } @@ -86,7 +87,7 @@ public void open(OpenContext openContext) throws Exception { */ public void open(Configuration parameters) { this.catalog = catalogLoader.load(); - this.allowUpperCase = this.catalog.allowUpperCase(); + this.caseSensitive = this.catalog.caseSensitive(); } protected void applySchemaChange( @@ -215,8 +216,7 @@ protected List extractSchemaChanges( List result = new ArrayList<>(); for (DataField newField : updatedDataFields) { - String newFieldName = - StringUtils.caseSensitiveConversion(newField.name(), allowUpperCase); + String newFieldName = StringUtils.toLowerCaseIfNeed(newField.name(), caseSensitive); if (oldFields.containsKey(newFieldName)) { DataField oldField = oldFields.get(newFieldName); // we compare by ignoring nullable, because partition keys and primary keys might be diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java index 9ba18376867f..46c8e98fb639 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.action.cdc; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.sink.cdc.UpdatedDataFieldsProcessFunction; @@ -202,7 +202,7 @@ public void testSchemaEvolution() throws Exception { DataStream> upDataFieldStream = env.fromCollection(prepareData()); Options options = new Options(); options.set("warehouse", tempPath.toString()); - final Catalog.Loader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(options); + final CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(options); Identifier identifier = Identifier.create(database, tableName); DataStream schemaChangeProcessFunction = upDataFieldStream diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java index 0e1f4e72ea8c..6e37c589ac92 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -535,7 +535,7 @@ public void testCaseInsensitive() throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index 8a4dc2f3035b..ed1885f5d774 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.action.cdc.kafka; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -1121,7 +1121,7 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withComputedColumnArgs("_YEAR=year(_DATE)") .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index de189bc20536..606c46e90e4a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -475,7 +475,7 @@ protected void testCaseInsensitive(String format) throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java index ae0b0b412ab4..92c2a7243a7c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.mongodb; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -196,7 +196,7 @@ public void testDynamicTableCreationInMongoDB() throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java index b4f31f2d6d3d..2d8489dc23df 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.mongodb; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -382,7 +382,7 @@ public void testComputedColumnWithCaseInsensitive() throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withComputedColumnArgs("_YEAR=year(_DATE)") .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java index b48b898d66e3..10ee548125bc 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.mysql; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.FileSystemCatalogOptions; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.MultiTablesSinkMode; import org.apache.paimon.options.CatalogOptions; @@ -475,7 +474,7 @@ public void testIgnoreCaseDivided() throws Exception { syncDatabaseActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withTableConfig(getBasicTableConfig()) .build(); runActionWithDefaultEnv(action); @@ -496,7 +495,7 @@ public void testIgnoreCaseCombined() throws Exception { syncDatabaseActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withMode(COMBINED.configString()) .withTableConfig(getBasicTableConfig()) .build(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index febbe4e1deaa..749d87eb0636 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.mysql; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.FileSystemCatalogOptions; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -1327,7 +1326,7 @@ public void testComputedColumnWithCaseInsensitive() throws Exception { syncTableActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withComputedColumnArgs("SUBSTRING=substring(UPPERCASE_STRING,2)") .build(); runActionWithDefaultEnv(action); @@ -1363,7 +1362,7 @@ public void testSpecifyKeysWithCaseInsensitive() throws Exception { syncTableActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withPrimaryKeys("ID1", "PART") .withPartitionKeys("PART") .build(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java index 867cbdbae002..43b7d2ba6399 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.Path; import org.apache.paimon.options.CatalogOptions; @@ -54,7 +55,7 @@ public class CdcMultiplexRecordChannelComputerTest { @TempDir java.nio.file.Path tempDir; - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; private Path warehouse; private String databaseName; private Identifier tableWithPartition; diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java index 9f35b25026bb..4436aa392d42 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.sink.MultiTableCommittable; import org.apache.paimon.flink.sink.MultiTableCommittableTypeInfo; @@ -82,7 +83,7 @@ public class CdcRecordStoreMultiWriteOperatorTest { private Identifier firstTable; private Catalog catalog; private Identifier secondTable; - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; private Schema firstTableSchema; @BeforeEach @@ -340,7 +341,7 @@ public void testSingleTableAddColumn() throws Exception { harness.close(); } - private Catalog.Loader createCatalogLoader() { + private CatalogLoader createCatalogLoader() { Options catalogOptions = createCatalogOptions(warehouse); return () -> CatalogFactory.createCatalog(CatalogContext.create(catalogOptions)); } @@ -688,7 +689,7 @@ public void testUsingTheSameCompactExecutor() throws Exception { } private OneInputStreamOperatorTestHarness - createTestHarness(Catalog.Loader catalogLoader) throws Exception { + createTestHarness(CatalogLoader catalogLoader) throws Exception { CdcRecordStoreMultiWriteOperator.Factory operatorFactory = new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index 28b137a93ed9..35286e3a88d4 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.FlinkCatalogFactory; @@ -162,8 +162,7 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB Options catalogOptions = new Options(); catalogOptions.set("warehouse", tempDir.toString()); - Catalog.Loader catalogLoader = - () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); + CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); new FlinkCdcSyncDatabaseSinkBuilder() .withInput(source) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index 8b19391f3eda..9fccaac99228 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.InternalRow; @@ -159,8 +159,7 @@ private void innerTestRandomCdcEvents( Options catalogOptions = new Options(); catalogOptions.set("warehouse", tempDir.toString()); - Catalog.Loader catalogLoader = - () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); + CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); new CdcSinkBuilder() .withInput(source) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index dd95c48af8d1..3407735b4b79 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TableType; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; @@ -519,7 +520,7 @@ protected Schema buildPaimonSchema( // Although catalog.createTable will copy the default options, but we need this info // here before create table, such as table-default.kafka.bootstrap.servers defined in // catalog options. Temporarily, we copy the default options here. - Catalog.tableDefaultOptions(catalog.options()).forEach(options::putIfAbsent); + CatalogUtils.tableDefaultOptions(catalog.options()).forEach(options::putIfAbsent); options.put(REGISTER_TIMEOUT.key(), logStoreAutoRegisterTimeout.toString()); registerLogSystem(catalog, identifier, options, classLoader); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java index 30e32d62efec..4490023e7b03 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.FlinkCatalog; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.LogicalTypeConversion; @@ -99,7 +100,7 @@ protected void execute(String defaultName) throws Exception { env.execute(name); } - protected Catalog.Loader catalogLoader() { + protected CatalogLoader catalogLoader() { // to make the action workflow serializable Options catalogOptions = this.catalogOptions; return () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java index 88730132ef68..e2fd5a9d318e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.compact; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -47,7 +47,7 @@ public class MultiAwareBucketTableScan extends MultiTableScanBase scansMap; public MultiAwareBucketTableScan( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java index f5940740b691..805e8da0a417 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java @@ -20,6 +20,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; @@ -59,7 +60,7 @@ public abstract class MultiTableScanBase implements AutoCloseable { protected boolean isStreaming; public MultiTableScanBase( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java index da86b93af512..2ad2642b6248 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java @@ -20,7 +20,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -41,7 +41,7 @@ public class MultiUnawareBucketTableScan protected transient Map tablesMap; public MultiUnawareBucketTableScan( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java index 83d51f302e51..07ec7d165e3a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.compact.UnawareBucketCompactor; import org.apache.paimon.options.Options; @@ -56,7 +57,7 @@ public class AppendOnlyMultiTableCompactionWorkerOperator LoggerFactory.getLogger(AppendOnlyMultiTableCompactionWorkerOperator.class); private final String commitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; // support multi table compaction private transient Map compactorContainer; @@ -67,7 +68,7 @@ public class AppendOnlyMultiTableCompactionWorkerOperator private AppendOnlyMultiTableCompactionWorkerOperator( StreamOperatorParameters parameters, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, String commitUser, Options options) { super(parameters, options); @@ -188,9 +189,9 @@ public static class Factory MultiTableUnawareAppendCompactionTask, MultiTableCommittable> { private final String commitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; - public Factory(Catalog.Loader catalogLoader, String commitUser, Options options) { + public Factory(CatalogLoader catalogLoader, String commitUser, Options options) { super(options); this.commitUser = commitUser; this.catalogLoader = catalogLoader; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index 25f76ce97683..53f1bf165d98 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -20,7 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.manifest.WrappedManifestCommittable; import org.apache.paimon.options.Options; @@ -55,14 +55,14 @@ public class CombinedTableCompactorSink implements Serializable { private static final String WRITER_NAME = "Writer"; private static final String GLOBAL_COMMITTER_NAME = "Global Committer"; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final boolean ignorePreviousFiles; private final boolean fullCompaction; private final Options options; public CombinedTableCompactorSink( - Catalog.Loader catalogLoader, Options options, boolean fullCompaction) { + CatalogLoader catalogLoader, Options options, boolean fullCompaction) { this.catalogLoader = catalogLoader; this.ignorePreviousFiles = false; this.fullCompaction = fullCompaction; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 58f6a3834096..02a7e6c1b3c8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.flink.utils.RuntimeContextUtils; @@ -72,7 +73,7 @@ public class MultiTablesStoreCompactOperator private transient StoreSinkWriteState state; private transient DataFileMetaSerializer dataFileMetaSerializer; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; protected Catalog catalog; protected Map tables; @@ -81,7 +82,7 @@ public class MultiTablesStoreCompactOperator private MultiTablesStoreCompactOperator( StreamOperatorParameters parameters, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, String initialCommitUser, CheckpointConfig checkpointConfig, boolean isStreaming, @@ -324,7 +325,7 @@ private StoreSinkWrite.Provider createWriteProvider( /** {@link StreamOperatorFactory} of {@link MultiTablesStoreCompactOperator}. */ public static class Factory extends PrepareCommitOperator.Factory { - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final CheckpointConfig checkpointConfig; private final boolean isStreaming; private final boolean ignorePreviousFiles; @@ -332,7 +333,7 @@ public static class Factory private final String initialCommitUser; public Factory( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, String initialCommitUser, CheckpointConfig checkpointConfig, boolean isStreaming, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java index 537a98f97fb0..01acddb9ad99 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.manifest.WrappedManifestCommittable; @@ -56,12 +57,12 @@ public class StoreMultiCommitter private final boolean ignoreEmptyCommit; private final Map dynamicOptions; - public StoreMultiCommitter(Catalog.Loader catalogLoader, Context context) { + public StoreMultiCommitter(CatalogLoader catalogLoader, Context context) { this(catalogLoader, context, false, Collections.emptyMap()); } public StoreMultiCommitter( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Context context, boolean ignoreEmptyCommit, Map dynamicOptions) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java index dbdf77601480..d190b9ccf39e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java @@ -58,8 +58,11 @@ public static PartitionListeners create(Committer.Context context, FileStoreTabl throws Exception { List listeners = new ArrayList<>(); - ReportHmsListener.create(context.isRestored(), context.stateStore(), table) + // partition statistics reporter + ReportPartStatsListener.create(context.isRestored(), context.stateStore(), table) .ifPresent(listeners::add); + + // partition mark done PartitionMarkDone.create( context.streamingCheckpointEnabled(), context.isRestored(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java similarity index 85% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java index 853dc52c20bf..b75889d567ee 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java @@ -40,22 +40,24 @@ import java.util.List; import java.util.Map; -import static org.apache.paimon.catalog.Catalog.HIVE_LAST_UPDATE_TIME_PROP; +import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; import static org.apache.paimon.utils.PartitionPathUtils.extractPartitionSpecFromPath; /** Action to report the table statistic from the latest snapshot to HMS. */ -public class HmsReporter implements Closeable { +public class PartitionStatisticsReporter implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(HmsReporter.class); + private static final Logger LOG = LoggerFactory.getLogger(PartitionStatisticsReporter.class); + + private static final String HIVE_LAST_UPDATE_TIME_PROP = "transient_lastDdlTime"; private final MetastoreClient metastoreClient; private final SnapshotReader snapshotReader; private final SnapshotManager snapshotManager; - public HmsReporter(FileStoreTable table, MetastoreClient client) { + public PartitionStatisticsReporter(FileStoreTable table, MetastoreClient client) { this.metastoreClient = Preconditions.checkNotNull(client, "the metastore client factory is null"); this.snapshotReader = table.newSnapshotReader(); @@ -90,7 +92,12 @@ public void report(String partition, long modifyTime) throws Exception { statistic.put(NUM_FILES_PROP, String.valueOf(fileCount)); statistic.put(TOTAL_SIZE_PROP, String.valueOf(totalSize)); statistic.put(NUM_ROWS_PROP, String.valueOf(rowCount)); - statistic.put(HIVE_LAST_UPDATE_TIME_PROP, String.valueOf(modifyTime / 1000)); + + String modifyTimeSeconds = String.valueOf(modifyTime / 1000); + statistic.put(LAST_UPDATE_TIME_PROP, modifyTimeSeconds); + + // just for being compatible with hive metastore + statistic.put(HIVE_LAST_UPDATE_TIME_PROP, modifyTimeSeconds); LOG.info("alter partition {} with statistic {}.", partitionSpec, statistic); metastoreClient.alterPartition(partitionSpec, statistic, modifyTime, true); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportPartStatsListener.java similarity index 91% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportPartStatsListener.java index 842dd012e88e..ca51c3df5b1a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportPartStatsListener.java @@ -49,7 +49,7 @@ * This listener will collect data from the newly touched partition and then decide when to trigger * a report based on the partition's idle time. */ -public class ReportHmsListener implements PartitionListener { +public class ReportPartStatsListener implements PartitionListener { @SuppressWarnings("unchecked") private static final ListStateDescriptor> PENDING_REPORT_STATE_DESC = @@ -58,20 +58,20 @@ public class ReportHmsListener implements PartitionListener { new MapSerializer<>(StringSerializer.INSTANCE, LongSerializer.INSTANCE)); private final InternalRowPartitionComputer partitionComputer; - private final HmsReporter hmsReporter; + private final PartitionStatisticsReporter partitionStatisticsReporter; private final ListState> pendingPartitionsState; private final Map pendingPartitions; private final long idleTime; - private ReportHmsListener( + private ReportPartStatsListener( InternalRowPartitionComputer partitionComputer, - HmsReporter hmsReporter, + PartitionStatisticsReporter partitionStatisticsReporter, OperatorStateStore store, boolean isRestored, long idleTime) throws Exception { this.partitionComputer = partitionComputer; - this.hmsReporter = hmsReporter; + this.partitionStatisticsReporter = partitionStatisticsReporter; this.pendingPartitionsState = store.getListState(PENDING_REPORT_STATE_DESC); this.pendingPartitions = new HashMap<>(); if (isRestored) { @@ -108,7 +108,7 @@ public void notifyCommittable(List committables) { try { Map partitions = reportPartition(endInput); for (Map.Entry entry : partitions.entrySet()) { - hmsReporter.report(entry.getKey(), entry.getValue()); + partitionStatisticsReporter.report(entry.getKey(), entry.getValue()); } } catch (Exception e) { throw new RuntimeException(e); @@ -138,7 +138,7 @@ public void snapshotState() throws Exception { pendingPartitionsState.update(Collections.singletonList(pendingPartitions)); } - public static Optional create( + public static Optional create( boolean isRestored, OperatorStateStore stateStore, FileStoreTable table) throws Exception { @@ -169,9 +169,9 @@ public static Optional create( coreOptions.legacyPartitionName()); return Optional.of( - new ReportHmsListener( + new ReportPartStatsListener( partitionComputer, - new HmsReporter( + new PartitionStatisticsReporter( table, table.catalogEnvironment().metastoreClientFactory().create()), stateStore, @@ -182,8 +182,8 @@ public static Optional create( @Override public void close() throws IOException { - if (hmsReporter != null) { - hmsReporter.close(); + if (partitionStatisticsReporter != null) { + partitionStatisticsReporter.close(); } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java index 415eddb037df..ac6af6a14f6a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.source; import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.LogicalTypeConversion; import org.apache.paimon.flink.source.operator.CombinedAwareBatchSource; import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSource; @@ -44,7 +44,8 @@ * compactor jobs in combined mode. */ public class CombinedTableCompactorSourceBuilder { - private final Catalog.Loader catalogLoader; + + private final CatalogLoader catalogLoader; private final Pattern includingPattern; private final Pattern excludingPattern; private final Pattern databasePattern; @@ -55,7 +56,7 @@ public class CombinedTableCompactorSourceBuilder { @Nullable private Duration partitionIdleTime = null; public CombinedTableCompactorSourceBuilder( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern databasePattern, Pattern includingPattern, Pattern excludingPattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java index c3a1258bb176..2f7a82c95184 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.source.operator; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; @@ -54,7 +54,7 @@ public class CombinedAwareBatchSource extends CombinedCompactorSource buildSource( StreamExecutionEnvironment env, String name, TypeInformation typeInfo, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java index 9bd4a84f571c..a23a3b41a441 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.source.operator; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; @@ -51,7 +51,7 @@ public class CombinedAwareStreamingSource extends CombinedCompactorSource buildSource( StreamExecutionEnvironment env, String name, TypeInformation typeInfo, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java index f58d86cdd65e..e292d2441ccd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.source.operator; import org.apache.paimon.append.UnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; import org.apache.paimon.table.source.Split; @@ -44,16 +44,17 @@ * single (non-parallel) monitoring task, it is responsible for the new Paimon table. */ public abstract class CombinedCompactorSource extends AbstractNonCoordinatedSource { - private static final long serialVersionUID = 2L; - protected final Catalog.Loader catalogLoader; + private static final long serialVersionUID = 3L; + + protected final CatalogLoader catalogLoader; protected final Pattern includingPattern; protected final Pattern excludingPattern; protected final Pattern databasePattern; protected final boolean isStreaming; public CombinedCompactorSource( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java index 64f0c38f5a11..5c0d9c42dd29 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java @@ -20,6 +20,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.flink.compact.MultiTableScanBase; @@ -63,7 +64,7 @@ public class CombinedUnawareBatchSource private static final Logger LOGGER = LoggerFactory.getLogger(CombinedUnawareBatchSource.class); public CombinedUnawareBatchSource( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern) { @@ -121,7 +122,7 @@ public void close() throws Exception { public static DataStream buildSource( StreamExecutionEnvironment env, String name, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java index 6ea1ead4db30..2e38d538a999 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.source.operator; import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; @@ -48,7 +48,7 @@ public class CombinedUnawareStreamingSource private final long monitorInterval; public CombinedUnawareStreamingSource( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, @@ -104,7 +104,7 @@ public void close() throws Exception { public static DataStream buildSource( StreamExecutionEnvironment env, String name, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java index fbc8bb9d756a..ae3099ec0628 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.source.operator; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; @@ -60,18 +61,18 @@ public class MultiTablesReadOperator extends AbstractStreamOperator private static final long serialVersionUID = 1L; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final boolean isStreaming; private Duration partitionIdleTime = null; - public MultiTablesReadOperator(Catalog.Loader catalogLoader, boolean isStreaming) { + public MultiTablesReadOperator(CatalogLoader catalogLoader, boolean isStreaming) { this.catalogLoader = catalogLoader; this.isStreaming = isStreaming; } public MultiTablesReadOperator( - Catalog.Loader catalogLoader, boolean isStreaming, Duration partitionIdleTime) { + CatalogLoader catalogLoader, boolean isStreaming, Duration partitionIdleTime) { this.catalogLoader = catalogLoader; this.isStreaming = isStreaming; this.partitionIdleTime = partitionIdleTime; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java index 0864741a178f..15fde93755fd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.manifest.PartitionEntry; @@ -54,12 +55,11 @@ public class MultiUnawareTablesReadOperator private static final Logger LOG = LoggerFactory.getLogger(MultiUnawareTablesReadOperator.class); - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final Duration partitionIdleTime; - public MultiUnawareTablesReadOperator( - Catalog.Loader catalogLoader, Duration partitionIdleTime) { + public MultiUnawareTablesReadOperator(CatalogLoader catalogLoader, Duration partitionIdleTime) { this.catalogLoader = catalogLoader; this.partitionIdleTime = partitionIdleTime; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index d487d75925eb..0e85f559d9de 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; @@ -273,7 +274,7 @@ protected StoreCompactOperator.Factory createCompactOperator(FileStoreTable tabl } protected MultiTablesStoreCompactOperator.Factory createMultiTablesCompactOperator( - Catalog.Loader catalogLoader) throws Exception { + CatalogLoader catalogLoader) throws Exception { return new MultiTablesStoreCompactOperator.Factory( catalogLoader, commitUser, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java index 752679fb5903..53e3a6dcb79c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryString; @@ -78,7 +79,7 @@ class StoreMultiCommitterTest { private String initialCommitUser; private Path warehouse; - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; private Catalog catalog; private Identifier firstTable; private Identifier secondTable; @@ -691,7 +692,7 @@ public void snapshotState( return harness; } - private Catalog.Loader createCatalogLoader() { + private CatalogLoader createCatalogLoader() { Options catalogOptions = createCatalogOptions(warehouse); return () -> CatalogFactory.createCatalog(CatalogContext.create(catalogOptions)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java similarity index 95% rename from paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java rename to paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java index f245940da57d..142a0c32f781 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java @@ -49,8 +49,8 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; -/** Test for {@link HmsReporter}. */ -public class HmsReporterTest { +/** Test for {@link PartitionStatisticsReporter}. */ +public class PartitionStatisticsReporterTest { @TempDir java.nio.file.Path tempDir; @@ -131,7 +131,7 @@ public void close() throws Exception { } }; - HmsReporter action = new HmsReporter(table, client); + PartitionStatisticsReporter action = new PartitionStatisticsReporter(table, client); long time = 1729598544974L; action.report("c1=a/", time); Assertions.assertThat(partitionParams).containsKey("c1=a/"); @@ -144,6 +144,8 @@ public void close() throws Exception { "591", "numRows", "1", + "lastUpdateTime", + String.valueOf(time / 1000), "transient_lastDdlTime", String.valueOf(time / 1000))); action.close(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java index fba5f33807de..3b41f39431bd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; @@ -662,7 +663,7 @@ private BinaryRow binaryRow(String dt, int hh) { return b; } - private Catalog.Loader catalogLoader() { + private CatalogLoader catalogLoader() { // to make the action workflow serializable catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse); return () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index c74ede981546..5744ac894d12 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -103,7 +103,7 @@ import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogOptions.LOCATION_IN_PROPERTIES; import static org.apache.paimon.hive.HiveTableUtils.convertToFormatTable; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.options.CatalogOptions.FORMAT_TABLE_ENABLED; import static org.apache.paimon.options.CatalogOptions.SYNC_ALL_PROPERTIES; import static org.apache.paimon.options.CatalogOptions.TABLE_TYPE; @@ -872,8 +872,8 @@ private void alterTableToHms(Table table, Identifier identifier, TableSchema new } @Override - public boolean allowUpperCase() { - return catalogOptions.getOptional(ALLOW_UPPER_CASE).orElse(false); + public boolean caseSensitive() { + return catalogOptions.getOptional(CASE_SENSITIVE).orElse(false); } @Override @@ -931,7 +931,6 @@ public void repairDatabase(String databaseName) { public void repairTable(Identifier identifier) throws TableNotExistException { checkNotBranch(identifier, "repairTable"); checkNotSystemTable(identifier, "repairTable"); - validateIdentifierNameCaseInsensitive(identifier); Path location = getTableLocation(identifier); TableSchema tableSchema = diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index d6318c723fe0..de6e2414fc8f 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -69,7 +69,7 @@ import static org.apache.paimon.CoreOptions.FILE_FORMAT; import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.TableType.FORMAT_TABLE; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE; import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf; @@ -96,9 +96,9 @@ public void initialize(String name, CaseInsensitiveStringMap options) { CatalogContext catalogContext = CatalogContext.create(Options.fromMap(options), sessionState.newHadoopConf()); - // if spark is case-insensitive, set allow upper case to catalog + // if spark is case-insensitive, set case-sensitive to catalog if (!sessionState.conf().caseSensitiveAnalysis()) { - newOptions.put(ALLOW_UPPER_CASE.key(), "true"); + newOptions.put(CASE_SENSITIVE.key(), "true"); } options = new CaseInsensitiveStringMap(newOptions); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index 6b7b17b1b1a5..b57228fa44f0 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -62,7 +62,7 @@ import java.util.Map; import java.util.concurrent.Callable; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.options.CatalogOptions.METASTORE; import static org.apache.paimon.options.CatalogOptions.WAREHOUSE; import static org.apache.paimon.spark.SparkCatalogOptions.CREATE_UNDERLYING_SESSION_CATALOG; @@ -331,9 +331,9 @@ private void fillCommonConfigurations(Map options, SQLConf sqlCo options.put(DEFAULT_DATABASE.key(), sessionCatalogDefaultDatabase); } - // if spark is case-insensitive, set allow upper case to catalog + // if spark is case-insensitive, set case-sensitive to catalog if (!sqlConf.caseSensitiveAnalysis()) { - options.put(ALLOW_UPPER_CASE.key(), "true"); + options.put(CASE_SENSITIVE.key(), "true"); } } From 76a2f46a0ac0649da4fefcfe7519e004557a0e08 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 17 Dec 2024 04:35:37 +0100 Subject: [PATCH 05/17] [pom] prefer central repo for releases; limit apache-snapshots to snapshots (#4707) --- pom.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/pom.xml b/pom.xml index dbef98af06b2..baa8184e2004 100644 --- a/pom.xml +++ b/pom.xml @@ -1029,15 +1029,28 @@ under the License. From 659cf3995752432aa939569ee244e548f70ec77e Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 17 Dec 2024 13:54:31 +0800 Subject: [PATCH 06/17] [core] Refactor MetastoreClient methods to simplify catalog (#4726) --- .../paimon/catalog/AbstractCatalog.java | 5 +- .../metastore/AddPartitionCommitCallback.java | 11 +- .../metastore/AddPartitionTagCallback.java | 2 +- .../paimon/metastore/MetastoreClient.java | 29 +---- .../paimon/metastore/PartitionStats.java | 64 ++++++++++ .../paimon/operation/PartitionExpire.java | 16 ++- .../actions/MarkPartitionDoneEventAction.java | 2 +- .../paimon/table/AbstractFileStoreTable.java | 11 +- .../paimon/operation/PartitionExpireTest.java | 65 +++++++++- .../PartitionStatisticsReporter.java | 27 +--- .../partition/AddDonePartitionActionTest.java | 30 ++--- .../PartitionStatisticsReporterTest.java | 45 +++---- .../org/apache/paimon/hive/HiveCatalog.java | 31 ++--- .../paimon/hive/HiveMetastoreClient.java | 118 ++++++++---------- .../spark/PaimonPartitionManagement.scala | 2 +- 15 files changed, 267 insertions(+), 191 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/metastore/PartitionStats.java diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index db6909295556..3fdefe6cacaa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -417,7 +417,7 @@ protected Table getDataOrFormatTable(Identifier identifier) throws TableNotExist lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableMeta.schema).orElse(null))); + metastoreClientFactory(identifier).orElse(null))); CoreOptions options = table.coreOptions(); if (options.type() == TableType.OBJECT_TABLE) { String objectLocation = options.objectLocation(); @@ -485,8 +485,7 @@ protected abstract TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException; /** Get metastore client factory for the table specified by {@code identifier}. */ - public Optional metastoreClientFactory( - Identifier identifier, TableSchema schema) { + public Optional metastoreClientFactory(Identifier identifier) { return Optional.empty(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java index 599f88e512c0..26fb9ed48db2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java @@ -25,6 +25,7 @@ import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.table.sink.CommitCallback; import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.shade.guava30.com.google.common.cache.Cache; import org.apache.paimon.shade.guava30.com.google.common.cache.CacheBuilder; @@ -48,9 +49,12 @@ public class AddPartitionCommitCallback implements CommitCallback { .build(); private final MetastoreClient client; + private final InternalRowPartitionComputer partitionComputer; - public AddPartitionCommitCallback(MetastoreClient client) { + public AddPartitionCommitCallback( + MetastoreClient client, InternalRowPartitionComputer partitionComputer) { this.client = client; + this.partitionComputer = partitionComputer; } @Override @@ -81,7 +85,10 @@ private void addPartitions(Set partitions) { } } if (!newPartitions.isEmpty()) { - client.addPartitions(newPartitions); + client.addPartitions( + newPartitions.stream() + .map(partitionComputer::generatePartValues) + .collect(Collectors.toList())); newPartitions.forEach(partition -> cache.put(partition, true)); } } catch (Exception e) { diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java index 70efe68e83f3..31bb521e88d1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java @@ -49,7 +49,7 @@ public void notifyDeletion(String tagName) { LinkedHashMap partitionSpec = new LinkedHashMap<>(); partitionSpec.put(partitionField, tagName); try { - client.deletePartition(partitionSpec); + client.dropPartition(partitionSpec); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java index 75f7af5abbdc..ccf5f3853873 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java @@ -18,12 +18,9 @@ package org.apache.paimon.metastore; -import org.apache.paimon.data.BinaryRow; - import java.io.Serializable; import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; /** * A metastore client related to a table. All methods of this interface operate on the same specific @@ -31,32 +28,18 @@ */ public interface MetastoreClient extends AutoCloseable { - void addPartition(BinaryRow partition) throws Exception; - - default void addPartitions(List partitions) throws Exception { - for (BinaryRow partition : partitions) { - addPartition(partition); - } - } + void addPartition(LinkedHashMap partition) throws Exception; - void addPartition(LinkedHashMap partitionSpec) throws Exception; + void addPartitions(List> partitions) throws Exception; - default void addPartitionsSpec(List> partitionSpecsList) - throws Exception { - for (LinkedHashMap partitionSpecs : partitionSpecsList) { - addPartition(partitionSpecs); - } - } + void dropPartition(LinkedHashMap partition) throws Exception; - void deletePartition(LinkedHashMap partitionSpec) throws Exception; + void dropPartitions(List> partitions) throws Exception; - void markDone(LinkedHashMap partitionSpec) throws Exception; + void markPartitionDone(LinkedHashMap partition) throws Exception; default void alterPartition( - LinkedHashMap partitionSpec, - Map parameters, - long modifyTime, - boolean ignoreIfNotExist) + LinkedHashMap partition, PartitionStats partitionStats) throws Exception { throw new UnsupportedOperationException(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/PartitionStats.java b/paimon-core/src/main/java/org/apache/paimon/metastore/PartitionStats.java new file mode 100644 index 000000000000..eacc400f52c3 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/PartitionStats.java @@ -0,0 +1,64 @@ +/* + * 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.metastore; + +/** Statistic for partition. */ +public interface PartitionStats { + + long numFiles(); + + long totalSize(); + + long numRows(); + + long lastUpdateTimeMillis(); + + static PartitionStats create( + long numFiles, long totalSize, long numRows, long lastUpdateTimeMillis) { + return new PartitionStats() { + + @Override + public long numFiles() { + return numFiles; + } + + @Override + public long totalSize() { + return totalSize; + } + + @Override + public long numRows() { + return numRows; + } + + @Override + public long lastUpdateTimeMillis() { + return lastUpdateTimeMillis; + } + + @Override + public String toString() { + return String.format( + "numFiles: %s, totalSize: %s, numRows: %s, lastUpdateTimeMillis: %s", + numFiles, totalSize, numRows, lastUpdateTimeMillis); + } + }; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java b/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java index d432a37dfd9c..68ef8a123746 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java @@ -167,15 +167,13 @@ private List> doExpire( } private void deleteMetastorePartitions(List> partitions) { - if (metastoreClient != null) { - partitions.forEach( - partition -> { - try { - metastoreClient.deletePartition(new LinkedHashMap<>(partition)); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); + if (metastoreClient != null && partitions.size() > 0) { + try { + metastoreClient.dropPartitions( + partitions.stream().map(LinkedHashMap::new).collect(Collectors.toList())); + } catch (Exception e) { + throw new RuntimeException(e); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/partition/actions/MarkPartitionDoneEventAction.java b/paimon-core/src/main/java/org/apache/paimon/partition/actions/MarkPartitionDoneEventAction.java index a5ebe34051c1..8cc1c93ba937 100644 --- a/paimon-core/src/main/java/org/apache/paimon/partition/actions/MarkPartitionDoneEventAction.java +++ b/paimon-core/src/main/java/org/apache/paimon/partition/actions/MarkPartitionDoneEventAction.java @@ -39,7 +39,7 @@ public MarkPartitionDoneEventAction(MetastoreClient metastoreClient) { public void markDone(String partition) throws Exception { LinkedHashMap partitionSpec = extractPartitionSpecFromPath(new Path(partition)); - metastoreClient.markDone(partitionSpec); + metastoreClient.markPartitionDone(partitionSpec); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 57966d24ce47..7e008698c4fd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -61,6 +61,7 @@ import org.apache.paimon.table.source.snapshot.TimeTravelUtil; import org.apache.paimon.tag.TagPreview; import org.apache.paimon.utils.BranchManager; +import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SegmentsCache; import org.apache.paimon.utils.SimpleFileReader; @@ -469,7 +470,15 @@ protected List createCommitCallbacks(String commitUser) { if (options.partitionedTableInMetastore() && metastoreClientFactory != null && !tableSchema.partitionKeys().isEmpty()) { - callbacks.add(new AddPartitionCommitCallback(metastoreClientFactory.create())); + InternalRowPartitionComputer partitionComputer = + new InternalRowPartitionComputer( + options.partitionDefaultName(), + tableSchema.logicalPartitionType(), + tableSchema.partitionKeys().toArray(new String[0]), + options.legacyPartitionName()); + callbacks.add( + new AddPartitionCommitCallback( + metastoreClientFactory.create(), partitionComputer)); } TagPreview tagPreview = TagPreview.create(options); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java index 931bac59c756..893fe1bf5762 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java @@ -27,8 +27,12 @@ import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.CatalogEnvironment; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.sink.CommitMessage; @@ -54,6 +58,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -61,9 +66,11 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; +import static org.apache.paimon.CoreOptions.METASTORE_PARTITIONED_TABLE; import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_CHECK_INTERVAL; import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_TIME; import static org.apache.paimon.CoreOptions.PARTITION_TIMESTAMP_FORMATTER; +import static org.apache.paimon.CoreOptions.PATH; import static org.apache.paimon.CoreOptions.WRITE_ONLY; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -75,12 +82,54 @@ public class PartitionExpireTest { private Path path; private FileStoreTable table; + private List> deletedPartitions; @BeforeEach public void beforeEach() { path = new Path(tempDir.toUri()); } + private void newTable() { + LocalFileIO fileIO = LocalFileIO.create(); + Options options = new Options(); + options.set(PATH, path.toString()); + Path tablePath = CoreOptions.path(options); + String branchName = CoreOptions.branch(options.toMap()); + TableSchema tableSchema = new SchemaManager(fileIO, tablePath, branchName).latest().get(); + deletedPartitions = new ArrayList<>(); + MetastoreClient.Factory factory = + () -> + new MetastoreClient() { + @Override + public void addPartition(LinkedHashMap partition) {} + + @Override + public void addPartitions( + List> partitions) {} + + @Override + public void dropPartition(LinkedHashMap partition) { + deletedPartitions.add(partition); + } + + @Override + public void dropPartitions( + List> partitions) { + deletedPartitions.addAll(partitions); + } + + @Override + public void markPartitionDone(LinkedHashMap partition) { + throw new UnsupportedOperationException(); + } + + @Override + public void close() {} + }; + CatalogEnvironment env = new CatalogEnvironment(null, null, Lock.emptyFactory(), factory); + table = FileStoreTableFactory.create(fileIO, path, tableSchema, env); + } + @Test public void testNonPartitionedTable() { SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), path); @@ -108,7 +157,7 @@ public void testIllegalPartition() throws Exception { emptyList(), Collections.emptyMap(), "")); - table = FileStoreTableFactory.create(LocalFileIO.create(), path); + newTable(); write("20230101", "11"); write("abcd", "12"); write("20230101", "12"); @@ -129,9 +178,9 @@ public void test() throws Exception { RowType.of(VarCharType.STRING_TYPE, VarCharType.STRING_TYPE).getFields(), singletonList("f0"), emptyList(), - Collections.emptyMap(), + Collections.singletonMap(METASTORE_PARTITIONED_TABLE.key(), "true"), "")); - table = FileStoreTableFactory.create(LocalFileIO.create(), path); + newTable(); write("20230101", "11"); write("20230101", "12"); @@ -156,6 +205,12 @@ public void test() throws Exception { expire.expire(date(8), Long.MAX_VALUE); assertThat(read()).isEmpty(); + + assertThat(deletedPartitions) + .containsExactlyInAnyOrder( + new LinkedHashMap<>(Collections.singletonMap("f0", "20230101")), + new LinkedHashMap<>(Collections.singletonMap("f0", "20230103")), + new LinkedHashMap<>(Collections.singletonMap("f0", "20230105"))); } @Test @@ -169,7 +224,7 @@ public void testFilterCommittedAfterExpiring() throws Exception { Collections.emptyMap(), "")); - table = FileStoreTableFactory.create(LocalFileIO.create(), path); + newTable(); // disable compaction and snapshot expiration table = table.copy(Collections.singletonMap(WRITE_ONLY.key(), "true")); String commitUser = UUID.randomUUID().toString(); @@ -243,7 +298,7 @@ public void testDeleteExpiredPartition() throws Exception { emptyList(), Collections.emptyMap(), "")); - table = FileStoreTableFactory.create(LocalFileIO.create(), path); + newTable(); table = newExpireTable(); List commitMessages = write("20230101", "11"); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java index b75889d567ee..ced37726f1eb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java @@ -22,6 +22,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.metastore.PartitionStats; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.ScanMode; @@ -35,15 +36,9 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; -import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; -import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; -import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; -import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; import static org.apache.paimon.utils.PartitionPathUtils.extractPartitionSpecFromPath; /** Action to report the table statistic from the latest snapshot to HMS. */ @@ -51,8 +46,6 @@ public class PartitionStatisticsReporter implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(PartitionStatisticsReporter.class); - private static final String HIVE_LAST_UPDATE_TIME_PROP = "transient_lastDdlTime"; - private final MetastoreClient metastoreClient; private final SnapshotReader snapshotReader; private final SnapshotManager snapshotManager; @@ -64,7 +57,7 @@ public PartitionStatisticsReporter(FileStoreTable table, MetastoreClient client) this.snapshotManager = table.snapshotManager(); } - public void report(String partition, long modifyTime) throws Exception { + public void report(String partition, long modifyTimeMillis) throws Exception { Snapshot snapshot = snapshotManager.latestSnapshot(); if (snapshot != null) { LinkedHashMap partitionSpec = @@ -88,19 +81,11 @@ public void report(String partition, long modifyTime) throws Exception { totalSize += fileMeta.fileSize(); } } - Map statistic = new HashMap<>(); - statistic.put(NUM_FILES_PROP, String.valueOf(fileCount)); - statistic.put(TOTAL_SIZE_PROP, String.valueOf(totalSize)); - statistic.put(NUM_ROWS_PROP, String.valueOf(rowCount)); - - String modifyTimeSeconds = String.valueOf(modifyTime / 1000); - statistic.put(LAST_UPDATE_TIME_PROP, modifyTimeSeconds); - - // just for being compatible with hive metastore - statistic.put(HIVE_LAST_UPDATE_TIME_PROP, modifyTimeSeconds); - LOG.info("alter partition {} with statistic {}.", partitionSpec, statistic); - metastoreClient.alterPartition(partitionSpec, statistic, modifyTime, true); + PartitionStats partitionStats = + PartitionStats.create(fileCount, totalSize, rowCount, modifyTimeMillis); + LOG.info("alter partition {} with statistic {}.", partitionSpec, partitionStats); + metastoreClient.alterPartition(partitionSpec, partitionStats); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java index fca5dcf0ed69..3bdbdd20ad3e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java @@ -18,15 +18,15 @@ package org.apache.paimon.flink.sink.partition; -import org.apache.paimon.data.BinaryRow; import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.metastore.PartitionStats; import org.apache.paimon.partition.actions.AddDonePartitionAction; import org.junit.jupiter.api.Test; import java.util.HashSet; import java.util.LinkedHashMap; -import java.util.Map; +import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -41,39 +41,41 @@ public void test() throws Exception { Set donePartitions = new HashSet<>(); MetastoreClient metastoreClient = new MetastoreClient() { + @Override - public void addPartition(BinaryRow partition) { - throw new UnsupportedOperationException(); + public void addPartition(LinkedHashMap partition) { + donePartitions.add(generatePartitionPath(partition)); + } + + @Override + public void addPartitions(List> partitions) { + partitions.forEach(this::addPartition); } @Override - public void addPartition(LinkedHashMap partitionSpec) { - donePartitions.add(generatePartitionPath(partitionSpec)); + public void dropPartition(LinkedHashMap partition) { + throw new UnsupportedOperationException(); } @Override - public void deletePartition(LinkedHashMap partitionSpec) { + public void dropPartitions(List> partitions) { throw new UnsupportedOperationException(); } @Override - public void markDone(LinkedHashMap partitionSpec) - throws Exception { + public void markPartitionDone(LinkedHashMap partitions) { throw new UnsupportedOperationException(); } @Override public void alterPartition( LinkedHashMap partitionSpec, - Map parameters, - long modifyTime, - boolean ignoreIfNotExist) - throws Exception { + PartitionStats partitionStats) { throw new UnsupportedOperationException(); } @Override - public void close() throws Exception { + public void close() { closed.set(true); } }; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java index 142a0c32f781..0f761efa2278 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java @@ -18,12 +18,12 @@ package org.apache.paimon.flink.sink.partition; -import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.metastore.PartitionStats; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.FileStoreTable; @@ -35,7 +35,6 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.utils.PartitionPathUtils; -import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; @@ -86,47 +85,47 @@ public void testReportAction() throws Exception { BatchTableCommit committer = table.newBatchWriteBuilder().newCommit(); committer.commit(messages); AtomicBoolean closed = new AtomicBoolean(false); - Map> partitionParams = Maps.newHashMap(); + Map partitionParams = Maps.newHashMap(); MetastoreClient client = new MetastoreClient() { + + @Override + public void addPartition(LinkedHashMap partition) { + throw new UnsupportedOperationException(); + } + @Override - public void addPartition(BinaryRow partition) throws Exception { + public void addPartitions(List> partitions) { throw new UnsupportedOperationException(); } @Override - public void addPartition(LinkedHashMap partitionSpec) - throws Exception { + public void dropPartition(LinkedHashMap partition) { throw new UnsupportedOperationException(); } @Override - public void deletePartition(LinkedHashMap partitionSpec) - throws Exception { + public void dropPartitions(List> partitions) { throw new UnsupportedOperationException(); } @Override - public void markDone(LinkedHashMap partitionSpec) - throws Exception { + public void markPartitionDone(LinkedHashMap partitionSpec) { throw new UnsupportedOperationException(); } @Override public void alterPartition( LinkedHashMap partitionSpec, - Map parameters, - long modifyTime, - boolean ignoreIfNotExist) - throws Exception { + PartitionStats partitionStats) { partitionParams.put( PartitionPathUtils.generatePartitionPath(partitionSpec), - parameters); + partitionStats); } @Override - public void close() throws Exception { + public void close() { closed.set(true); } }; @@ -135,19 +134,9 @@ public void close() throws Exception { long time = 1729598544974L; action.report("c1=a/", time); Assertions.assertThat(partitionParams).containsKey("c1=a/"); - Assertions.assertThat(partitionParams.get("c1=a/")) + Assertions.assertThat(partitionParams.get("c1=a/").toString()) .isEqualTo( - ImmutableMap.of( - "numFiles", - "1", - "totalSize", - "591", - "numRows", - "1", - "lastUpdateTime", - String.valueOf(time / 1000), - "transient_lastDdlTime", - String.valueOf(time / 1000))); + "numFiles: 1, totalSize: 591, numRows: 1, lastUpdateTimeMillis: 1729598544974"); action.close(); Assertions.assertThat(closed).isTrue(); } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 5744ac894d12..f5ae504850ca 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -27,7 +27,6 @@ import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.client.ClientPool; -import org.apache.paimon.data.BinaryRow; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.hive.pool.CachedClientPool; @@ -48,6 +47,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.view.View; @@ -191,13 +191,12 @@ public Optional lockContext() { } @Override - public Optional metastoreClientFactory( - Identifier identifier, TableSchema schema) { + public Optional metastoreClientFactory(Identifier identifier) { Identifier tableIdentifier = new Identifier(identifier.getDatabaseName(), identifier.getTableName()); return Optional.of( new HiveMetastoreClient.Factory( - tableIdentifier, schema, hiveConf, clientClassName, options)); + tableIdentifier, hiveConf, clientClassName, options)); } @Override @@ -350,9 +349,8 @@ && new CoreOptions(tableSchema.options()).partitionedTableInMetastore() new HiveMetastoreClient( new Identifier( identifier.getDatabaseName(), identifier.getTableName()), - tableSchema, clients); - metastoreClient.deletePartition(new LinkedHashMap<>(partitionSpec)); + metastoreClient.dropPartition(new LinkedHashMap<>(partitionSpec)); } catch (Exception e) { throw new RuntimeException(e); } @@ -610,7 +608,7 @@ public org.apache.paimon.table.Table getDataOrFormatTable(Identifier identifier) lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableMeta.schema()).orElse(null))); + metastoreClientFactory(identifier).orElse(null))); } catch (TableNotExistException ignore) { } @@ -968,14 +966,19 @@ public void repairTable(Identifier identifier) throws TableNotExistException { // repair partitions if (!tableSchema.partitionKeys().isEmpty() && !newTable.getPartitionKeys().isEmpty()) { // Do not close client, it is for HiveCatalog + CoreOptions options = new CoreOptions(tableSchema.options()); + InternalRowPartitionComputer partitionComputer = + new InternalRowPartitionComputer( + options.partitionDefaultName(), + tableSchema.logicalPartitionType(), + tableSchema.partitionKeys().toArray(new String[0]), + options.legacyPartitionName()); @SuppressWarnings("resource") - HiveMetastoreClient metastoreClient = - new HiveMetastoreClient(identifier, tableSchema, clients); - List partitions = - getTable(identifier).newReadBuilder().newScan().listPartitions(); - for (BinaryRow partition : partitions) { - metastoreClient.addPartition(partition); - } + HiveMetastoreClient metastoreClient = new HiveMetastoreClient(identifier, clients); + metastoreClient.addPartitions( + getTable(identifier).newReadBuilder().newScan().listPartitions().stream() + .map(partitionComputer::generatePartValues) + .collect(Collectors.toList())); } } catch (Exception e) { throw new RuntimeException(e); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java index 885fa463e5a7..f7be538c259d 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java @@ -18,15 +18,12 @@ package org.apache.paimon.hive; -import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.client.ClientPool; -import org.apache.paimon.data.BinaryRow; import org.apache.paimon.hive.pool.CachedClientPool; import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.metastore.PartitionStats; import org.apache.paimon.options.Options; -import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.PartitionPathUtils; import org.apache.hadoop.hive.conf.HiveConf; @@ -39,34 +36,30 @@ import org.apache.thrift.TException; import java.util.ArrayList; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; +import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; + /** {@link MetastoreClient} for Hive tables. */ public class HiveMetastoreClient implements MetastoreClient { + private static final String HIVE_LAST_UPDATE_TIME_PROP = "transient_lastDdlTime"; + private final Identifier identifier; - private final InternalRowPartitionComputer partitionComputer; private final ClientPool clients; private final StorageDescriptor sd; - HiveMetastoreClient( - Identifier identifier, - TableSchema schema, - ClientPool clients) + HiveMetastoreClient(Identifier identifier, ClientPool clients) throws TException, InterruptedException { this.identifier = identifier; - CoreOptions options = new CoreOptions(schema.options()); - this.partitionComputer = - new InternalRowPartitionComputer( - options.partitionDefaultName(), - schema.logicalPartitionType(), - schema.partitionKeys().toArray(new String[0]), - options.legacyPartitionName()); - this.clients = clients; this.sd = this.clients @@ -79,22 +72,9 @@ public class HiveMetastoreClient implements MetastoreClient { } @Override - public void addPartition(BinaryRow partition) throws Exception { - addPartition(partitionComputer.generatePartValues(partition)); - } - - @Override - public void addPartitions(List partitions) throws Exception { - addPartitionsSpec( - partitions.stream() - .map(partitionComputer::generatePartValues) - .collect(Collectors.toList())); - } - - @Override - public void addPartition(LinkedHashMap partitionSpec) throws Exception { + public void addPartition(LinkedHashMap partition) throws Exception { Partition hivePartition = - toHivePartition(partitionSpec, (int) (System.currentTimeMillis() / 1000)); + toHivePartition(partition, (int) (System.currentTimeMillis() / 1000)); clients.execute( client -> { try { @@ -105,11 +85,10 @@ public void addPartition(LinkedHashMap partitionSpec) throws Exc } @Override - public void addPartitionsSpec(List> partitionSpecsList) - throws Exception { + public void addPartitions(List> partitions) throws Exception { int currentTime = (int) (System.currentTimeMillis() / 1000); List hivePartitions = - partitionSpecsList.stream() + partitions.stream() .map(partitionSpec -> toHivePartition(partitionSpec, currentTime)) .collect(Collectors.toList()); clients.execute(client -> client.add_partitions(hivePartitions, true, false)); @@ -117,43 +96,45 @@ public void addPartitionsSpec(List> partitionSpecs @Override public void alterPartition( - LinkedHashMap partitionSpec, - Map parameters, - long modifyTime, - boolean ignoreIfNotExist) + LinkedHashMap partition, PartitionStats partitionStats) throws Exception { - List partitionValues = new ArrayList<>(partitionSpec.values()); - int currentTime = (int) (modifyTime / 1000); - Partition hivePartition; + List partitionValues = new ArrayList<>(partition.values()); + + Map statistic = new HashMap<>(); + statistic.put(NUM_FILES_PROP, String.valueOf(partitionStats.numFiles())); + statistic.put(TOTAL_SIZE_PROP, String.valueOf(partitionStats.totalSize())); + statistic.put(NUM_ROWS_PROP, String.valueOf(partitionStats.numRows())); + + String modifyTimeSeconds = String.valueOf(partitionStats.lastUpdateTimeMillis() / 1000); + statistic.put(LAST_UPDATE_TIME_PROP, modifyTimeSeconds); + + // just for being compatible with hive metastore + statistic.put(HIVE_LAST_UPDATE_TIME_PROP, modifyTimeSeconds); + try { - hivePartition = + Partition hivePartition = clients.run( client -> client.getPartition( identifier.getDatabaseName(), identifier.getObjectName(), partitionValues)); + hivePartition.setValues(partitionValues); + hivePartition.setLastAccessTime((int) (partitionStats.lastUpdateTimeMillis() / 1000)); + hivePartition.getParameters().putAll(statistic); + clients.execute( + client -> + client.alter_partition( + identifier.getDatabaseName(), + identifier.getObjectName(), + hivePartition)); } catch (NoSuchObjectException e) { - if (ignoreIfNotExist) { - return; - } else { - throw e; - } + // do nothing if the partition not exists } - - hivePartition.setValues(partitionValues); - hivePartition.setLastAccessTime(currentTime); - hivePartition.getParameters().putAll(parameters); - clients.execute( - client -> - client.alter_partition( - identifier.getDatabaseName(), - identifier.getObjectName(), - hivePartition)); } @Override - public void deletePartition(LinkedHashMap partitionSpec) throws Exception { + public void dropPartition(LinkedHashMap partitionSpec) throws Exception { List partitionValues = new ArrayList<>(partitionSpec.values()); try { clients.execute( @@ -169,7 +150,14 @@ public void deletePartition(LinkedHashMap partitionSpec) throws } @Override - public void markDone(LinkedHashMap partitionSpec) throws Exception { + public void dropPartitions(List> partitions) throws Exception { + for (LinkedHashMap partition : partitions) { + dropPartition(partition); + } + } + + @Override + public void markPartitionDone(LinkedHashMap partitionSpec) throws Exception { try { clients.execute( client -> @@ -213,19 +201,13 @@ public static class Factory implements MetastoreClient.Factory { private static final long serialVersionUID = 1L; private final Identifier identifier; - private final TableSchema schema; private final SerializableHiveConf hiveConf; private final String clientClassName; private final Options options; public Factory( - Identifier identifier, - TableSchema schema, - HiveConf hiveConf, - String clientClassName, - Options options) { + Identifier identifier, HiveConf hiveConf, String clientClassName, Options options) { this.identifier = identifier; - this.schema = schema; this.hiveConf = new SerializableHiveConf(hiveConf); this.clientClassName = clientClassName; this.options = options; @@ -236,7 +218,7 @@ public MetastoreClient create() { HiveConf conf = hiveConf.conf(); try { return new HiveMetastoreClient( - identifier, schema, new CachedClientPool(conf, options, clientClassName)); + identifier, new CachedClientPool(conf, options, clientClassName)); } catch (TException e) { throw new RuntimeException( "Can not get table " + identifier + " info from metastore.", e); diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala index 840f1341a69d..c385f243ae66 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala @@ -78,7 +78,7 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { // sync to metastore with delete partitions if (clientFactory != null && fileStoreTable.coreOptions().partitionedTableInMetastore()) { metastoreClient = clientFactory.create() - toPaimonPartitions(rows).foreach(metastoreClient.deletePartition) + metastoreClient.dropPartitions(toPaimonPartitions(rows).toSeq.asJava) } } finally { commit.close() From 0fb607632831952a91b371c4edf4b76f6e764a70 Mon Sep 17 00:00:00 2001 From: lining Date: Tue, 17 Dec 2024 15:21:23 +0800 Subject: [PATCH 07/17] [core] Support alter database (#4700) --- docs/content/flink/sql-alter.md | 16 ++++ docs/content/program-api/catalog-api.md | 26 +++++- docs/content/spark/sql-alter.md | 18 ++++ .../paimon/catalog/AbstractCatalog.java | 20 ++++ .../apache/paimon/catalog/CachingCatalog.java | 7 ++ .../org/apache/paimon/catalog/Catalog.java | 13 +++ .../paimon/catalog/DelegateCatalog.java | 6 ++ .../paimon/catalog/FileSystemCatalog.java | 5 + .../apache/paimon/catalog/PropertyChange.java | 91 +++++++++++++++++++ .../org/apache/paimon/jdbc/JdbcCatalog.java | 45 +++++++++ .../org/apache/paimon/jdbc/JdbcUtils.java | 90 ++++++++++++++++++ .../privilege/AllGrantedPrivilegeChecker.java | 3 + .../paimon/privilege/PrivilegeChecker.java | 2 + .../privilege/PrivilegeCheckerImpl.java | 8 ++ .../paimon/privilege/PrivilegeType.java | 1 + .../paimon/privilege/PrivilegedCatalog.java | 8 ++ .../org/apache/paimon/rest/RESTCatalog.java | 37 +++++++- .../org/apache/paimon/rest/ResourcePaths.java | 9 ++ .../rest/requests/AlterDatabaseRequest.java | 59 ++++++++++++ .../rest/requests/CreateDatabaseRequest.java | 11 --- .../rest/responses/AlterDatabaseResponse.java | 69 ++++++++++++++ .../paimon/catalog/CachingCatalogTest.java | 23 +++++ .../paimon/catalog/CatalogTestBase.java | 47 ++++++++++ .../paimon/catalog/FileSystemCatalogTest.java | 16 ++++ .../apache/paimon/jdbc/JdbcCatalogTest.java | 5 + .../apache/paimon/rest/MockRESTMessage.java | 18 +++- .../apache/paimon/rest/RESTCatalogTest.java | 28 ++++++ .../paimon/rest/RESTObjectMapperTest.java | 23 ++++- paimon-flink/paimon-flink-common/pom.xml | 9 ++ .../org/apache/paimon/flink/FlinkCatalog.java | 67 +++++++++++--- .../apache/paimon/flink/FlinkCatalogTest.java | 83 ++++++++++++++++- .../org/apache/paimon/hive/HiveCatalog.java | 30 ++++++ .../apache/paimon/hive/HiveCatalogTest.java | 5 + paimon-open-api/rest-catalog-open-api.yaml | 81 +++++++++++++++-- .../open/api/RESTCatalogController.java | 33 ++++++- .../org/apache/paimon/spark/SparkCatalog.java | 50 ++++++++-- .../sql/DDLWithHiveCatalogTestBase.scala | 42 ++++++++- 37 files changed, 1049 insertions(+), 55 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java diff --git a/docs/content/flink/sql-alter.md b/docs/content/flink/sql-alter.md index 877995cc631b..6c3186b4af75 100644 --- a/docs/content/flink/sql-alter.md +++ b/docs/content/flink/sql-alter.md @@ -227,3 +227,19 @@ The following SQL modifies the watermark strategy to `ts - INTERVAL '2' HOUR`. ```sql ALTER TABLE my_table MODIFY WATERMARK FOR ts AS ts - INTERVAL '2' HOUR ``` + +# ALTER DATABASE + +The following SQL sets one or more properties in the specified database. If a particular property is already set in the database, override the old value with the new one. + +```sql +ALTER DATABASE [catalog_name.]db_name SET (key1=val1, key2=val2, ...) +``` + +## Altering Database Location + +The following SQL changes location of database `my_database` to `file:/temp/my_database`. + +```sql +ALTER DATABASE my_database SET ('location' = 'file:/temp/my_database') +``` diff --git a/docs/content/program-api/catalog-api.md b/docs/content/program-api/catalog-api.md index 570577437d86..7e716aad15bb 100644 --- a/docs/content/program-api/catalog-api.md +++ b/docs/content/program-api/catalog-api.md @@ -82,7 +82,7 @@ public class ListDatabases { ## Drop Database -You can use the catalog to drop databases. +You can use the catalog to drop database. ```java import org.apache.paimon.catalog.Catalog; @@ -102,6 +102,30 @@ public class DropDatabase { } ``` +## Alter Database + +You can use the catalog to alter database's properties.(ps: only support hive and jdbc catalog) + +```java +import java.util.ArrayList; +import org.apache.paimon.catalog.Catalog; + +public class AlterDatabase { + + public static void main(String[] args) { + try { + Catalog catalog = CreateCatalog.createHiveCatalog(); + List changes = new ArrayList<>(); + changes.add(DatabaseChange.setProperty("k1", "v1")); + changes.add(DatabaseChange.removeProperty("k2")); + catalog.alterDatabase("my_db", changes, true); + } catch (Catalog.DatabaseNotExistException e) { + // do something + } + } +} +``` + ## Determine Whether Table Exists You can use the catalog to determine whether the table exists diff --git a/docs/content/spark/sql-alter.md b/docs/content/spark/sql-alter.md index 3ad72048029b..359b1187292d 100644 --- a/docs/content/spark/sql-alter.md +++ b/docs/content/spark/sql-alter.md @@ -240,3 +240,21 @@ The following SQL changes the type of a nested column `f2` to `BIGINT` in a stru -- column v previously has type MAP> ALTER TABLE my_table ALTER COLUMN v.value.f2 TYPE BIGINT; ``` + + +# ALTER DATABASE + +The following SQL sets one or more properties in the specified database. If a particular property is already set in the database, override the old value with the new one. + +```sql +ALTER { DATABASE | SCHEMA | NAMESPACE } my_database + SET { DBPROPERTIES | PROPERTIES } ( property_name = property_value [ , ... ] ) +``` + +## Altering Database Location + +The following SQL sets the location of the specified database to `file:/temp/my_database.db`. + +```sql +ALTER DATABASE my_database SET LOCATION 'file:/temp/my_database.db' +``` \ No newline at end of file diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index 3fdefe6cacaa..51cb346d4bd7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -223,6 +223,26 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade protected abstract void dropDatabaseImpl(String name); + @Override + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + checkNotSystemDatabase(name); + try { + if (changes == null || changes.isEmpty()) { + return; + } + alterDatabaseImpl(name, changes); + } catch (DatabaseNotExistException e) { + if (ignoreIfNotExists) { + return; + } + throw new DatabaseNotExistException(name); + } + } + + protected abstract void alterDatabaseImpl(String name, List changes) + throws DatabaseNotExistException; + @Override public List listTables(String databaseName) throws DatabaseNotExistException { if (isSystemDatabase(databaseName)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 82d503b7a272..99540cf0cea5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -187,6 +187,13 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } } + @Override + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + super.alterDatabase(name, changes, ignoreIfNotExists); + databaseCache.invalidate(name); + } + @Override public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throws TableNotExistException { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 7b1fe0ea072e..37ea6fa5e203 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -126,6 +126,19 @@ void createDatabase(String name, boolean ignoreIfExists, Map pro void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException; + /** + * Alter a database. + * + * @param name Name of the database to alter. + * @param changes the property changes + * @param ignoreIfNotExists Flag to specify behavior when the database does not exist: if set to + * false, throw an exception, if set to true, do nothing. + * @throws DatabaseNotExistException if the given database is not exist and ignoreIfNotExists is + * false + */ + void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException; + /** * Return a {@link Table} identified by the given {@link Identifier}. * diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index 93e8ce2581ad..968f00cfcae5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -83,6 +83,12 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade wrapped.dropDatabase(name, ignoreIfNotExists, cascade); } + @Override + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + wrapped.alterDatabase(name, changes, ignoreIfNotExists); + } + @Override public List listTables(String databaseName) throws DatabaseNotExistException { return wrapped.listTables(databaseName); diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index 279ddb26ee53..cb0c358259f8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -92,6 +92,11 @@ protected void dropDatabaseImpl(String name) { uncheck(() -> fileIO.delete(newDatabasePath(name), true)); } + @Override + protected void alterDatabaseImpl(String name, List changes) { + throw new UnsupportedOperationException("Alter database is not supported."); + } + @Override protected List listTablesImpl(String databaseName) { return uncheck(() -> listTablesInFileSystem(newDatabasePath(databaseName))); diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java new file mode 100644 index 000000000000..c3423efd081e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java @@ -0,0 +1,91 @@ +/* + * 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.catalog; + +import org.apache.paimon.utils.Pair; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** define change to the database property. */ +public interface PropertyChange { + + static PropertyChange setProperty(String property, String value) { + return new SetProperty(property, value); + } + + static PropertyChange removeProperty(String property) { + return new RemoveProperty(property); + } + + static Pair, Set> getSetPropertiesToRemoveKeys( + List changes) { + Map setProperties = Maps.newHashMap(); + Set removeKeys = Sets.newHashSet(); + changes.forEach( + change -> { + if (change instanceof PropertyChange.SetProperty) { + PropertyChange.SetProperty setProperty = + (PropertyChange.SetProperty) change; + setProperties.put(setProperty.property(), setProperty.value()); + } else { + removeKeys.add(((PropertyChange.RemoveProperty) change).property()); + } + }); + return Pair.of(setProperties, removeKeys); + } + + /** Set property for database change. */ + final class SetProperty implements PropertyChange { + + private final String property; + private final String value; + + private SetProperty(String property, String value) { + this.property = property; + this.value = value; + } + + public String property() { + return this.property; + } + + public String value() { + return this.value; + } + } + + /** Remove property for database change. */ + final class RemoveProperty implements PropertyChange { + + private final String property; + + private RemoveProperty(String property) { + this.property = property; + } + + public String property() { + return this.property; + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 551b2d8fc910..63cb54c180f5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -24,6 +24,7 @@ import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.operation.Lock; @@ -33,11 +34,13 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,12 +55,15 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.paimon.jdbc.JdbcCatalogLock.acquireTimeout; import static org.apache.paimon.jdbc.JdbcCatalogLock.checkMaxSleep; +import static org.apache.paimon.jdbc.JdbcUtils.deleteProperties; import static org.apache.paimon.jdbc.JdbcUtils.execute; import static org.apache.paimon.jdbc.JdbcUtils.insertProperties; +import static org.apache.paimon.jdbc.JdbcUtils.updateProperties; import static org.apache.paimon.jdbc.JdbcUtils.updateTable; /* This file is based on source code from the Iceberg Project (http://iceberg.apache.org/), licensed by the Apache @@ -197,6 +203,45 @@ protected void dropDatabaseImpl(String name) { execute(connections, JdbcUtils.DELETE_ALL_DATABASE_PROPERTIES_SQL, catalogKey, name); } + @Override + protected void alterDatabaseImpl(String name, List changes) { + Pair, Set> setPropertiesToRemoveKeys = + PropertyChange.getSetPropertiesToRemoveKeys(changes); + Map setProperties = setPropertiesToRemoveKeys.getLeft(); + Set removeKeys = setPropertiesToRemoveKeys.getRight(); + Map startingProperties = fetchProperties(name); + Map inserts = Maps.newHashMap(); + Map updates = Maps.newHashMap(); + Set removes = Sets.newHashSet(); + if (!setProperties.isEmpty()) { + setProperties.forEach( + (k, v) -> { + if (!startingProperties.containsKey(k)) { + inserts.put(k, v); + } else { + updates.put(k, v); + } + }); + } + if (!removeKeys.isEmpty()) { + removeKeys.forEach( + k -> { + if (startingProperties.containsKey(k)) { + removes.add(k); + } + }); + } + if (!inserts.isEmpty()) { + insertProperties(connections, catalogKey, name, inserts); + } + if (!updates.isEmpty()) { + updateProperties(connections, catalogKey, name, updates); + } + if (!removes.isEmpty()) { + deleteProperties(connections, catalogKey, name, removes); + } + } + @Override protected List listTablesImpl(String databaseName) { return fetch( diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java index 4acb0f25aa91..1b9e599d72bc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java @@ -30,8 +30,10 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.SQLIntegrityConstraintViolationException; +import java.util.Collections; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.function.Consumer; import java.util.stream.Stream; @@ -202,6 +204,16 @@ public class JdbcUtils { + " = ? AND " + DATABASE_NAME + " = ? "; + static final String DELETE_DATABASE_PROPERTIES_SQL = + "DELETE FROM " + + DATABASE_PROPERTIES_TABLE_NAME + + " WHERE " + + CATALOG_KEY + + " = ? AND " + + DATABASE_NAME + + " = ? AND " + + DATABASE_PROPERTY_KEY + + " IN "; static final String DELETE_ALL_DATABASE_PROPERTIES_SQL = "DELETE FROM " + DATABASE_PROPERTIES_TABLE_NAME @@ -403,6 +415,75 @@ private static String insertPropertiesStatement(int size) { return sqlStatement.toString(); } + public static boolean updateProperties( + JdbcClientPool connections, + String storeKey, + String databaseName, + Map properties) { + Stream caseArgs = + properties.entrySet().stream() + .flatMap(entry -> Stream.of(entry.getKey(), entry.getValue())); + Stream whereArgs = + Stream.concat(Stream.of(storeKey, databaseName), properties.keySet().stream()); + + String[] args = Stream.concat(caseArgs, whereArgs).toArray(String[]::new); + + int updatedRecords = + execute(connections, JdbcUtils.updatePropertiesStatement(properties.size()), args); + if (updatedRecords == properties.size()) { + return true; + } + throw new IllegalStateException( + String.format( + "Failed to update: %d of %d succeeded", updatedRecords, properties.size())); + } + + private static String updatePropertiesStatement(int size) { + StringBuilder sqlStatement = + new StringBuilder( + "UPDATE " + + DATABASE_PROPERTIES_TABLE_NAME + + " SET " + + DATABASE_PROPERTY_VALUE + + " = CASE"); + for (int i = 0; i < size; i += 1) { + sqlStatement.append(" WHEN " + DATABASE_PROPERTY_KEY + " = ? THEN ?"); + } + + sqlStatement.append( + " END WHERE " + + CATALOG_KEY + + " = ? AND " + + DATABASE_NAME + + " = ? AND " + + DATABASE_PROPERTY_KEY + + " IN "); + + String values = String.join(",", Collections.nCopies(size, String.valueOf('?'))); + sqlStatement.append("(").append(values).append(")"); + + return sqlStatement.toString(); + } + + public static boolean deleteProperties( + JdbcClientPool connections, + String storeKey, + String databaseName, + Set removeKeys) { + String[] args = + Stream.concat(Stream.of(storeKey, databaseName), removeKeys.stream()) + .toArray(String[]::new); + + int deleteRecords = + execute(connections, JdbcUtils.deletePropertiesStatement(removeKeys), args); + if (deleteRecords > 0) { + return true; + } + throw new IllegalStateException( + String.format( + "Failed to delete: %d of %d succeeded", deleteRecords, removeKeys.size())); + } + public static void createDistributedLockTable(JdbcClientPool connections, Options options) throws SQLException, InterruptedException { DistributedLockDialectFactory.create(connections.getProtocol()) @@ -427,4 +508,13 @@ public static void release(JdbcClientPool connections, String lockId) DistributedLockDialectFactory.create(connections.getProtocol()) .releaseLock(connections, lockId); } + + private static String deletePropertiesStatement(Set properties) { + StringBuilder sqlStatement = new StringBuilder(JdbcUtils.DELETE_DATABASE_PROPERTIES_SQL); + String values = + String.join(",", Collections.nCopies(properties.size(), String.valueOf('?'))); + sqlStatement.append("(").append(values).append(")"); + + return sqlStatement.toString(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java b/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java index 09944681a2e7..8e8e4cd53d04 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java @@ -41,6 +41,9 @@ public void assertCanCreateTable(String databaseName) {} @Override public void assertCanDropDatabase(String databaseName) {} + @Override + public void assertCanAlterDatabase(String databaseName) {} + @Override public void assertCanCreateDatabase() {} diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java index 1771d40f4028..14cbbc6f36a3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java @@ -53,6 +53,8 @@ default void assertCanSelectOrInsert(Identifier identifier) { void assertCanDropDatabase(String databaseName); + void assertCanAlterDatabase(String databaseName); + void assertCanCreateDatabase(); void assertCanCreateUser(); diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java index 19c1813ee852..7e7876fa4e44 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java @@ -85,6 +85,14 @@ public void assertCanDropDatabase(String databaseName) { } } + @Override + public void assertCanAlterDatabase(String databaseName) { + if (!check(databaseName, PrivilegeType.ALTER_DATABASE)) { + throw new NoPrivilegeException( + user, "database", databaseName, PrivilegeType.ALTER_DATABASE); + } + } + @Override public void assertCanCreateDatabase() { if (!check( diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java index 375f5030d557..00b3a50596cb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java @@ -34,6 +34,7 @@ public enum PrivilegeType { CREATE_TABLE(PrivilegeTarget.DATABASE), DROP_DATABASE(PrivilegeTarget.DATABASE), + ALTER_DATABASE(PrivilegeTarget.DATABASE), CREATE_DATABASE(PrivilegeTarget.CATALOG), // you can create and drop users, grant and revoke any privileges to or from others diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java index 2e88213a24b9..35822471a2d6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java @@ -21,6 +21,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.DelegateCatalog; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; import org.apache.paimon.options.Options; @@ -82,6 +83,13 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade privilegeManager.objectDropped(name); } + @Override + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + privilegeManager.getPrivilegeChecker().assertCanAlterDatabase(name); + super.alterDatabase(name, changes, ignoreIfNotExists); + } + @Override public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throws TableNotExistException { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index c30e1109e2ec..8b53bef8486b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -21,6 +21,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.CatalogOptions; @@ -30,7 +31,9 @@ import org.apache.paimon.rest.auth.CredentialsProviderFactory; import org.apache.paimon.rest.exceptions.AlreadyExistsException; import org.apache.paimon.rest.exceptions.NoSuchResourceException; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; @@ -39,6 +42,7 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; @@ -49,6 +53,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; @@ -136,12 +141,14 @@ public List listDatabases() { @Override public void createDatabase(String name, boolean ignoreIfExists, Map properties) throws DatabaseAlreadyExistException { - CreateDatabaseRequest request = new CreateDatabaseRequest(name, ignoreIfExists, properties); + CreateDatabaseRequest request = new CreateDatabaseRequest(name, properties); try { client.post( resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); } catch (AlreadyExistsException e) { - throw new DatabaseAlreadyExistException(name); + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(name); + } } } @@ -172,6 +179,32 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } } + @Override + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + try { + Pair, Set> setPropertiesToRemoveKeys = + PropertyChange.getSetPropertiesToRemoveKeys(changes); + Map updateProperties = setPropertiesToRemoveKeys.getLeft(); + Set removeKeys = setPropertiesToRemoveKeys.getRight(); + AlterDatabaseRequest request = + new AlterDatabaseRequest(new ArrayList<>(removeKeys), updateProperties); + AlterDatabaseResponse response = + client.post( + resourcePaths.databaseProperties(name), + request, + AlterDatabaseResponse.class, + headers()); + if (response.getUpdated().isEmpty()) { + throw new IllegalStateException("Failed to update properties"); + } + } catch (NoSuchResourceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(name); + } + } + } + @Override public Table getTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index b58053374daa..51277454ffb0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -43,4 +43,13 @@ public String databases() { public String database(String databaseName) { return SLASH.add("v1").add(prefix).add("databases").add(databaseName).toString(); } + + public String databaseProperties(String databaseName) { + return SLASH.add("v1") + .add(prefix) + .add("databases") + .add(databaseName) + .add("properties") + .toString(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java new file mode 100644 index 000000000000..c1330142bb7e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java @@ -0,0 +1,59 @@ +/* + * 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.rest.requests; + +import org.apache.paimon.rest.RESTRequest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Map; + +/** Request for altering database. */ +public class AlterDatabaseRequest implements RESTRequest { + + private static final String FIELD_REMOVALS = "removals"; + private static final String FIELD_UPDATES = "updates"; + + @JsonProperty(FIELD_REMOVALS) + private List removals; + + @JsonProperty(FIELD_UPDATES) + private Map updates; + + @JsonCreator + public AlterDatabaseRequest( + @JsonProperty(FIELD_REMOVALS) List removals, + @JsonProperty(FIELD_UPDATES) Map updates) { + this.removals = removals; + this.updates = updates; + } + + @JsonGetter(FIELD_REMOVALS) + public List getRemovals() { + return removals; + } + + @JsonGetter(FIELD_UPDATES) + public Map getUpdates() { + return updates; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java index 6067bf544b87..07e5cf2462f2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -30,25 +30,19 @@ public class CreateDatabaseRequest implements RESTRequest { private static final String FIELD_NAME = "name"; - private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; private static final String FIELD_OPTIONS = "options"; @JsonProperty(FIELD_NAME) private String name; - @JsonProperty(FIELD_IGNORE_IF_EXISTS) - private boolean ignoreIfExists; - @JsonProperty(FIELD_OPTIONS) private Map options; @JsonCreator public CreateDatabaseRequest( @JsonProperty(FIELD_NAME) String name, - @JsonProperty(FIELD_IGNORE_IF_EXISTS) boolean ignoreIfExists, @JsonProperty(FIELD_OPTIONS) Map options) { this.name = name; - this.ignoreIfExists = ignoreIfExists; this.options = options; } @@ -57,11 +51,6 @@ public String getName() { return name; } - @JsonGetter(FIELD_IGNORE_IF_EXISTS) - public boolean getIgnoreIfExists() { - return ignoreIfExists; - } - @JsonGetter(FIELD_OPTIONS) public Map getOptions() { return options; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java new file mode 100644 index 000000000000..08d751dc595c --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java @@ -0,0 +1,69 @@ +/* + * 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.rest.responses; + +import org.apache.paimon.rest.RESTResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** Response for altering database. */ +public class AlterDatabaseResponse implements RESTResponse { + + private static final String FIELD_REMOVED = "removed"; + private static final String FIELD_UPDATED = "updated"; + private static final String FIELD_MISSING = "missing"; + + @JsonProperty(FIELD_REMOVED) + private List removed; + + @JsonProperty(FIELD_UPDATED) + private List updated; + + @JsonProperty(FIELD_MISSING) + private List missing; + + @JsonCreator + public AlterDatabaseResponse( + @JsonProperty(FIELD_REMOVED) List removed, + @JsonProperty(FIELD_UPDATED) List updated, + @JsonProperty(FIELD_MISSING) List missing) { + this.removed = removed; + this.updated = updated; + this.missing = missing; + } + + @JsonGetter(FIELD_REMOVED) + public List getRemoved() { + return removed; + } + + @JsonGetter(FIELD_UPDATED) + public List getUpdated() { + return updated; + } + + @JsonGetter(FIELD_MISSING) + public List getMissing() { + return missing; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java index 4792e33c932b..fee6d1433143 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java @@ -44,6 +44,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.io.FileNotFoundException; import java.time.Duration; @@ -63,6 +64,8 @@ import static org.apache.paimon.options.CatalogOptions.CACHE_MANIFEST_SMALL_FILE_THRESHOLD; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.when; class CachingCatalogTest extends CatalogTestBase { @@ -86,6 +89,26 @@ public void testListDatabasesWhenNoDatabases() { assertThat(databases).contains("db"); } + @Test + public void testInvalidateWhenDatabaseIsAltered() throws Exception { + Catalog mockcatalog = Mockito.mock(Catalog.class); + Catalog catalog = new CachingCatalog(mockcatalog); + String databaseName = "db"; + boolean ignoreIfExists = false; + Database database = Database.of(databaseName); + Database secondDatabase = Database.of(databaseName); + when(mockcatalog.getDatabase(databaseName)).thenReturn(database, secondDatabase); + doNothing().when(mockcatalog).alterDatabase(databaseName, emptyList(), ignoreIfExists); + Database cachingDatabase = catalog.getDatabase(databaseName); + assertThat(cachingDatabase.name()).isEqualTo(databaseName); + catalog.alterDatabase(databaseName, emptyList(), ignoreIfExists); + Database newCachingDatabase = catalog.getDatabase(databaseName); + // same as secondDatabase means cache is invalidated, so call getDatabase again then return + // secondDatabase + assertThat(newCachingDatabase).isNotSameAs(database); + assertThat(newCachingDatabase).isSameAs(secondDatabase); + } + @Test public void testInvalidateSystemTablesIfBaseTableIsModified() throws Exception { Catalog catalog = new CachingCatalog(this.catalog); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 98a9b92c5c38..31c4c8e682b8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -54,6 +54,7 @@ import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; /** Base test class of paimon catalog in {@link Catalog}. */ public abstract class CatalogTestBase { @@ -960,4 +961,50 @@ public void testTableUUID() throws Exception { assertThat(Long.parseLong(uuid.substring((identifier.getFullName() + ".").length()))) .isGreaterThan(0); } + + protected void alterDatabaseWhenSupportAlter() throws Exception { + // Alter database + String databaseName = "db_to_alter"; + catalog.createDatabase(databaseName, false); + String key = "key1"; + String key2 = "key2"; + // Add property + catalog.alterDatabase( + databaseName, + Lists.newArrayList( + PropertyChange.setProperty(key, "value"), + PropertyChange.setProperty(key2, "value")), + false); + Database db = catalog.getDatabase(databaseName); + assertEquals("value", db.options().get(key)); + assertEquals("value", db.options().get(key2)); + // Update property + catalog.alterDatabase( + databaseName, + Lists.newArrayList( + PropertyChange.setProperty(key, "value1"), + PropertyChange.setProperty(key2, "value1")), + false); + db = catalog.getDatabase(databaseName); + assertEquals("value1", db.options().get(key)); + assertEquals("value1", db.options().get(key2)); + // remove property + catalog.alterDatabase( + databaseName, + Lists.newArrayList( + PropertyChange.removeProperty(key), PropertyChange.removeProperty(key2)), + false); + db = catalog.getDatabase(databaseName); + assertEquals(false, db.options().containsKey(key)); + assertEquals(false, db.options().containsKey(key2)); + // Remove non-existent property + catalog.alterDatabase( + databaseName, + Lists.newArrayList( + PropertyChange.removeProperty(key), PropertyChange.removeProperty(key2)), + false); + db = catalog.getDatabase(databaseName); + assertEquals(false, db.options().containsKey(key)); + assertEquals(false, db.options().containsKey(key2)); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index 65ea6721c220..7045daca8e86 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -24,10 +24,13 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link FileSystemCatalog}. */ public class FileSystemCatalogTest extends CatalogTestBase { @@ -67,4 +70,17 @@ public void testCreateTableCaseSensitive() throws Exception { .isThrownBy(() -> catalog.createTable(identifier, schema, false)) .withMessage("Field name [Pk1, Col1] cannot contain upper case in the catalog."); } + + @Test + public void testAlterDatabase() throws Exception { + String databaseName = "test_alter_db"; + catalog.createDatabase(databaseName, false); + assertThatThrownBy( + () -> + catalog.alterDatabase( + databaseName, + Lists.newArrayList(PropertyChange.removeProperty("a")), + false)) + .isInstanceOf(UnsupportedOperationException.class); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index f5befc724f8b..f01a46fd6bb4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -122,4 +122,9 @@ public void testSerializeTable() throws Exception { } }); } + + @Test + public void testAlterDatabase() throws Exception { + this.alterDatabaseWhenSupportAlter(); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java index a605e5e77c2a..821257a0e10e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java @@ -18,13 +18,17 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -40,10 +44,9 @@ public static String databaseName() { } public static CreateDatabaseRequest createDatabaseRequest(String name) { - boolean ignoreIfExists = true; Map options = new HashMap<>(); options.put("a", "b"); - return new CreateDatabaseRequest(name, ignoreIfExists, options); + return new CreateDatabaseRequest(name, options); } public static CreateDatabaseResponse createDatabaseResponse(String name) { @@ -69,4 +72,15 @@ public static ListDatabasesResponse listDatabasesResponse(String name) { public static ErrorResponse noSuchResourceExceptionErrorResponse() { return new ErrorResponse("message", 404, new ArrayList<>()); } + + public static AlterDatabaseRequest alterDatabaseRequest() { + Map add = new HashMap<>(); + add.put("add", "value"); + return new AlterDatabaseRequest(Lists.newArrayList("remove"), add); + } + + public static AlterDatabaseResponse alterDatabaseResponse() { + return new AlterDatabaseResponse( + Lists.newArrayList("remove"), Lists.newArrayList("add"), new ArrayList<>()); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java index 0fff81afdcde..9b1582929560 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Database; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; +import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; @@ -185,6 +186,33 @@ public void testDropDatabaseWhenCascadeIsFalseAndTablesExist() throws Exception verify(mockRestCatalog, times(1)).listTables(eq(name)); } + @Test + public void testAlterDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + AlterDatabaseResponse response = MockRESTMessage.alterDatabaseResponse(); + mockResponse(mapper.writeValueAsString(response), 200); + assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); + } + + @Test + public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsFalse() + throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertThrows( + Catalog.DatabaseNotExistException.class, + () -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), false)); + } + + @Test + public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsTrue() throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); + } + private void mockResponse(String mockResponse, int httpCode) { MockResponse mockResponseObj = new MockResponse() diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java index 7fee81ef1024..0e5a71be39c0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -18,7 +18,9 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; @@ -68,7 +70,6 @@ public void createDatabaseRequestParseTest() throws Exception { String requestStr = mapper.writeValueAsString(request); CreateDatabaseRequest parseData = mapper.readValue(requestStr, CreateDatabaseRequest.class); assertEquals(request.getName(), parseData.getName()); - assertEquals(request.getIgnoreIfExists(), parseData.getIgnoreIfExists()); assertEquals(request.getOptions().size(), parseData.getOptions().size()); } @@ -104,4 +105,24 @@ public void listDatabaseResponseParseTest() throws Exception { assertEquals(response.getDatabases().size(), parseData.getDatabases().size()); assertEquals(name, parseData.getDatabases().get(0).getName()); } + + @Test + public void alterDatabaseRequestParseTest() throws Exception { + AlterDatabaseRequest request = MockRESTMessage.alterDatabaseRequest(); + String requestStr = mapper.writeValueAsString(request); + AlterDatabaseRequest parseData = mapper.readValue(requestStr, AlterDatabaseRequest.class); + assertEquals(request.getRemovals().size(), parseData.getRemovals().size()); + assertEquals(request.getUpdates().size(), parseData.getUpdates().size()); + } + + @Test + public void alterDatabaseResponseParseTest() throws Exception { + AlterDatabaseResponse response = MockRESTMessage.alterDatabaseResponse(); + String responseStr = mapper.writeValueAsString(response); + AlterDatabaseResponse parseData = + mapper.readValue(responseStr, AlterDatabaseResponse.class); + assertEquals(response.getRemoved().size(), parseData.getRemoved().size()); + assertEquals(response.getUpdated().size(), parseData.getUpdated().size()); + assertEquals(response.getMissing().size(), parseData.getMissing().size()); + } } diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index 91222983bf6b..7388f8944109 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -169,8 +169,17 @@ under the License. + + + org.mockito + mockito-inline + ${mockito.version} + jar + test + + diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 3407735b4b79..ec3c4a47a69d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -20,9 +20,12 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TableType; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogUtils; +import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.paimon.flink.utils.FlinkDescriptorProperties; @@ -133,6 +136,7 @@ import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_MODE; import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_STATUS; import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.catalog.Catalog.COMMENT_PROP; import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; @@ -236,19 +240,19 @@ public CatalogDatabase getDatabase(String databaseName) @Override public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) throws DatabaseAlreadyExistException, CatalogException { + Map properties; if (database != null) { + properties = new HashMap<>(database.getProperties()); if (database.getDescription().isPresent() && !database.getDescription().get().equals("")) { - throw new UnsupportedOperationException( - "Create database with description is unsupported."); + properties.put(COMMENT_PROP, database.getDescription().get()); } + } else { + properties = Collections.emptyMap(); } try { - catalog.createDatabase( - name, - ignoreIfExists, - database == null ? Collections.emptyMap() : database.getProperties()); + catalog.createDatabase(name, ignoreIfExists, properties); } catch (Catalog.DatabaseAlreadyExistException e) { throw new DatabaseAlreadyExistException(getName(), e.database()); } @@ -620,7 +624,7 @@ private List toSchemaChange( SchemaManager.checkAlterTablePath(key); - if (Catalog.COMMENT_PROP.equals(key)) { + if (COMMENT_PROP.equals(key)) { schemaChanges.add(SchemaChange.updateComment(value)); } else { schemaChanges.add(SchemaChange.setOption(key, value)); @@ -629,7 +633,7 @@ private List toSchemaChange( } else if (change instanceof ResetOption) { ResetOption resetOption = (ResetOption) change; String key = resetOption.getKey(); - if (Catalog.COMMENT_PROP.equals(key)) { + if (COMMENT_PROP.equals(key)) { schemaChanges.add(SchemaChange.updateComment(null)); } else { schemaChanges.add(SchemaChange.removeOption(resetOption.getKey())); @@ -1209,13 +1213,22 @@ public static Identifier toIdentifier(ObjectPath path) { return new Identifier(path.getDatabaseName(), path.getObjectName()); } - // --------------------- unsupported methods ---------------------------- - @Override public final void alterDatabase( String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); + throws CatalogException, DatabaseNotExistException { + try { + Database oldDatabase = catalog.getDatabase(name); + List changes = + getPropertyChanges(oldDatabase.options(), newDatabase.getProperties()); + getPropertyChangeFromComment(oldDatabase.comment(), newDatabase.getDescription()) + .ifPresent(changes::add); + catalog.alterDatabase(name, changes, ignoreIfNotExists); + } catch (Catalog.DatabaseNotExistException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), e.database()); + } + } } @Override @@ -1264,6 +1277,36 @@ public final List listPartitions(ObjectPath tablePath) return getPartitionSpecs(tablePath, null); } + @VisibleForTesting + static List getPropertyChanges( + Map oldOptions, Map newOptions) { + List changes = new ArrayList<>(); + newOptions.forEach( + (k, v) -> { + if (!oldOptions.containsKey(k) || !oldOptions.get(k).equals(v)) { + changes.add(PropertyChange.setProperty(k, v)); + } + }); + oldOptions + .keySet() + .forEach( + (k) -> { + if (!newOptions.containsKey(k)) { + changes.add(PropertyChange.removeProperty(k)); + } + }); + return changes; + } + + @VisibleForTesting + static Optional getPropertyChangeFromComment( + Optional oldComment, Optional newComment) { + if (newComment.isPresent() && !oldComment.equals(newComment)) { + return Optional.of(PropertyChange.setProperty(COMMENT_PROP, newComment.get())); + } + return Optional.empty(); + } + private Table getPaimonTable(ObjectPath tablePath) throws TableNotExistException { try { Identifier identifier = toIdentifier(tablePath); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index 734a47dead06..4b8cf7912192 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.TableType; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.flink.log.LogSinkProvider; import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.log.LogStoreRegister; @@ -83,6 +84,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.stream.Stream; @@ -98,7 +100,13 @@ import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatCollection; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** Test for {@link FlinkCatalog}. */ public class FlinkCatalogTest { @@ -571,11 +579,9 @@ public void testCreateDb_DatabaseWithProperties() throws Exception { } @Test - public void testCreateDb_DatabaseWithCommentException() { + public void testCreateDb_DatabaseWithCommentSuccessful() throws DatabaseAlreadyExistException { CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), "haha"); - assertThatThrownBy(() -> catalog.createDatabase(path1.getDatabaseName(), database, false)) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Create database with description is unsupported."); + assertDoesNotThrow(() -> catalog.createDatabase(path1.getDatabaseName(), database, false)); } @ParameterizedTest @@ -596,6 +602,75 @@ public void testDropDb_DatabaseNotExistException() { .hasMessage("Database db1 does not exist in Catalog test-catalog."); } + @Test + public void testAlterDb() throws DatabaseAlreadyExistException, DatabaseNotExistException { + CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); + catalog.createDatabase(path1.getDatabaseName(), database, false); + Map properties = Collections.singletonMap("haa", "ccc"); + CatalogDatabaseImpl newDatabase = new CatalogDatabaseImpl(properties, "haha"); + // as file system catalog don't support alter database, so we have to use mock to overview + // this method to test + Catalog mockCatalog = spy(catalog); + doNothing().when(mockCatalog).alterDatabase(path1.getDatabaseName(), newDatabase, false); + when(mockCatalog.getDatabase(path1.getDatabaseName())).thenReturn(database); + mockCatalog.alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).getDatabase(path1.getDatabaseName()); + } + + @Test + public void testAlterDbComment() + throws DatabaseAlreadyExistException, DatabaseNotExistException { + CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); + catalog.createDatabase(path1.getDatabaseName(), database, false); + Catalog mockCatalog = spy(catalog); + when(mockCatalog.getDatabase(path1.getDatabaseName())).thenReturn(database); + CatalogDatabaseImpl newDatabase = new CatalogDatabaseImpl(Collections.emptyMap(), "aa"); + doNothing().when(mockCatalog).alterDatabase(path1.getDatabaseName(), newDatabase, false); + mockCatalog.alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).getDatabase(path1.getDatabaseName()); + } + + @Test + public void testAlterDb_DatabaseNotExistException() { + CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); + assertThatThrownBy(() -> catalog.alterDatabase(path1.getDatabaseName(), database, false)) + .isInstanceOf(DatabaseNotExistException.class) + .hasMessage("Database db1 does not exist in Catalog test-catalog."); + } + + @Test + public void testGetProperties() throws Exception { + Map oldProperties = Collections.emptyMap(); + Map newProperties = Collections.singletonMap("haa", "ccc"); + List propertyChanges = + FlinkCatalog.getPropertyChanges(oldProperties, newProperties); + assertThat(propertyChanges.size()).isEqualTo(1); + oldProperties = newProperties; + propertyChanges = FlinkCatalog.getPropertyChanges(oldProperties, newProperties); + assertThat(propertyChanges.size()).isEqualTo(0); + oldProperties = Collections.singletonMap("aa", "ccc"); + propertyChanges = FlinkCatalog.getPropertyChanges(oldProperties, newProperties); + assertThat(propertyChanges.size()).isEqualTo(2); + } + + @Test + public void testGetPropertyChangeFromComment() { + Optional commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.empty(), Optional.empty()); + assertThat(commentChange.isPresent()).isFalse(); + commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.of("aa"), Optional.of("bb")); + assertThat(commentChange.isPresent()).isTrue(); + commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.of("aa"), Optional.empty()); + assertThat(commentChange.isPresent()).isFalse(); + commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.empty(), Optional.of("bb")); + assertThat(commentChange.isPresent()).isTrue(); + } + @Test public void testCreateTableWithColumnOptions() throws Exception { ResolvedExpression expression = diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index f5ae504850ca..a590ede6d2e6 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -26,6 +26,7 @@ import org.apache.paimon.catalog.CatalogLockContext; import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.client.ClientPool; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -53,6 +54,8 @@ import org.apache.paimon.view.View; import org.apache.paimon.view.ViewImpl; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + import org.apache.flink.table.hive.LegacyHiveClasses; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -403,6 +406,33 @@ protected void dropDatabaseImpl(String name) { } } + @Override + protected void alterDatabaseImpl(String name, List changes) { + try { + Database database = clients.run(client -> client.getDatabase(name)); + Map parameter = Maps.newHashMap(); + parameter.putAll(database.getParameters()); + Pair, Set> setPropertiesToRemoveKeys = + PropertyChange.getSetPropertiesToRemoveKeys(changes); + Map setProperties = setPropertiesToRemoveKeys.getLeft(); + Set removeKeys = setPropertiesToRemoveKeys.getRight(); + if (setProperties.size() > 0) { + parameter.putAll(setProperties); + } + if (removeKeys.size() > 0) { + parameter.keySet().removeAll(removeKeys); + } + Map newProperties = Collections.unmodifiableMap(parameter); + Database alterDatabase = convertToHiveDatabase(name, newProperties); + clients.execute(client -> client.alterDatabase(name, alterDatabase)); + } catch (TException e) { + throw new RuntimeException("Failed to alter database " + name, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to alterDatabase " + name, e); + } + } + @Override protected List listTablesImpl(String databaseName) { try { diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index 267bdf0c7100..e3b48f02a696 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -173,6 +173,11 @@ private void testHiveConfDirFromEnvImpl() { assertThat(hiveConf.get("hive.metastore.uris")).isEqualTo("dummy-hms"); } + @Test + public void testAlterDatabase() throws Exception { + this.alterDatabaseWhenSupportAlter(); + } + @Test public void testAddHiveTableParameters() { try { diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 9b69b3de2776..f7f9529f53dd 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -80,6 +80,43 @@ paths: $ref: '#/components/schemas/ErrorResponse' "500": description: Internal Server Error + /v1/{prefix}/databases/{database}/properties: + post: + tags: + - database + summary: Alter Database + operationId: alterDatabase + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/AlterDatabaseRequest' + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/AlterDatabaseResponse' + "404": + description: Resource not found + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error /v1/{prefix}/databases/{database}: get: tags: @@ -116,7 +153,7 @@ paths: tags: - database summary: Drop Database - operationId: dropDatabases + operationId: dropDatabase parameters: - name: prefix in: path @@ -159,12 +196,22 @@ components: properties: name: type: string - ignoreIfExists: - type: boolean options: type: object additionalProperties: type: string + ErrorResponse: + type: object + properties: + message: + type: string + code: + type: integer + format: int32 + stack: + type: array + items: + type: string CreateDatabaseResponse: type: object properties: @@ -174,15 +221,29 @@ components: type: object additionalProperties: type: string - ErrorResponse: + AlterDatabaseRequest: type: object properties: - message: - type: string - code: - type: integer - format: int32 - stack: + removals: + type: array + items: + type: string + updates: + type: object + additionalProperties: + type: string + AlterDatabaseResponse: + type: object + properties: + removed: + type: array + items: + type: string + updated: + type: array + items: + type: string + missing: type: array items: type: string diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java index 19f6f8cdf673..5331b65d71b6 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -19,7 +19,9 @@ package org.apache.paimon.open.api; import org.apache.paimon.rest.ResourcePaths; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; @@ -28,6 +30,7 @@ import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import io.swagger.v3.oas.annotations.Operation; import io.swagger.v3.oas.annotations.media.Content; @@ -149,5 +152,33 @@ public GetDatabaseResponse getDatabases( content = {@Content(schema = @Schema())}) }) @DeleteMapping("/v1/{prefix}/databases/{database}") - public void dropDatabases(@PathVariable String prefix, @PathVariable String database) {} + public void dropDatabase(@PathVariable String prefix, @PathVariable String database) {} + + @Operation( + summary = "Alter Database", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = { + @Content(schema = @Schema(implementation = AlterDatabaseResponse.class)) + }), + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/v1/{prefix}/databases/{database}/properties") + public AlterDatabaseResponse alterDatabase( + @PathVariable String prefix, + @PathVariable String database, + @RequestBody AlterDatabaseRequest request) { + return new AlterDatabaseResponse( + Lists.newArrayList("remove"), + Lists.newArrayList("add"), + Lists.newArrayList("missing")); + } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index de6e2414fc8f..12023cb84779 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -130,7 +131,8 @@ public void createNamespace(String[] namespace, Map metadata) throws NamespaceAlreadyExistsException { checkNamespace(namespace); try { - catalog.createDatabase(namespace[0], false, metadata); + String databaseName = getDatabaseNameFromNamespace(namespace); + catalog.createDatabase(databaseName, false, metadata); } catch (Catalog.DatabaseAlreadyExistException e) { throw new NamespaceAlreadyExistsException(namespace); } @@ -153,7 +155,8 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep } checkNamespace(namespace); try { - catalog.getDatabase(namespace[0]); + String databaseName = getDatabaseNameFromNamespace(namespace); + catalog.getDatabase(databaseName); return new String[0][]; } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); @@ -164,9 +167,9 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep public Map loadNamespaceMetadata(String[] namespace) throws NoSuchNamespaceException { checkNamespace(namespace); - String dataBaseName = namespace[0]; try { - return catalog.getDatabase(dataBaseName).options(); + String databaseName = getDatabaseNameFromNamespace(namespace); + return catalog.getDatabase(databaseName).options(); } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); } @@ -203,7 +206,8 @@ public boolean dropNamespace(String[] namespace, boolean cascade) throws NoSuchNamespaceException { checkNamespace(namespace); try { - catalog.dropDatabase(namespace[0], false, cascade); + String databaseName = getDatabaseNameFromNamespace(namespace); + catalog.dropDatabase(databaseName, false, cascade); return true; } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); @@ -217,7 +221,8 @@ public boolean dropNamespace(String[] namespace, boolean cascade) public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { checkNamespace(namespace); try { - return catalog.listTables(namespace[0]).stream() + String databaseName = getDatabaseNameFromNamespace(namespace); + return catalog.listTables(databaseName).stream() .map(table -> Identifier.of(namespace, table)) .toArray(Identifier[]::new); } catch (Catalog.DatabaseNotExistException e) { @@ -516,10 +521,35 @@ protected List convertPartitionTransforms(Transform[] transforms) { return partitionColNames; } - // --------------------- unsupported methods ---------------------------- - @Override - public void alterNamespace(String[] namespace, NamespaceChange... changes) { - throw new UnsupportedOperationException("Alter namespace in Spark is not supported yet."); + public void alterNamespace(String[] namespace, NamespaceChange... changes) + throws NoSuchNamespaceException { + checkNamespace(namespace); + try { + String databaseName = getDatabaseNameFromNamespace(namespace); + List propertyChanges = + Arrays.stream(changes).map(this::toPropertyChange).collect(Collectors.toList()); + catalog.alterDatabase(databaseName, propertyChanges, false); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + private PropertyChange toPropertyChange(NamespaceChange change) { + if (change instanceof NamespaceChange.SetProperty) { + NamespaceChange.SetProperty set = (NamespaceChange.SetProperty) change; + return PropertyChange.setProperty(set.property(), set.value()); + } else if (change instanceof NamespaceChange.RemoveProperty) { + NamespaceChange.RemoveProperty remove = (NamespaceChange.RemoveProperty) change; + return PropertyChange.removeProperty(remove.property()); + + } else { + throw new UnsupportedOperationException( + "Change is not supported: " + change.getClass()); + } + } + + private String getDatabaseNameFromNamespace(String[] namespace) { + return namespace[0]; } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 4ba079ea0bb2..526e24250751 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -22,7 +22,7 @@ import org.apache.paimon.hive.HiveMetastoreClient import org.apache.paimon.spark.PaimonHiveTestBase import org.apache.paimon.table.FileStoreTable -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.junit.jupiter.api.Assertions abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { @@ -194,6 +194,46 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: alter database's properties") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + val databaseName = "paimon_db" + withDatabase(databaseName) { + spark.sql(s"CREATE DATABASE $databaseName WITH DBPROPERTIES ('k1' = 'v1', 'k2' = 'v2')") + var props = getDatabaseProps(databaseName) + Assertions.assertEquals(props("k1"), "v1") + Assertions.assertEquals(props("k2"), "v2") + spark.sql(s"ALTER DATABASE $databaseName SET DBPROPERTIES ('k1' = 'v11', 'k2' = 'v22')") + props = getDatabaseProps(databaseName) + Assertions.assertEquals(props("k1"), "v11") + Assertions.assertEquals(props("k2"), "v22") + } + } + } + + test("Paimon DDL with hive catalog: alter database location") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + val databaseName = "paimon_db" + withDatabase(databaseName) { + spark.sql(s"CREATE DATABASE $databaseName WITH DBPROPERTIES ('k1' = 'v1', 'k2' = 'v2')") + withTempDir { + dBLocation => + try { + spark.sql( + s"ALTER DATABASE $databaseName SET LOCATION '${dBLocation.getCanonicalPath}'") + } catch { + case e: AnalysisException => + Assertions.assertTrue( + e.getMessage.contains("does not support altering database location")) + } + } + } + } + } + test("Paimon DDL with hive catalog: set default database") { var reusedSpark = spark From 3a9e1e83fad577d8408fb820563735a5d5f13945 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Tue, 17 Dec 2024 18:15:37 +0800 Subject: [PATCH 08/17] [core] Introduce Variant Data (#4729) --- LICENSE | 5 + .../paimon/data/variant/GenericVariant.java | 484 +++++++++++++ .../data/variant/GenericVariantBuilder.java | 639 ++++++++++++++++++ .../data/variant/GenericVariantUtil.java | 628 +++++++++++++++++ .../paimon/data/variant/PathSegment.java | 104 +++ .../apache/paimon/data/variant/Variant.java | 52 ++ .../data/variant/GenericVariantTest.java | 98 +++ 7 files changed, 2010 insertions(+) create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariant.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantBuilder.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantUtil.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/variant/PathSegment.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/data/variant/Variant.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/data/variant/GenericVariantTest.java diff --git a/LICENSE b/LICENSE index 38bbe5ec276d..d669daad0244 100644 --- a/LICENSE +++ b/LICENSE @@ -270,6 +270,11 @@ paimon-format/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java paimon-format/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java from https://parquet.apache.org/ version 1.14.0 +paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariant.java +paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantBuilder.java +paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantUtil.java +from https://spark.apache.org/ version 4.0.0-preview2 + MIT License ----------- diff --git a/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariant.java b/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariant.java new file mode 100644 index 000000000000..355e9123cc2d --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariant.java @@ -0,0 +1,484 @@ +/* + * 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.data.variant; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonFactory; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; + +import java.io.CharArrayWriter; +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDate; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Base64; +import java.util.Locale; +import java.util.Objects; + +import static org.apache.paimon.data.variant.GenericVariantUtil.BINARY_SEARCH_THRESHOLD; +import static org.apache.paimon.data.variant.GenericVariantUtil.SIZE_LIMIT; +import static org.apache.paimon.data.variant.GenericVariantUtil.Type; +import static org.apache.paimon.data.variant.GenericVariantUtil.VERSION; +import static org.apache.paimon.data.variant.GenericVariantUtil.VERSION_MASK; +import static org.apache.paimon.data.variant.GenericVariantUtil.checkIndex; +import static org.apache.paimon.data.variant.GenericVariantUtil.getMetadataKey; +import static org.apache.paimon.data.variant.GenericVariantUtil.handleArray; +import static org.apache.paimon.data.variant.GenericVariantUtil.handleObject; +import static org.apache.paimon.data.variant.GenericVariantUtil.malformedVariant; +import static org.apache.paimon.data.variant.GenericVariantUtil.readUnsigned; +import static org.apache.paimon.data.variant.GenericVariantUtil.valueSize; +import static org.apache.paimon.data.variant.GenericVariantUtil.variantConstructorSizeLimit; + +/* This file is based on source code from the Spark Project (http://spark.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** An internal data structure implementing {@link Variant}. */ +public final class GenericVariant implements Variant { + + private final byte[] value; + private final byte[] metadata; + // The variant value doesn't use the whole `value` binary, but starts from its `pos` index and + // spans a size of `valueSize(value, pos)`. This design avoids frequent copies of the value + // binary when reading a sub-variant in the array/object element. + private final int pos; + + public GenericVariant(byte[] value, byte[] metadata) { + this(value, metadata, 0); + } + + private GenericVariant(byte[] value, byte[] metadata, int pos) { + this.value = value; + this.metadata = metadata; + this.pos = pos; + // There is currently only one allowed version. + if (metadata.length < 1 || (metadata[0] & VERSION_MASK) != VERSION) { + throw malformedVariant(); + } + // Don't attempt to use a Variant larger than 16 MiB. We'll never produce one, and it risks + // memory instability. + if (metadata.length > SIZE_LIMIT || value.length > SIZE_LIMIT) { + throw variantConstructorSizeLimit(); + } + } + + @Override + public byte[] value() { + if (pos == 0) { + return value; + } + int size = valueSize(value, pos); + checkIndex(pos + size - 1, value.length); + return Arrays.copyOfRange(value, pos, pos + size); + } + + public byte[] rawValue() { + return value; + } + + @Override + public byte[] metadata() { + return metadata; + } + + public int pos() { + return pos; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + GenericVariant that = (GenericVariant) o; + return pos == that.pos + && Objects.deepEquals(value, that.value) + && Objects.deepEquals(metadata, that.metadata); + } + + @Override + public int hashCode() { + return Objects.hash(Arrays.hashCode(value), Arrays.hashCode(metadata), pos); + } + + public static Variant fromJson(String json) { + try { + return GenericVariantBuilder.parseJson(json, false); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public String toJson() { + return toJson(ZoneOffset.UTC); + } + + // Stringify the variant in JSON format. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public String toJson(ZoneId zoneId) { + StringBuilder sb = new StringBuilder(); + toJsonImpl(value, metadata, pos, sb, zoneId); + return sb.toString(); + } + + @Override + public String toString() { + return toJson(); + } + + public Object variantGet(String path) { + GenericVariant v = this; + PathSegment[] parsedPath = PathSegment.parse(path); + for (PathSegment pathSegment : parsedPath) { + if (pathSegment.isKey() && v.getType() == Type.OBJECT) { + v = v.getFieldByKey(pathSegment.getKey()); + } else if (pathSegment.isIndex() && v.getType() == Type.ARRAY) { + v = v.getElementAtIndex(pathSegment.getIndex()); + } else { + return null; + } + } + + switch (v.getType()) { + case OBJECT: + case ARRAY: + return v.toJson(); + case STRING: + return v.getString(); + case LONG: + return v.getLong(); + case DOUBLE: + return v.getDouble(); + case DECIMAL: + return v.getDecimal(); + case BOOLEAN: + return v.getBoolean(); + case NULL: + return null; + default: + // todo: support other types + throw new IllegalArgumentException("Unsupported type: " + v.getType()); + } + } + + // Get a boolean value from the variant. + public boolean getBoolean() { + return GenericVariantUtil.getBoolean(value, pos); + } + + // Get a long value from the variant. + public long getLong() { + return GenericVariantUtil.getLong(value, pos); + } + + // Get a double value from the variant. + public double getDouble() { + return GenericVariantUtil.getDouble(value, pos); + } + + // Get a decimal value from the variant. + public BigDecimal getDecimal() { + return GenericVariantUtil.getDecimal(value, pos); + } + + // Get a float value from the variant. + public float getFloat() { + return GenericVariantUtil.getFloat(value, pos); + } + + // Get a binary value from the variant. + public byte[] getBinary() { + return GenericVariantUtil.getBinary(value, pos); + } + + // Get a string value from the variant. + public String getString() { + return GenericVariantUtil.getString(value, pos); + } + + // Get the type info bits from a variant value. + public int getTypeInfo() { + return GenericVariantUtil.getTypeInfo(value, pos); + } + + // Get the value type of the variant. + public Type getType() { + return GenericVariantUtil.getType(value, pos); + } + + // Get the number of object fields in the variant. + // It is only legal to call it when `getType()` is `Type.OBJECT`. + public int objectSize() { + return handleObject( + value, pos, (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> size); + } + + // Find the field value whose key is equal to `key`. Return null if the key is not found. + // It is only legal to call it when `getType()` is `Type.OBJECT`. + public GenericVariant getFieldByKey(String key) { + return handleObject( + value, + pos, + (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> { + // Use linear search for a short list. Switch to binary search when the length + // reaches `BINARY_SEARCH_THRESHOLD`. + if (size < BINARY_SEARCH_THRESHOLD) { + for (int i = 0; i < size; ++i) { + int id = readUnsigned(value, idStart + idSize * i, idSize); + if (key.equals(getMetadataKey(metadata, id))) { + int offset = + readUnsigned( + value, offsetStart + offsetSize * i, offsetSize); + return new GenericVariant(value, metadata, dataStart + offset); + } + } + } else { + int low = 0; + int high = size - 1; + while (low <= high) { + // Use unsigned right shift to compute the middle of `low` and `high`. + // This is not only a performance optimization, because it can properly + // handle the case where `low + high` overflows int. + int mid = (low + high) >>> 1; + int id = readUnsigned(value, idStart + idSize * mid, idSize); + int cmp = getMetadataKey(metadata, id).compareTo(key); + if (cmp < 0) { + low = mid + 1; + } else if (cmp > 0) { + high = mid - 1; + } else { + int offset = + readUnsigned( + value, offsetStart + offsetSize * mid, offsetSize); + return new GenericVariant(value, metadata, dataStart + offset); + } + } + } + return null; + }); + } + + /** Variant object field. */ + public static final class ObjectField { + public final String key; + public final Variant value; + + public ObjectField(String key, Variant value) { + this.key = key; + this.value = value; + } + } + + // Get the object field at the `index` slot. Return null if `index` is out of the bound of + // `[0, objectSize())`. + // It is only legal to call it when `getType()` is `Type.OBJECT`. + public ObjectField getFieldAtIndex(int index) { + return handleObject( + value, + pos, + (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> { + if (index < 0 || index >= size) { + return null; + } + int id = readUnsigned(value, idStart + idSize * index, idSize); + int offset = readUnsigned(value, offsetStart + offsetSize * index, offsetSize); + String key = getMetadataKey(metadata, id); + Variant v = new GenericVariant(value, metadata, dataStart + offset); + return new ObjectField(key, v); + }); + } + + // Get the dictionary ID for the object field at the `index` slot. Throws malformedVariant if + // `index` is out of the bound of `[0, objectSize())`. + // It is only legal to call it when `getType()` is `Type.OBJECT`. + public int getDictionaryIdAtIndex(int index) { + return handleObject( + value, + pos, + (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> { + if (index < 0 || index >= size) { + throw malformedVariant(); + } + return readUnsigned(value, idStart + idSize * index, idSize); + }); + } + + // Get the number of array elements in the variant. + // It is only legal to call it when `getType()` is `Type.ARRAY`. + public int arraySize() { + return handleArray(value, pos, (size, offsetSize, offsetStart, dataStart) -> size); + } + + // Get the array element at the `index` slot. Return null if `index` is out of the bound of + // `[0, arraySize())`. + // It is only legal to call it when `getType()` is `Type.ARRAY`. + public GenericVariant getElementAtIndex(int index) { + return handleArray( + value, + pos, + (size, offsetSize, offsetStart, dataStart) -> { + if (index < 0 || index >= size) { + return null; + } + int offset = readUnsigned(value, offsetStart + offsetSize * index, offsetSize); + return new GenericVariant(value, metadata, dataStart + offset); + }); + } + + // Escape a string so that it can be pasted into JSON structure. + // For example, if `str` only contains a new-line character, then the result content is "\n" + // (4 characters). + private static String escapeJson(String str) { + try (CharArrayWriter writer = new CharArrayWriter(); + JsonGenerator gen = new JsonFactory().createGenerator(writer)) { + gen.writeString(str); + gen.flush(); + return writer.toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + // A simplified and more performant version of `sb.append(escapeJson(str))`. It is used when we + // know `str` doesn't contain any special character that needs escaping. + static void appendQuoted(StringBuilder sb, String str) { + sb.append('"'); + sb.append(str); + sb.append('"'); + } + + private static final DateTimeFormatter TIMESTAMP_NTZ_FORMATTER = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(DateTimeFormatter.ISO_LOCAL_TIME) + .toFormatter(Locale.US); + + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + new DateTimeFormatterBuilder() + .append(TIMESTAMP_NTZ_FORMATTER) + .appendOffset("+HH:MM", "+00:00") + .toFormatter(Locale.US); + + private static Instant microsToInstant(long timestamp) { + return Instant.EPOCH.plus(timestamp, ChronoUnit.MICROS); + } + + private static void toJsonImpl( + byte[] value, byte[] metadata, int pos, StringBuilder sb, ZoneId zoneId) { + switch (GenericVariantUtil.getType(value, pos)) { + case OBJECT: + handleObject( + value, + pos, + (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> { + sb.append('{'); + for (int i = 0; i < size; ++i) { + int id = readUnsigned(value, idStart + idSize * i, idSize); + int offset = + readUnsigned( + value, offsetStart + offsetSize * i, offsetSize); + int elementPos = dataStart + offset; + if (i != 0) { + sb.append(','); + } + sb.append(escapeJson(getMetadataKey(metadata, id))); + sb.append(':'); + toJsonImpl(value, metadata, elementPos, sb, zoneId); + } + sb.append('}'); + return null; + }); + break; + case ARRAY: + handleArray( + value, + pos, + (size, offsetSize, offsetStart, dataStart) -> { + sb.append('['); + for (int i = 0; i < size; ++i) { + int offset = + readUnsigned( + value, offsetStart + offsetSize * i, offsetSize); + int elementPos = dataStart + offset; + if (i != 0) { + sb.append(','); + } + toJsonImpl(value, metadata, elementPos, sb, zoneId); + } + sb.append(']'); + return null; + }); + break; + case NULL: + sb.append("null"); + break; + case BOOLEAN: + sb.append(GenericVariantUtil.getBoolean(value, pos)); + break; + case LONG: + sb.append(GenericVariantUtil.getLong(value, pos)); + break; + case STRING: + sb.append(escapeJson(GenericVariantUtil.getString(value, pos))); + break; + case DOUBLE: + sb.append(GenericVariantUtil.getDouble(value, pos)); + break; + case DECIMAL: + sb.append(GenericVariantUtil.getDecimal(value, pos).toPlainString()); + break; + case DATE: + appendQuoted( + sb, + LocalDate.ofEpochDay((int) GenericVariantUtil.getLong(value, pos)) + .toString()); + break; + case TIMESTAMP: + appendQuoted( + sb, + TIMESTAMP_FORMATTER.format( + microsToInstant(GenericVariantUtil.getLong(value, pos)) + .atZone(zoneId))); + break; + case TIMESTAMP_NTZ: + appendQuoted( + sb, + TIMESTAMP_NTZ_FORMATTER.format( + microsToInstant(GenericVariantUtil.getLong(value, pos)) + .atZone(ZoneOffset.UTC))); + break; + case FLOAT: + sb.append(GenericVariantUtil.getFloat(value, pos)); + break; + case BINARY: + appendQuoted( + sb, + Base64.getEncoder() + .encodeToString(GenericVariantUtil.getBinary(value, pos))); + break; + } + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantBuilder.java b/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantBuilder.java new file mode 100644 index 000000000000..187fb9259e0e --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantBuilder.java @@ -0,0 +1,639 @@ +/* + * 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.data.variant; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonFactory; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonParseException; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonToken; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.exc.InputCoercionException; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; + +import static org.apache.paimon.data.variant.GenericVariantUtil.ARRAY; +import static org.apache.paimon.data.variant.GenericVariantUtil.BASIC_TYPE_MASK; +import static org.apache.paimon.data.variant.GenericVariantUtil.BINARY; +import static org.apache.paimon.data.variant.GenericVariantUtil.DATE; +import static org.apache.paimon.data.variant.GenericVariantUtil.DECIMAL16; +import static org.apache.paimon.data.variant.GenericVariantUtil.DECIMAL4; +import static org.apache.paimon.data.variant.GenericVariantUtil.DECIMAL8; +import static org.apache.paimon.data.variant.GenericVariantUtil.DOUBLE; +import static org.apache.paimon.data.variant.GenericVariantUtil.FALSE; +import static org.apache.paimon.data.variant.GenericVariantUtil.FLOAT; +import static org.apache.paimon.data.variant.GenericVariantUtil.INT1; +import static org.apache.paimon.data.variant.GenericVariantUtil.INT2; +import static org.apache.paimon.data.variant.GenericVariantUtil.INT4; +import static org.apache.paimon.data.variant.GenericVariantUtil.INT8; +import static org.apache.paimon.data.variant.GenericVariantUtil.LONG_STR; +import static org.apache.paimon.data.variant.GenericVariantUtil.MAX_DECIMAL16_PRECISION; +import static org.apache.paimon.data.variant.GenericVariantUtil.MAX_DECIMAL4_PRECISION; +import static org.apache.paimon.data.variant.GenericVariantUtil.MAX_DECIMAL8_PRECISION; +import static org.apache.paimon.data.variant.GenericVariantUtil.MAX_SHORT_STR_SIZE; +import static org.apache.paimon.data.variant.GenericVariantUtil.NULL; +import static org.apache.paimon.data.variant.GenericVariantUtil.OBJECT; +import static org.apache.paimon.data.variant.GenericVariantUtil.SIZE_LIMIT; +import static org.apache.paimon.data.variant.GenericVariantUtil.TIMESTAMP; +import static org.apache.paimon.data.variant.GenericVariantUtil.TIMESTAMP_NTZ; +import static org.apache.paimon.data.variant.GenericVariantUtil.TRUE; +import static org.apache.paimon.data.variant.GenericVariantUtil.U16_MAX; +import static org.apache.paimon.data.variant.GenericVariantUtil.U24_MAX; +import static org.apache.paimon.data.variant.GenericVariantUtil.U24_SIZE; +import static org.apache.paimon.data.variant.GenericVariantUtil.U32_SIZE; +import static org.apache.paimon.data.variant.GenericVariantUtil.U8_MAX; +import static org.apache.paimon.data.variant.GenericVariantUtil.VERSION; +import static org.apache.paimon.data.variant.GenericVariantUtil.arrayHeader; +import static org.apache.paimon.data.variant.GenericVariantUtil.checkIndex; +import static org.apache.paimon.data.variant.GenericVariantUtil.getMetadataKey; +import static org.apache.paimon.data.variant.GenericVariantUtil.handleArray; +import static org.apache.paimon.data.variant.GenericVariantUtil.handleObject; +import static org.apache.paimon.data.variant.GenericVariantUtil.objectHeader; +import static org.apache.paimon.data.variant.GenericVariantUtil.primitiveHeader; +import static org.apache.paimon.data.variant.GenericVariantUtil.readUnsigned; +import static org.apache.paimon.data.variant.GenericVariantUtil.shortStrHeader; +import static org.apache.paimon.data.variant.GenericVariantUtil.valueSize; +import static org.apache.paimon.data.variant.GenericVariantUtil.writeLong; + +/* This file is based on source code from the Spark Project (http://spark.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Build variant value and metadata by parsing JSON values. */ +public class GenericVariantBuilder { + public GenericVariantBuilder(boolean allowDuplicateKeys) { + this.allowDuplicateKeys = allowDuplicateKeys; + } + + /** + * Parse a JSON string as a Variant value. + * + * @throws IOException if any JSON parsing error happens. + */ + public static Variant parseJson(String json, boolean allowDuplicateKeys) throws IOException { + try (JsonParser parser = new JsonFactory().createParser(json)) { + parser.nextToken(); + return parseJson(parser, allowDuplicateKeys); + } + } + + /** + * Similar {@link #parseJson(String, boolean)}, but takes a JSON parser instead of string input. + */ + public static Variant parseJson(JsonParser parser, boolean allowDuplicateKeys) + throws IOException { + GenericVariantBuilder builder = new GenericVariantBuilder(allowDuplicateKeys); + builder.buildJson(parser); + return builder.result(); + } + + // Build the variant metadata from `dictionaryKeys` and return the variant result. + public Variant result() { + int numKeys = dictionaryKeys.size(); + // Use long to avoid overflow in accumulating lengths. + long dictionaryStringSize = 0; + for (byte[] key : dictionaryKeys) { + dictionaryStringSize += key.length; + } + // Determine the number of bytes required per offset entry. + // The largest offset is the one-past-the-end value, which is total string size. It's very + // unlikely that the number of keys could be larger, but incorporate that into the + // calculation in case of pathological data. + long maxSize = Math.max(dictionaryStringSize, numKeys); + if (maxSize > SIZE_LIMIT) { + throw new RuntimeException(); + } + int offsetSize = getIntegerSize((int) maxSize); + + int offsetStart = 1 + offsetSize; + int stringStart = offsetStart + (numKeys + 1) * offsetSize; + long metadataSize = stringStart + dictionaryStringSize; + + if (metadataSize > SIZE_LIMIT) { + throw new RuntimeException(); + } + byte[] metadata = new byte[(int) metadataSize]; + int headerByte = VERSION | ((offsetSize - 1) << 6); + writeLong(metadata, 0, headerByte, 1); + writeLong(metadata, 1, numKeys, offsetSize); + int currentOffset = 0; + for (int i = 0; i < numKeys; ++i) { + writeLong(metadata, offsetStart + i * offsetSize, currentOffset, offsetSize); + byte[] key = dictionaryKeys.get(i); + System.arraycopy(key, 0, metadata, stringStart + currentOffset, key.length); + currentOffset += key.length; + } + writeLong(metadata, offsetStart + numKeys * offsetSize, currentOffset, offsetSize); + return new GenericVariant(Arrays.copyOfRange(writeBuffer, 0, writePos), metadata); + } + + // Return the variant value only, without metadata. + // Used in shredding to produce a final value, where all shredded values refer to a common + // metadata. It is expected to be called instead of `result()`, although it is valid to call + // both + // methods, in any order. + public byte[] valueWithoutMetadata() { + return Arrays.copyOfRange(writeBuffer, 0, writePos); + } + + public void appendString(String str) { + byte[] text = str.getBytes(StandardCharsets.UTF_8); + boolean longStr = text.length > MAX_SHORT_STR_SIZE; + checkCapacity((longStr ? 1 + U32_SIZE : 1) + text.length); + if (longStr) { + writeBuffer[writePos++] = primitiveHeader(LONG_STR); + writeLong(writeBuffer, writePos, text.length, U32_SIZE); + writePos += U32_SIZE; + } else { + writeBuffer[writePos++] = shortStrHeader(text.length); + } + System.arraycopy(text, 0, writeBuffer, writePos, text.length); + writePos += text.length; + } + + public void appendNull() { + checkCapacity(1); + writeBuffer[writePos++] = primitiveHeader(NULL); + } + + public void appendBoolean(boolean b) { + checkCapacity(1); + writeBuffer[writePos++] = primitiveHeader(b ? TRUE : FALSE); + } + + // Append a long value to the variant builder. The actual used integer type depends on the value + // range of the long value. + public void appendLong(long l) { + checkCapacity(1 + 8); + if (l == (byte) l) { + writeBuffer[writePos++] = primitiveHeader(INT1); + writeLong(writeBuffer, writePos, l, 1); + writePos += 1; + } else if (l == (short) l) { + writeBuffer[writePos++] = primitiveHeader(INT2); + writeLong(writeBuffer, writePos, l, 2); + writePos += 2; + } else if (l == (int) l) { + writeBuffer[writePos++] = primitiveHeader(INT4); + writeLong(writeBuffer, writePos, l, 4); + writePos += 4; + } else { + writeBuffer[writePos++] = primitiveHeader(INT8); + writeLong(writeBuffer, writePos, l, 8); + writePos += 8; + } + } + + public void appendDouble(double d) { + checkCapacity(1 + 8); + writeBuffer[writePos++] = primitiveHeader(DOUBLE); + writeLong(writeBuffer, writePos, Double.doubleToLongBits(d), 8); + writePos += 8; + } + + // Append a decimal value to the variant builder. The caller should guarantee that its precision + // and scale fit into `MAX_DECIMAL16_PRECISION`. + public void appendDecimal(BigDecimal d) { + checkCapacity(2 + 16); + BigInteger unscaled = d.unscaledValue(); + if (d.scale() <= MAX_DECIMAL4_PRECISION && d.precision() <= MAX_DECIMAL4_PRECISION) { + writeBuffer[writePos++] = primitiveHeader(DECIMAL4); + writeBuffer[writePos++] = (byte) d.scale(); + writeLong(writeBuffer, writePos, unscaled.intValueExact(), 4); + writePos += 4; + } else if (d.scale() <= MAX_DECIMAL8_PRECISION && d.precision() <= MAX_DECIMAL8_PRECISION) { + writeBuffer[writePos++] = primitiveHeader(DECIMAL8); + writeBuffer[writePos++] = (byte) d.scale(); + writeLong(writeBuffer, writePos, unscaled.longValueExact(), 8); + writePos += 8; + } else { + assert d.scale() <= MAX_DECIMAL16_PRECISION && d.precision() <= MAX_DECIMAL16_PRECISION; + writeBuffer[writePos++] = primitiveHeader(DECIMAL16); + writeBuffer[writePos++] = (byte) d.scale(); + // `toByteArray` returns a big-endian representation. We need to copy it reversely and + // sign extend it to 16 bytes. + byte[] bytes = unscaled.toByteArray(); + for (int i = 0; i < bytes.length; ++i) { + writeBuffer[writePos + i] = bytes[bytes.length - 1 - i]; + } + byte sign = (byte) (bytes[0] < 0 ? -1 : 0); + for (int i = bytes.length; i < 16; ++i) { + writeBuffer[writePos + i] = sign; + } + writePos += 16; + } + } + + public void appendDate(int daysSinceEpoch) { + checkCapacity(1 + 4); + writeBuffer[writePos++] = primitiveHeader(DATE); + writeLong(writeBuffer, writePos, daysSinceEpoch, 4); + writePos += 4; + } + + public void appendTimestamp(long microsSinceEpoch) { + checkCapacity(1 + 8); + writeBuffer[writePos++] = primitiveHeader(TIMESTAMP); + writeLong(writeBuffer, writePos, microsSinceEpoch, 8); + writePos += 8; + } + + public void appendTimestampNtz(long microsSinceEpoch) { + checkCapacity(1 + 8); + writeBuffer[writePos++] = primitiveHeader(TIMESTAMP_NTZ); + writeLong(writeBuffer, writePos, microsSinceEpoch, 8); + writePos += 8; + } + + public void appendFloat(float f) { + checkCapacity(1 + 4); + writeBuffer[writePos++] = primitiveHeader(FLOAT); + writeLong(writeBuffer, writePos, Float.floatToIntBits(f), 8); + writePos += 4; + } + + public void appendBinary(byte[] binary) { + checkCapacity(1 + U32_SIZE + binary.length); + writeBuffer[writePos++] = primitiveHeader(BINARY); + writeLong(writeBuffer, writePos, binary.length, U32_SIZE); + writePos += U32_SIZE; + System.arraycopy(binary, 0, writeBuffer, writePos, binary.length); + writePos += binary.length; + } + + // Add a key to the variant dictionary. If the key already exists, the dictionary is not + // modified. + // In either case, return the id of the key. + public int addKey(String key) { + int id; + if (dictionary.containsKey(key)) { + id = dictionary.get(key); + } else { + id = dictionaryKeys.size(); + dictionary.put(key, id); + dictionaryKeys.add(key.getBytes(StandardCharsets.UTF_8)); + } + return id; + } + + // Return the current write position of the variant builder. It is used together with + // `finishWritingObject` or `finishWritingArray`. + public int getWritePos() { + return writePos; + } + + // Finish writing a variant object after all of its fields have already been written. The + // process + // is as follows: + // 1. The caller calls `getWritePos` before writing any fields to obtain the `start` parameter. + // 2. The caller appends all the object fields to the builder. In the meantime, it should + // maintain + // the `fields` parameter. Before appending each field, it should append an entry to `fields` to + // record the offset of the field. The offset is computed as `getWritePos() - start`. + // 3. The caller calls `finishWritingObject` to finish writing a variant object. + // + // This function is responsible to sort the fields by key. If there are duplicate field keys: + // - when `allowDuplicateKeys` is true, the field with the greatest offset value (the last + // appended one) is kept. + // - otherwise, throw an exception. + public void finishWritingObject(int start, ArrayList fields) { + int size = fields.size(); + Collections.sort(fields); + int maxId = size == 0 ? 0 : fields.get(0).id; + if (allowDuplicateKeys) { + int distinctPos = 0; + // Maintain a list of distinct keys in-place. + for (int i = 1; i < size; ++i) { + maxId = Math.max(maxId, fields.get(i).id); + if (fields.get(i).id == fields.get(i - 1).id) { + // Found a duplicate key. Keep the field with a greater offset. + if (fields.get(distinctPos).offset < fields.get(i).offset) { + fields.set( + distinctPos, + fields.get(distinctPos).withNewOffset(fields.get(i).offset)); + } + } else { + // Found a distinct key. Add the field to the list. + ++distinctPos; + fields.set(distinctPos, fields.get(i)); + } + } + if (distinctPos + 1 < fields.size()) { + size = distinctPos + 1; + // Resize `fields` to `size`. + fields.subList(size, fields.size()).clear(); + // Sort the fields by offsets so that we can move the value data of each field to + // the new offset without overwriting the fields after it. + fields.sort(Comparator.comparingInt(f -> f.offset)); + int currentOffset = 0; + for (int i = 0; i < size; ++i) { + int oldOffset = fields.get(i).offset; + int fieldSize = GenericVariantUtil.valueSize(writeBuffer, start + oldOffset); + System.arraycopy( + writeBuffer, + start + oldOffset, + writeBuffer, + start + currentOffset, + fieldSize); + fields.set(i, fields.get(i).withNewOffset(currentOffset)); + currentOffset += fieldSize; + } + writePos = start + currentOffset; + // Change back to the sort order by field keys to meet the variant spec. + Collections.sort(fields); + } + } else { + for (int i = 1; i < size; ++i) { + maxId = Math.max(maxId, fields.get(i).id); + String key = fields.get(i).key; + if (key.equals(fields.get(i - 1).key)) { + throw new RuntimeException("VARIANT_DUPLICATE_KEY"); + } + } + } + int dataSize = writePos - start; + boolean largeSize = size > U8_MAX; + int sizeBytes = largeSize ? U32_SIZE : 1; + int idSize = getIntegerSize(maxId); + int offsetSize = getIntegerSize(dataSize); + // The space for header byte, object size, id list, and offset list. + int headerSize = 1 + sizeBytes + size * idSize + (size + 1) * offsetSize; + checkCapacity(headerSize); + // Shift the just-written field data to make room for the object header section. + System.arraycopy(writeBuffer, start, writeBuffer, start + headerSize, dataSize); + writePos += headerSize; + writeBuffer[start] = objectHeader(largeSize, idSize, offsetSize); + writeLong(writeBuffer, start + 1, size, sizeBytes); + int idStart = start + 1 + sizeBytes; + int offsetStart = idStart + size * idSize; + for (int i = 0; i < size; ++i) { + writeLong(writeBuffer, idStart + i * idSize, fields.get(i).id, idSize); + writeLong(writeBuffer, offsetStart + i * offsetSize, fields.get(i).offset, offsetSize); + } + writeLong(writeBuffer, offsetStart + size * offsetSize, dataSize, offsetSize); + } + + // Finish writing a variant array after all of its elements have already been written. The + // process is similar to that of `finishWritingObject`. + public void finishWritingArray(int start, ArrayList offsets) { + int dataSize = writePos - start; + int size = offsets.size(); + boolean largeSize = size > U8_MAX; + int sizeBytes = largeSize ? U32_SIZE : 1; + int offsetSize = getIntegerSize(dataSize); + // The space for header byte, object size, and offset list. + int headerSize = 1 + sizeBytes + (size + 1) * offsetSize; + checkCapacity(headerSize); + // Shift the just-written field data to make room for the header section. + System.arraycopy(writeBuffer, start, writeBuffer, start + headerSize, dataSize); + writePos += headerSize; + writeBuffer[start] = arrayHeader(largeSize, offsetSize); + writeLong(writeBuffer, start + 1, size, sizeBytes); + int offsetStart = start + 1 + sizeBytes; + for (int i = 0; i < size; ++i) { + writeLong(writeBuffer, offsetStart + i * offsetSize, offsets.get(i), offsetSize); + } + writeLong(writeBuffer, offsetStart + size * offsetSize, dataSize, offsetSize); + } + + // Append a variant value to the variant builder. We need to insert the keys in the input + // variant into the current variant dictionary and rebuild it with new field ids. For scalar + // values in the input variant, we can directly copy the binary slice. + public void appendVariant(GenericVariant v) { + appendVariantImpl(v.rawValue(), v.metadata(), v.pos()); + } + + private void appendVariantImpl(byte[] value, byte[] metadata, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + switch (basicType) { + case OBJECT: + handleObject( + value, + pos, + (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> { + ArrayList fields = new ArrayList<>(size); + int start = writePos; + for (int i = 0; i < size; ++i) { + int id = readUnsigned(value, idStart + idSize * i, idSize); + int offset = + readUnsigned( + value, offsetStart + offsetSize * i, offsetSize); + int elementPos = dataStart + offset; + String key = getMetadataKey(metadata, id); + int newId = addKey(key); + fields.add(new FieldEntry(key, newId, writePos - start)); + appendVariantImpl(value, metadata, elementPos); + } + finishWritingObject(start, fields); + return null; + }); + break; + case ARRAY: + handleArray( + value, + pos, + (size, offsetSize, offsetStart, dataStart) -> { + ArrayList offsets = new ArrayList<>(size); + int start = writePos; + for (int i = 0; i < size; ++i) { + int offset = + readUnsigned( + value, offsetStart + offsetSize * i, offsetSize); + int elementPos = dataStart + offset; + offsets.add(writePos - start); + appendVariantImpl(value, metadata, elementPos); + } + finishWritingArray(start, offsets); + return null; + }); + break; + default: + shallowAppendVariantImpl(value, pos); + break; + } + } + + // Append the variant value without rewriting or creating any metadata. This is used when + // building an object during shredding, where there is a fixed pre-existing metadata that + // all shredded values will refer to. + public void shallowAppendVariant(GenericVariant v) { + shallowAppendVariantImpl(v.rawValue(), v.pos()); + } + + private void shallowAppendVariantImpl(byte[] value, int pos) { + int size = valueSize(value, pos); + checkIndex(pos + size - 1, value.length); + checkCapacity(size); + System.arraycopy(value, pos, writeBuffer, writePos, size); + writePos += size; + } + + private void checkCapacity(int additional) { + int required = writePos + additional; + if (required > writeBuffer.length) { + // Allocate a new buffer with a capacity of the next power of 2 of `required`. + int newCapacity = Integer.highestOneBit(required); + newCapacity = newCapacity < required ? newCapacity * 2 : newCapacity; + if (newCapacity > SIZE_LIMIT) { + throw new RuntimeException(); + } + byte[] newValue = new byte[newCapacity]; + System.arraycopy(writeBuffer, 0, newValue, 0, writePos); + writeBuffer = newValue; + } + } + + /** + * Temporarily store the information of a field. We need to collect all fields in an JSON + * object, sort them by their keys, and build the variant object in sorted order. + */ + public static final class FieldEntry implements Comparable { + final String key; + final int id; + final int offset; + + public FieldEntry(String key, int id, int offset) { + this.key = key; + this.id = id; + this.offset = offset; + } + + FieldEntry withNewOffset(int newOffset) { + return new FieldEntry(key, id, newOffset); + } + + @Override + public int compareTo(FieldEntry other) { + return key.compareTo(other.key); + } + } + + private void buildJson(JsonParser parser) throws IOException { + JsonToken token = parser.currentToken(); + if (token == null) { + throw new JsonParseException(parser, "Unexpected null token"); + } + switch (token) { + case START_OBJECT: + { + ArrayList fields = new ArrayList<>(); + int start = writePos; + while (parser.nextToken() != JsonToken.END_OBJECT) { + String key = parser.currentName(); + parser.nextToken(); + int id = addKey(key); + fields.add(new FieldEntry(key, id, writePos - start)); + buildJson(parser); + } + finishWritingObject(start, fields); + break; + } + case START_ARRAY: + { + ArrayList offsets = new ArrayList<>(); + int start = writePos; + while (parser.nextToken() != JsonToken.END_ARRAY) { + offsets.add(writePos - start); + buildJson(parser); + } + finishWritingArray(start, offsets); + break; + } + case VALUE_STRING: + appendString(parser.getText()); + break; + case VALUE_NUMBER_INT: + try { + appendLong(parser.getLongValue()); + } catch (InputCoercionException ignored) { + // If the value doesn't fit any integer type, parse it as decimal or floating + // instead. + parseFloatingPoint(parser); + } + break; + case VALUE_NUMBER_FLOAT: + parseFloatingPoint(parser); + break; + case VALUE_TRUE: + appendBoolean(true); + break; + case VALUE_FALSE: + appendBoolean(false); + break; + case VALUE_NULL: + appendNull(); + break; + default: + throw new JsonParseException(parser, "Unexpected token " + token); + } + } + + // Choose the smallest unsigned integer type that can store `value`. It must be within + // `[0, U24_MAX]`. + private int getIntegerSize(int value) { + assert value >= 0 && value <= U24_MAX; + if (value <= U8_MAX) { + return 1; + } + if (value <= U16_MAX) { + return 2; + } + return U24_SIZE; + } + + private void parseFloatingPoint(JsonParser parser) throws IOException { + if (!tryParseDecimal(parser.getText())) { + appendDouble(parser.getDoubleValue()); + } + } + + // Try to parse a JSON number as a decimal. Return whether the parsing succeeds. The input must + // only use the decimal format (an integer value with an optional '.' in it) and must not use + // scientific notation. It also must fit into the precision limitation of decimal types. + private boolean tryParseDecimal(String input) { + for (int i = 0; i < input.length(); ++i) { + char ch = input.charAt(i); + if (ch != '-' && ch != '.' && !(ch >= '0' && ch <= '9')) { + return false; + } + } + BigDecimal d = new BigDecimal(input); + if (d.scale() <= MAX_DECIMAL16_PRECISION && d.precision() <= MAX_DECIMAL16_PRECISION) { + appendDecimal(d); + return true; + } + return false; + } + + // The write buffer in building the variant value. Its first `writePos` bytes has been written. + private byte[] writeBuffer = new byte[128]; + private int writePos = 0; + // Map keys to a monotonically increasing id. + private final HashMap dictionary = new HashMap<>(); + // Store all keys in `dictionary` in the order of id. + private final ArrayList dictionaryKeys = new ArrayList<>(); + private final boolean allowDuplicateKeys; +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantUtil.java b/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantUtil.java new file mode 100644 index 000000000000..b37cbd7f6f29 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/variant/GenericVariantUtil.java @@ -0,0 +1,628 @@ +/* + * 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.data.variant; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Arrays; + +/* This file is based on source code from the Spark Project (http://spark.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * This class defines constants related to the variant format and provides functions for + * manipulating variant binaries. + * + *

A variant is made up of 2 binaries: value and metadata. A variant value consists of a one-byte + * header and a number of content bytes (can be zero). The header byte is divided into upper 6 bits + * (called "type info") and lower 2 bits (called "basic type"). The content format is explained in + * the below constants for all possible basic type and type info values. + * + *

The variant metadata includes a version id and a dictionary of distinct strings + * (case-sensitive). Its binary format is: - Version: 1-byte unsigned integer. The only acceptable + * value is 1 currently. - Dictionary size: 4-byte little-endian unsigned integer. The number of + * keys in the dictionary. - Offsets: (size + 1) * 4-byte little-endian unsigned integers. + * `offsets[i]` represents the starting position of string i, counting starting from the address of + * `offsets[0]`. Strings must be stored contiguously, so we don’t need to store the string size, + * instead, we compute it with `offset[i + 1] - offset[i]`. - UTF-8 string data. + */ +public class GenericVariantUtil { + public static final int BASIC_TYPE_BITS = 2; + public static final int BASIC_TYPE_MASK = 0x3; + public static final int TYPE_INFO_MASK = 0x3F; + // The inclusive maximum value of the type info value. It is the size limit of `SHORT_STR`. + public static final int MAX_SHORT_STR_SIZE = 0x3F; + + // Below is all possible basic type values. + // Primitive value. The type info value must be one of the values in the below section. + public static final int PRIMITIVE = 0; + // Short string value. The type info value is the string size, which must be in `[0, + // kMaxShortStrSize]`. + // The string content bytes directly follow the header byte. + public static final int SHORT_STR = 1; + // Object value. The content contains a size, a list of field ids, a list of field offsets, and + // the actual field data. The length of the id list is `size`, while the length of the offset + // list is `size + 1`, where the last offset represent the total size of the field data. The + // fields in an object must be sorted by the field name in alphabetical order. Duplicate field + // names in one object are not allowed. + // We use 5 bits in the type info to specify the integer type of the object header: it should + // be 0_b4_b3b2_b1b0 (MSB is 0), where: + // - b4 specifies the type of size. When it is 0/1, `size` is a little-endian 1/4-byte + // unsigned integer. + // - b3b2/b1b0 specifies the integer type of id and offset. When the 2 bits are 0/1/2, the + // list contains 1/2/3-byte little-endian unsigned integers. + public static final int OBJECT = 2; + // Array value. The content contains a size, a list of field offsets, and the actual element + // data. It is similar to an object without the id list. The length of the offset list + // is `size + 1`, where the last offset represent the total size of the element data. + // Its type info should be: 000_b2_b1b0: + // - b2 specifies the type of size. + // - b1b0 specifies the integer type of offset. + public static final int ARRAY = 3; + + // Below is all possible type info values for `PRIMITIVE`. + // JSON Null value. Empty content. + public static final int NULL = 0; + // True value. Empty content. + public static final int TRUE = 1; + // False value. Empty content. + public static final int FALSE = 2; + // 1-byte little-endian signed integer. + public static final int INT1 = 3; + // 2-byte little-endian signed integer. + public static final int INT2 = 4; + // 4-byte little-endian signed integer. + public static final int INT4 = 5; + // 4-byte little-endian signed integer. + public static final int INT8 = 6; + // 8-byte IEEE double. + public static final int DOUBLE = 7; + // 4-byte decimal. Content is 1-byte scale + 4-byte little-endian signed integer. + public static final int DECIMAL4 = 8; + // 8-byte decimal. Content is 1-byte scale + 8-byte little-endian signed integer. + public static final int DECIMAL8 = 9; + // 16-byte decimal. Content is 1-byte scale + 16-byte little-endian signed integer. + public static final int DECIMAL16 = 10; + // Date value. Content is 4-byte little-endian signed integer that represents the number of days + // from the Unix epoch. + public static final int DATE = 11; + // Timestamp value. Content is 8-byte little-endian signed integer that represents the number of + // microseconds elapsed since the Unix epoch, 1970-01-01 00:00:00 UTC. It is displayed to users + // in + // their local time zones and may be displayed differently depending on the execution + // environment. + public static final int TIMESTAMP = 12; + // Timestamp_ntz value. It has the same content as `TIMESTAMP` but should always be interpreted + // as if the local time zone is UTC. + public static final int TIMESTAMP_NTZ = 13; + // 4-byte IEEE float. + public static final int FLOAT = 14; + // Binary value. The content is (4-byte little-endian unsigned integer representing the binary + // size) + (size bytes of binary content). + public static final int BINARY = 15; + // Long string value. The content is (4-byte little-endian unsigned integer representing the + // string size) + (size bytes of string content). + public static final int LONG_STR = 16; + + public static final byte VERSION = 1; + // The lower 4 bits of the first metadata byte contain the version. + public static final byte VERSION_MASK = 0x0F; + + public static final int U8_MAX = 0xFF; + public static final int U16_MAX = 0xFFFF; + public static final int U24_MAX = 0xFFFFFF; + public static final int U24_SIZE = 3; + public static final int U32_SIZE = 4; + + // Both variant value and variant metadata need to be no longer than 16MiB. + public static final int SIZE_LIMIT = U24_MAX + 1; + + public static final int MAX_DECIMAL4_PRECISION = 9; + public static final int MAX_DECIMAL8_PRECISION = 18; + public static final int MAX_DECIMAL16_PRECISION = 38; + + public static final int BINARY_SEARCH_THRESHOLD = 32; + + // Write the least significant `numBytes` bytes in `value` into `bytes[pos, pos + numBytes)` in + // little endian. + public static void writeLong(byte[] bytes, int pos, long value, int numBytes) { + for (int i = 0; i < numBytes; ++i) { + bytes[pos + i] = (byte) ((value >>> (8 * i)) & 0xFF); + } + } + + public static byte primitiveHeader(int type) { + return (byte) (type << 2 | PRIMITIVE); + } + + public static byte shortStrHeader(int size) { + return (byte) (size << 2 | SHORT_STR); + } + + public static byte objectHeader(boolean largeSize, int idSize, int offsetSize) { + return (byte) + (((largeSize ? 1 : 0) << (BASIC_TYPE_BITS + 4)) + | ((idSize - 1) << (BASIC_TYPE_BITS + 2)) + | ((offsetSize - 1) << BASIC_TYPE_BITS) + | OBJECT); + } + + public static byte arrayHeader(boolean largeSize, int offsetSize) { + return (byte) + (((largeSize ? 1 : 0) << (BASIC_TYPE_BITS + 2)) + | ((offsetSize - 1) << BASIC_TYPE_BITS) + | ARRAY); + } + + // An exception indicating that the variant value or metadata doesn't + static RuntimeException malformedVariant() { + return new RuntimeException("MALFORMED_VARIANT"); + } + + static RuntimeException unknownPrimitiveTypeInVariant(int id) { + return new RuntimeException("UNKNOWN_PRIMITIVE_TYPE_IN_VARIANT, id: " + id); + } + + // An exception indicating that an external caller tried to call the Variant constructor with + // value or metadata exceeding the 16MiB size limit. We will never construct a Variant this + // large, + // so it should only be possible to encounter this exception when reading a Variant produced by + // another tool. + static RuntimeException variantConstructorSizeLimit() { + return new RuntimeException("VARIANT_CONSTRUCTOR_SIZE_LIMIT"); + } + + // Check the validity of an array index `pos`. Throw `MALFORMED_VARIANT` if it is out of bound, + // meaning that the variant is malformed. + static void checkIndex(int pos, int length) { + if (pos < 0 || pos >= length) { + throw malformedVariant(); + } + } + + // Read a little-endian signed long value from `bytes[pos, pos + numBytes)`. + static long readLong(byte[] bytes, int pos, int numBytes) { + checkIndex(pos, bytes.length); + checkIndex(pos + numBytes - 1, bytes.length); + long result = 0; + // All bytes except the most significant byte should be unsign-extended and shifted (so we + // need `& 0xFF`). The most significant byte should be sign-extended and is handled after + // the loop. + for (int i = 0; i < numBytes - 1; ++i) { + long unsignedByteValue = bytes[pos + i] & 0xFF; + result |= unsignedByteValue << (8 * i); + } + long signedByteValue = bytes[pos + numBytes - 1]; + result |= signedByteValue << (8 * (numBytes - 1)); + return result; + } + + // Read a little-endian unsigned int value from `bytes[pos, pos + numBytes)`. The value must fit + // into a non-negative int (`[0, Integer.MAX_VALUE]`). + static int readUnsigned(byte[] bytes, int pos, int numBytes) { + checkIndex(pos, bytes.length); + checkIndex(pos + numBytes - 1, bytes.length); + int result = 0; + // Similar to the `readLong` loop, but all bytes should be unsign-extended. + for (int i = 0; i < numBytes; ++i) { + int unsignedByteValue = bytes[pos + i] & 0xFF; + result |= unsignedByteValue << (8 * i); + } + if (result < 0) { + throw malformedVariant(); + } + return result; + } + + /** + * The value type of variant value. It is determined by the header byte but not a 1:1 mapping + * (for example, INT1/2/4/8 all maps to `Type.LONG`). + */ + public enum Type { + OBJECT, + ARRAY, + NULL, + BOOLEAN, + LONG, + STRING, + DOUBLE, + DECIMAL, + DATE, + TIMESTAMP, + TIMESTAMP_NTZ, + FLOAT, + BINARY + } + + public static int getTypeInfo(byte[] value, int pos) { + checkIndex(pos, value.length); + return (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + } + + // Get the value type of variant value `value[pos...]`. It is only legal to call `get*` if + // `getType` returns this type (for example, it is only legal to call `getLong` if `getType` + // returns `Type.Long`). + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static Type getType(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + switch (basicType) { + case SHORT_STR: + return Type.STRING; + case OBJECT: + return Type.OBJECT; + case ARRAY: + return Type.ARRAY; + default: + switch (typeInfo) { + case NULL: + return Type.NULL; + case TRUE: + case FALSE: + return Type.BOOLEAN; + case INT1: + case INT2: + case INT4: + case INT8: + return Type.LONG; + case DOUBLE: + return Type.DOUBLE; + case DECIMAL4: + case DECIMAL8: + case DECIMAL16: + return Type.DECIMAL; + case DATE: + return Type.DATE; + case TIMESTAMP: + return Type.TIMESTAMP; + case TIMESTAMP_NTZ: + return Type.TIMESTAMP_NTZ; + case FLOAT: + return Type.FLOAT; + case BINARY: + return Type.BINARY; + case LONG_STR: + return Type.STRING; + default: + throw unknownPrimitiveTypeInVariant(typeInfo); + } + } + } + + // Compute the size in bytes of the variant value `value[pos...]`. `value.length - pos` is an + // upper bound of the size, but the actual size can be smaller. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static int valueSize(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + switch (basicType) { + case SHORT_STR: + return 1 + typeInfo; + case OBJECT: + return handleObject( + value, + pos, + (size, idSize, offsetSize, idStart, offsetStart, dataStart) -> + dataStart + - pos + + readUnsigned( + value, + offsetStart + size * offsetSize, + offsetSize)); + case ARRAY: + return handleArray( + value, + pos, + (size, offsetSize, offsetStart, dataStart) -> + dataStart + - pos + + readUnsigned( + value, + offsetStart + size * offsetSize, + offsetSize)); + default: + switch (typeInfo) { + case NULL: + case TRUE: + case FALSE: + return 1; + case INT1: + return 2; + case INT2: + return 3; + case INT4: + case DATE: + case FLOAT: + return 5; + case INT8: + case DOUBLE: + case TIMESTAMP: + case TIMESTAMP_NTZ: + return 9; + case DECIMAL4: + return 6; + case DECIMAL8: + return 10; + case DECIMAL16: + return 18; + case BINARY: + case LONG_STR: + return 1 + U32_SIZE + readUnsigned(value, pos + 1, U32_SIZE); + default: + throw unknownPrimitiveTypeInVariant(typeInfo); + } + } + } + + static IllegalStateException unexpectedType(Type type) { + return new IllegalStateException("Expect type to be " + type); + } + + // Get a boolean value from variant value `value[pos...]`. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static boolean getBoolean(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != PRIMITIVE || (typeInfo != TRUE && typeInfo != FALSE)) { + throw unexpectedType(Type.BOOLEAN); + } + return typeInfo == TRUE; + } + + // Get a long value from variant value `value[pos...]`. + // It is only legal to call it if `getType` returns one of `Type.LONG/DATE/TIMESTAMP/ + // TIMESTAMP_NTZ`. If the type is `DATE`, the return value is guaranteed to fit into an int and + // represents the number of days from the Unix epoch. + // If the type is `TIMESTAMP/TIMESTAMP_NTZ`, the return value represents the number of + // microseconds from the Unix epoch. + public static long getLong(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + String exceptionMessage = "Expect type to be LONG/DATE/TIMESTAMP/TIMESTAMP_NTZ"; + if (basicType != PRIMITIVE) { + throw new IllegalStateException(exceptionMessage); + } + switch (typeInfo) { + case INT1: + return readLong(value, pos + 1, 1); + case INT2: + return readLong(value, pos + 1, 2); + case INT4: + case DATE: + return readLong(value, pos + 1, 4); + case INT8: + case TIMESTAMP: + case TIMESTAMP_NTZ: + return readLong(value, pos + 1, 8); + default: + throw new IllegalStateException(exceptionMessage); + } + } + + // Get a double value from variant value `value[pos...]`. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static double getDouble(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != PRIMITIVE || typeInfo != DOUBLE) { + throw unexpectedType(Type.DOUBLE); + } + return Double.longBitsToDouble(readLong(value, pos + 1, 8)); + } + + // Check whether the precision and scale of the decimal are within the limit. + private static void checkDecimal(BigDecimal d, int maxPrecision) { + if (d.precision() > maxPrecision || d.scale() > maxPrecision) { + throw malformedVariant(); + } + } + + // Get a decimal value from variant value `value[pos...]`. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static BigDecimal getDecimalWithOriginalScale(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != PRIMITIVE) { + throw unexpectedType(Type.DECIMAL); + } + // Interpret the scale byte as unsigned. If it is a negative byte, the unsigned value must + // be greater than `MAX_DECIMAL16_PRECISION` and will trigger an error in `checkDecimal`. + int scale = value[pos + 1] & 0xFF; + BigDecimal result; + switch (typeInfo) { + case DECIMAL4: + result = BigDecimal.valueOf(readLong(value, pos + 2, 4), scale); + checkDecimal(result, MAX_DECIMAL4_PRECISION); + break; + case DECIMAL8: + result = BigDecimal.valueOf(readLong(value, pos + 2, 8), scale); + checkDecimal(result, MAX_DECIMAL8_PRECISION); + break; + case DECIMAL16: + checkIndex(pos + 17, value.length); + byte[] bytes = new byte[16]; + // Copy the bytes reversely because the `BigInteger` constructor expects a + // big-endian representation. + for (int i = 0; i < 16; ++i) { + bytes[i] = value[pos + 17 - i]; + } + result = new BigDecimal(new BigInteger(bytes), scale); + checkDecimal(result, MAX_DECIMAL16_PRECISION); + break; + default: + throw unexpectedType(Type.DECIMAL); + } + return result; + } + + public static BigDecimal getDecimal(byte[] value, int pos) { + return getDecimalWithOriginalScale(value, pos).stripTrailingZeros(); + } + + // Get a float value from variant value `value[pos...]`. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static float getFloat(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != PRIMITIVE || typeInfo != FLOAT) { + throw unexpectedType(Type.FLOAT); + } + return Float.intBitsToFloat((int) readLong(value, pos + 1, 4)); + } + + // Get a binary value from variant value `value[pos...]`. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static byte[] getBinary(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != PRIMITIVE || typeInfo != BINARY) { + throw unexpectedType(Type.BINARY); + } + int start = pos + 1 + U32_SIZE; + int length = readUnsigned(value, pos + 1, U32_SIZE); + checkIndex(start + length - 1, value.length); + return Arrays.copyOfRange(value, start, start + length); + } + + // Get a string value from variant value `value[pos...]`. + // Throw `MALFORMED_VARIANT` if the variant is malformed. + public static String getString(byte[] value, int pos) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType == SHORT_STR || (basicType == PRIMITIVE && typeInfo == LONG_STR)) { + int start; + int length; + if (basicType == SHORT_STR) { + start = pos + 1; + length = typeInfo; + } else { + start = pos + 1 + U32_SIZE; + length = readUnsigned(value, pos + 1, U32_SIZE); + } + checkIndex(start + length - 1, value.length); + return new String(value, start, length); + } + throw unexpectedType(Type.STRING); + } + + /** 1. */ + public interface ObjectHandler { + /** + * @param size Number of object fields. + * @param idSize The integer size of the field id list. + * @param offsetSize The integer size of the offset list. + * @param idStart The starting index of the field id list in the variant value array. + * @param offsetStart The starting index of the offset list in the variant value array. + * @param dataStart The starting index of field data in the variant value array. + */ + T apply(int size, int idSize, int offsetSize, int idStart, int offsetStart, int dataStart); + } + + // A helper function to access a variant object. It provides `handler` with its required + // parameters and returns what it returns. + public static T handleObject(byte[] value, int pos, ObjectHandler handler) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != OBJECT) { + throw unexpectedType(Type.OBJECT); + } + // Refer to the comment of the `OBJECT` constant for the details of the object header + // encoding. Suppose `typeInfo` has a bit representation of 0_b4_b3b2_b1b0, the following + // line extracts b4 to determine whether the object uses a 1/4-byte size. + boolean largeSize = ((typeInfo >> 4) & 0x1) != 0; + int sizeBytes = (largeSize ? U32_SIZE : 1); + int size = readUnsigned(value, pos + 1, sizeBytes); + // Extracts b3b2 to determine the integer size of the field id list. + int idSize = ((typeInfo >> 2) & 0x3) + 1; + // Extracts b1b0 to determine the integer size of the offset list. + int offsetSize = (typeInfo & 0x3) + 1; + int idStart = pos + 1 + sizeBytes; + int offsetStart = idStart + size * idSize; + int dataStart = offsetStart + (size + 1) * offsetSize; + return handler.apply(size, idSize, offsetSize, idStart, offsetStart, dataStart); + } + + /** 1. */ + public interface ArrayHandler { + /** + * @param size Number of array elements. + * @param offsetSize The integer size of the offset list. + * @param offsetStart The starting index of the offset list in the variant value array. + * @param dataStart The starting index of element data in the variant value array. + */ + T apply(int size, int offsetSize, int offsetStart, int dataStart); + } + + // A helper function to access a variant array. + public static T handleArray(byte[] value, int pos, ArrayHandler handler) { + checkIndex(pos, value.length); + int basicType = value[pos] & BASIC_TYPE_MASK; + int typeInfo = (value[pos] >> BASIC_TYPE_BITS) & TYPE_INFO_MASK; + if (basicType != ARRAY) { + throw unexpectedType(Type.ARRAY); + } + // Refer to the comment of the `ARRAY` constant for the details of the object header + // encoding. + // Suppose `typeInfo` has a bit representation of 000_b2_b1b0, the following line extracts + // b2 to determine whether the object uses a 1/4-byte size. + boolean largeSize = ((typeInfo >> 2) & 0x1) != 0; + int sizeBytes = (largeSize ? U32_SIZE : 1); + int size = readUnsigned(value, pos + 1, sizeBytes); + // Extracts b1b0 to determine the integer size of the offset list. + int offsetSize = (typeInfo & 0x3) + 1; + int offsetStart = pos + 1 + sizeBytes; + int dataStart = offsetStart + (size + 1) * offsetSize; + return handler.apply(size, offsetSize, offsetStart, dataStart); + } + + // Get a key at `id` in the variant metadata. + // Throw `MALFORMED_VARIANT` if the variant is malformed. An out-of-bound `id` is also + // considered a malformed variant because it is read from the corresponding variant value. + public static String getMetadataKey(byte[] metadata, int id) { + checkIndex(0, metadata.length); + // Extracts the highest 2 bits in the metadata header to determine the integer size of the + // offset list. + int offsetSize = ((metadata[0] >> 6) & 0x3) + 1; + int dictSize = readUnsigned(metadata, 1, offsetSize); + if (id >= dictSize) { + throw malformedVariant(); + } + // There are a header byte, a `dictSize` with `offsetSize` bytes, and `(dictSize + 1)` + // offsets before the string data. + int stringStart = 1 + (dictSize + 2) * offsetSize; + int offset = readUnsigned(metadata, 1 + (id + 1) * offsetSize, offsetSize); + int nextOffset = readUnsigned(metadata, 1 + (id + 2) * offsetSize, offsetSize); + if (offset > nextOffset) { + throw malformedVariant(); + } + checkIndex(stringStart + nextOffset - 1, metadata.length); + return new String(metadata, stringStart + offset, nextOffset - offset); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/variant/PathSegment.java b/paimon-common/src/main/java/org/apache/paimon/data/variant/PathSegment.java new file mode 100644 index 000000000000..5804fb9fcb3e --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/variant/PathSegment.java @@ -0,0 +1,104 @@ +/* + * 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.data.variant; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * A path segment for variant get to represent either an object key access or an array index access. + */ +public class PathSegment { + private final String key; + private final Integer index; + + private PathSegment(String key, Integer index) { + this.key = key; + this.index = index; + } + + public static PathSegment createKeySegment(String key) { + return new PathSegment(key, null); + } + + public static PathSegment createIndexSegment(int index) { + return new PathSegment(null, index); + } + + public boolean isKey() { + return key != null; + } + + public boolean isIndex() { + return index != null; + } + + public String getKey() { + return key; + } + + public Integer getIndex() { + return index; + } + + private static final Pattern ROOT_PATTERN = Pattern.compile("\\$"); + // Parse index segment like `[123]`. + private static final Pattern INDEX_PATTERN = Pattern.compile("\\[(\\d+)]"); + // Parse key segment like `.name` or `['name']` or `["name"]`. + private static final Pattern KEY_PATTERN = + Pattern.compile("\\.([^.\\[]+)|\\['([^']+)']|\\[\"([^\"]+)\"]"); + + public static PathSegment[] parse(String str) { + // Validate root + Matcher rootMatcher = ROOT_PATTERN.matcher(str); + if (str.isEmpty() || !rootMatcher.find()) { + throw new IllegalArgumentException("Invalid path: " + str); + } + + List segments = new ArrayList<>(); + String remaining = str.substring(rootMatcher.end()); + // Parse indexes and keys + while (!remaining.isEmpty()) { + Matcher indexMatcher = INDEX_PATTERN.matcher(remaining); + if (indexMatcher.lookingAt()) { + int index = Integer.parseInt(indexMatcher.group(1)); + segments.add(PathSegment.createIndexSegment(index)); + remaining = remaining.substring(indexMatcher.end()); + continue; + } + + Matcher keyMatcher = KEY_PATTERN.matcher(remaining); + if (keyMatcher.lookingAt()) { + for (int i = 1; i <= 3; i++) { + if (keyMatcher.group(i) != null) { + segments.add(PathSegment.createKeySegment(keyMatcher.group(i))); + break; + } + } + remaining = remaining.substring(keyMatcher.end()); + continue; + } + throw new IllegalArgumentException("Invalid path: " + str); + } + + return segments.toArray(new PathSegment[0]); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/data/variant/Variant.java b/paimon-common/src/main/java/org/apache/paimon/data/variant/Variant.java new file mode 100644 index 000000000000..bfecfd573ad7 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/data/variant/Variant.java @@ -0,0 +1,52 @@ +/* + * 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.data.variant; + +/** + * A Variant represents a type that contain one of: 1) Primitive: A type and corresponding value + * (e.g. INT, STRING); 2) Array: An ordered list of Variant values; 3) Object: An unordered + * collection of string/Variant pairs (i.e. key/value pairs). An object may not contain duplicate + * keys. + * + *

A Variant is encoded with 2 binary values, the value and the metadata. + * + *

The Variant Binary Encoding allows representation of semi-structured data (e.g. JSON) in a + * form that can be efficiently queried by path. The design is intended to allow efficient access to + * nested data even in the presence of very wide or deep structures. + */ +public interface Variant { + + /** Returns the variant metadata. */ + byte[] metadata(); + + /** Returns the variant value. */ + byte[] value(); + + /** Parses the variant to json. */ + String toJson(); + + /** + * Extracts a sub-variant value according to a path which start with a `$`. e.g. + * + *

access object's field: `$.key` or `$['key']` or `$["key"]`. + * + *

access array's first elem: `$.array[0]` + */ + Object variantGet(String path); +} diff --git a/paimon-common/src/test/java/org/apache/paimon/data/variant/GenericVariantTest.java b/paimon-common/src/test/java/org/apache/paimon/data/variant/GenericVariantTest.java new file mode 100644 index 000000000000..dbe9bdf3c1ba --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/data/variant/GenericVariantTest.java @@ -0,0 +1,98 @@ +/* + * 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.data.variant; + +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; + +import static org.apache.paimon.types.DataTypesTest.assertThat; + +/** Test of {@link GenericVariant}. */ +public class GenericVariantTest { + + @Test + public void testToJson() { + String json = + "{\n" + + " \"object\": {\n" + + " \"name\": \"Apache Paimon\",\n" + + " \"age\": 2,\n" + + " \"address\": {\n" + + " \"street\": \"Main St\",\n" + + " \"city\": \"Hangzhou\"\n" + + " }\n" + + " },\n" + + " \"array\": [1, 2, 3, 4, 5],\n" + + " \"string\": \"Hello, World!\",\n" + + " \"long\": 12345678901234,\n" + + " \"double\": 1.0123456789012345678901234567890123456789,\n" + + " \"decimal\": 100.99,\n" + + " \"boolean1\": true,\n" + + " \"boolean2\": false,\n" + + " \"nullField\": null\n" + + "}\n"; + + assertThat(GenericVariant.fromJson(json).toJson()) + .isEqualTo( + "{\"array\":[1,2,3,4,5],\"boolean1\":true,\"boolean2\":false,\"decimal\":100.99,\"double\":1.0123456789012346,\"long\":12345678901234,\"nullField\":null,\"object\":{\"address\":{\"city\":\"Hangzhou\",\"street\":\"Main St\"},\"age\":2,\"name\":\"Apache Paimon\"},\"string\":\"Hello, World!\"}"); + } + + @Test + public void testVariantGet() { + String json = + "{\n" + + " \"object\": {\n" + + " \"name\": \"Apache Paimon\",\n" + + " \"age\": 2,\n" + + " \"address\": {\n" + + " \"street\": \"Main St\",\n" + + " \"city\": \"Hangzhou\"\n" + + " }\n" + + " },\n" + + " \"array\": [1, 2, 3, 4, 5],\n" + + " \"string\": \"Hello, World!\",\n" + + " \"long\": 12345678901234,\n" + + " \"double\": 1.0123456789012345678901234567890123456789,\n" + + " \"decimal\": 100.99,\n" + + " \"boolean1\": true,\n" + + " \"boolean2\": false,\n" + + " \"nullField\": null\n" + + "}\n"; + + Variant variant = GenericVariant.fromJson(json); + assertThat(variant.variantGet("$.object")) + .isEqualTo( + "{\"address\":{\"city\":\"Hangzhou\",\"street\":\"Main St\"},\"age\":2,\"name\":\"Apache Paimon\"}"); + assertThat(variant.variantGet("$.object.name")).isEqualTo("Apache Paimon"); + assertThat(variant.variantGet("$.object.address.street")).isEqualTo("Main St"); + assertThat(variant.variantGet("$[\"object\"]['address'].city")).isEqualTo("Hangzhou"); + assertThat(variant.variantGet("$.array")).isEqualTo("[1,2,3,4,5]"); + assertThat(variant.variantGet("$.array[0]")).isEqualTo(1L); + assertThat(variant.variantGet("$.array[3]")).isEqualTo(4L); + assertThat(variant.variantGet("$.string")).isEqualTo("Hello, World!"); + assertThat(variant.variantGet("$.long")).isEqualTo(12345678901234L); + assertThat(variant.variantGet("$.double")) + .isEqualTo(1.0123456789012345678901234567890123456789); + assertThat(variant.variantGet("$.decimal")).isEqualTo(new BigDecimal("100.99")); + assertThat(variant.variantGet("$.boolean1")).isEqualTo(true); + assertThat(variant.variantGet("$.boolean2")).isEqualTo(false); + assertThat(variant.variantGet("$.nullField")).isNull(); + } +} From 0d28a310301a36213c9b469afc0b7ebcf36dbdae Mon Sep 17 00:00:00 2001 From: JackeyLee007 Date: Wed, 18 Dec 2024 09:52:21 +0800 Subject: [PATCH 09/17] [flink] kafka_sync_database supports db whitelist and blacklist (#4732) --- docs/content/cdc-ingestion/kafka-cdc.md | 6 +- .../generated/kafka_sync_database.html | 26 ++- .../action/cdc/CdcActionCommonUtils.java | 2 + .../action/cdc/SyncDatabaseActionBase.java | 26 ++- .../cdc/SyncDatabaseActionFactoryBase.java | 4 + .../RichCdcMultiplexRecordEventParser.java | 67 ++++++-- .../cdc/SyncDatabaseActionBaseTest.java | 156 ++++++++++++++++++ 7 files changed, 261 insertions(+), 26 deletions(-) create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java diff --git a/docs/content/cdc-ingestion/kafka-cdc.md b/docs/content/cdc-ingestion/kafka-cdc.md index 26a5be340942..fc16c5b0fc1f 100644 --- a/docs/content/cdc-ingestion/kafka-cdc.md +++ b/docs/content/cdc-ingestion/kafka-cdc.md @@ -199,12 +199,14 @@ To use this feature through `flink run`, run the following shell command. --warehouse \ --database \ [--table_mapping =] \ - [--table_prefix_db ] \ [--table_prefix ] \ - [--table_suffix_db ] \ [--table_suffix ] \ + [--table_prefix_db ] \ + [--table_suffix_db ] \ [--including_tables ] \ [--excluding_tables ] \ + [--including_dbs ] \ + [--excluding_dbs ] \ [--type_mapping to-string] \ [--partition_keys ] \ [--primary_keys ] \ diff --git a/docs/layouts/shortcodes/generated/kafka_sync_database.html b/docs/layouts/shortcodes/generated/kafka_sync_database.html index 3664128a26ca..9f0b817e6647 100644 --- a/docs/layouts/shortcodes/generated/kafka_sync_database.html +++ b/docs/layouts/shortcodes/generated/kafka_sync_database.html @@ -41,22 +41,22 @@

--table_mapping
The table name mapping between source database and Paimon. For example, if you want to synchronize a source table named "test" to a Paimon table named "paimon_test", you can specify "--table_mapping test=paimon_test". Multiple mappings could be specified with multiple "--table_mapping" options. "--table_mapping" has higher priority than "--table_prefix" and "--table_suffix". - -
--table_prefix_db
- The prefix of the Paimon tables to be synchronized from the specified db. For example, if you want to prefix the tables from db1 with "ods_db1_", you can specify "--table_prefix_db db1=ods_db1_". "--table_prefix_db" has higher priority than "--table_prefix". -
--table_prefix
The prefix of all Paimon tables to be synchronized except those specified by "--table_mapping" or "--table_prefix_db". For example, if you want all synchronized tables to have "ods_" as prefix, you can specify "--table_prefix ods_". - -
--table_suffix_db
- The suffix of the Paimon tables to be synchronized from the specified db. The usage is same as "--table_prefix_db". -
--table_suffix
The suffix of all Paimon tables to be synchronized except those specified by "--table_mapping" or "--table_suffix_db". The usage is same as "--table_prefix". + +
--table_prefix_db
+ The prefix of the Paimon tables to be synchronized from the specified db. For example, if you want to prefix the tables from db1 with "ods_db1_", you can specify "--table_prefix_db db1=ods_db1_". "--table_prefix_db" has higher priority than "--table_prefix". + + +
--table_suffix_db
+ The suffix of the Paimon tables to be synchronized from the specified db. The usage is same as "--table_prefix_db". +
--including_tables
It is used to specify which source tables are to be synchronized. You must use '|' to separate multiple tables.Because '|' is a special character, a comma is required, for example: 'a|b|c'.Regular expression is supported, for example, specifying "--including_tables test|paimon.*" means to synchronize table 'test' and all tables start with 'paimon'. @@ -65,6 +65,14 @@
--excluding_tables
It is used to specify which source tables are not to be synchronized. The usage is same as "--including_tables". "--excluding_tables" has higher priority than "--including_tables" if you specified both. + +
--including_dbs
+ It is used to specify the databases within which the tables are to be synchronized. The usage is same as "--including_tables". + + +
--excluding_dbs
+ It is used to specify the databases within which the tables are not to be synchronized. The usage is same as "--excluding_tables". "--excluding_dbs" has higher priority than "--including_dbs" if you specified both. +
--type_mapping
It is used to specify how to map MySQL data type to Paimon type.
@@ -114,4 +122,4 @@ The configuration for Paimon table sink. Each configuration should be specified in the format "key=value". See here for a complete list of table configurations. - \ No newline at end of file + diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java index 6482a625f4c7..c107500eba86 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java @@ -61,6 +61,8 @@ public class CdcActionCommonUtils { public static final String TABLE_MAPPING = "table_mapping"; public static final String INCLUDING_TABLES = "including_tables"; public static final String EXCLUDING_TABLES = "excluding_tables"; + public static final String INCLUDING_DBS = "including_dbs"; + public static final String EXCLUDING_DBS = "excluding_dbs"; public static final String TYPE_MAPPING = "type_mapping"; public static final String PARTITION_KEYS = "partition_keys"; public static final String PRIMARY_KEYS = "primary_keys"; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java index 56334c1e7bff..63e29d6a0ed8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java @@ -60,6 +60,8 @@ public abstract class SyncDatabaseActionBase extends SynchronizationActionBase { protected List partitionKeys = new ArrayList<>(); protected List primaryKeys = new ArrayList<>(); @Nullable protected String excludingTables; + protected String includingDbs = ".*"; + @Nullable protected String excludingDbs; protected List tables = new ArrayList<>(); protected Map> partitionKeyMultiple = new HashMap<>(); @@ -144,6 +146,18 @@ public SyncDatabaseActionBase excludingTables(@Nullable String excludingTables) return this; } + public SyncDatabaseActionBase includingDbs(@Nullable String includingDbs) { + if (includingDbs != null) { + this.includingDbs = includingDbs; + } + return this; + } + + public SyncDatabaseActionBase excludingDbs(@Nullable String excludingDbs) { + this.excludingDbs = excludingDbs; + return this; + } + public SyncDatabaseActionBase withPartitionKeys(String... partitionKeys) { this.partitionKeys.addAll(Arrays.asList(partitionKeys)); return this; @@ -186,9 +200,11 @@ protected EventParser.Factory buildEventParserFactory() requirePrimaryKeys(), partitionKeyMultiple, metadataConverters); - Pattern includingPattern = Pattern.compile(includingTables); - Pattern excludingPattern = + Pattern tblIncludingPattern = Pattern.compile(includingTables); + Pattern tblExcludingPattern = excludingTables == null ? null : Pattern.compile(excludingTables); + Pattern dbIncludingPattern = Pattern.compile(includingDbs); + Pattern dbExcludingPattern = excludingDbs == null ? null : Pattern.compile(excludingDbs); TableNameConverter tableNameConverter = new TableNameConverter( caseSensitive, @@ -207,8 +223,10 @@ protected EventParser.Factory buildEventParserFactory() return () -> new RichCdcMultiplexRecordEventParser( schemaBuilder, - includingPattern, - excludingPattern, + tblIncludingPattern, + tblExcludingPattern, + dbIncludingPattern, + dbExcludingPattern, tableNameConverter, createdTables); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java index d497b588c2af..c82039a9a021 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java @@ -24,7 +24,9 @@ import java.util.Optional; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.EXCLUDING_DBS; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.EXCLUDING_TABLES; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.INCLUDING_DBS; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.INCLUDING_TABLES; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.MULTIPLE_TABLE_PARTITION_KEYS; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PARTITION_KEYS; @@ -59,6 +61,8 @@ protected void withParams(MultipleParameterToolAdapter params, T action) { .withTableMapping(optionalConfigMap(params, TABLE_MAPPING)) .includingTables(params.get(INCLUDING_TABLES)) .excludingTables(params.get(EXCLUDING_TABLES)) + .includingDbs(params.get(INCLUDING_DBS)) + .excludingDbs(params.get(EXCLUDING_DBS)) .withPartitionKeyMultiple( optionalConfigMapList(params, MULTIPLE_TABLE_PARTITION_KEYS)) .withPartitionKeys(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcMultiplexRecordEventParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcMultiplexRecordEventParser.java index 939410bf46ce..47367c423406 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcMultiplexRecordEventParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcMultiplexRecordEventParser.java @@ -46,8 +46,10 @@ public class RichCdcMultiplexRecordEventParser implements EventParser createdTables; @@ -55,24 +57,32 @@ public class RichCdcMultiplexRecordEventParser implements EventParser includedTables = new HashSet<>(); private final Set excludedTables = new HashSet<>(); + private final Set includedDbs = new HashSet<>(); + private final Set excludedDbs = new HashSet<>(); + private RichCdcMultiplexRecord record; private String currentTable; + private String currentDb; private boolean shouldSynchronizeCurrentTable; private RichEventParser currentParser; public RichCdcMultiplexRecordEventParser(boolean caseSensitive) { - this(null, null, null, new TableNameConverter(caseSensitive), new HashSet<>()); + this(null, null, null, null, null, new TableNameConverter(caseSensitive), new HashSet<>()); } public RichCdcMultiplexRecordEventParser( @Nullable NewTableSchemaBuilder schemaBuilder, - @Nullable Pattern includingPattern, - @Nullable Pattern excludingPattern, + @Nullable Pattern tblIncludingPattern, + @Nullable Pattern tblExcludingPattern, + @Nullable Pattern dbIncludingPattern, + @Nullable Pattern dbExcludingPattern, TableNameConverter tableNameConverter, Set createdTables) { this.schemaBuilder = schemaBuilder; - this.includingPattern = includingPattern; - this.excludingPattern = excludingPattern; + this.tblIncludingPattern = tblIncludingPattern; + this.tblExcludingPattern = tblExcludingPattern; + this.dbIncludingPattern = dbIncludingPattern; + this.dbExcludingPattern = dbExcludingPattern; this.tableNameConverter = tableNameConverter; this.createdTables = createdTables; } @@ -81,6 +91,7 @@ public RichCdcMultiplexRecordEventParser( public void setRawEvent(RichCdcMultiplexRecord record) { this.record = record; this.currentTable = record.tableName(); + this.currentDb = record.databaseName(); this.shouldSynchronizeCurrentTable = shouldSynchronizeCurrentTable(); if (shouldSynchronizeCurrentTable) { this.currentParser = parsers.computeIfAbsent(currentTable, t -> new RichEventParser()); @@ -124,7 +135,41 @@ public Optional parseNewTable() { return Optional.empty(); } + private boolean shouldSynchronizeCurrentDb() { + // In case the record is incomplete, we let the null value pass validation + // and handle the null value when we really need it + if (currentDb == null) { + return true; + } + if (includedDbs.contains(currentDb)) { + return true; + } + if (excludedDbs.contains(currentDb)) { + return false; + } + boolean shouldSynchronize = true; + if (dbIncludingPattern != null) { + shouldSynchronize = dbIncludingPattern.matcher(currentDb).matches(); + } + if (dbExcludingPattern != null) { + shouldSynchronize = + shouldSynchronize && !dbExcludingPattern.matcher(currentDb).matches(); + } + if (!shouldSynchronize) { + LOG.debug( + "Source database {} won't be synchronized because it was excluded. ", + currentDb); + excludedDbs.add(currentDb); + return false; + } + includedDbs.add(currentDb); + return true; + } + private boolean shouldSynchronizeCurrentTable() { + if (!shouldSynchronizeCurrentDb()) { + return false; + } // In case the record is incomplete, we let the null value pass validation // and handle the null value when we really need it if (currentTable == null) { @@ -139,12 +184,12 @@ private boolean shouldSynchronizeCurrentTable() { } boolean shouldSynchronize = true; - if (includingPattern != null) { - shouldSynchronize = includingPattern.matcher(currentTable).matches(); + if (tblIncludingPattern != null) { + shouldSynchronize = tblIncludingPattern.matcher(currentTable).matches(); } - if (excludingPattern != null) { + if (tblExcludingPattern != null) { shouldSynchronize = - shouldSynchronize && !excludingPattern.matcher(currentTable).matches(); + shouldSynchronize && !tblExcludingPattern.matcher(currentTable).matches(); } if (!shouldSynchronize) { LOG.debug( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java new file mode 100644 index 000000000000..5247225caff4 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java @@ -0,0 +1,156 @@ +/* + * 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.flink.action.cdc; + +import org.apache.paimon.flink.action.cdc.kafka.KafkaSyncDatabaseAction; +import org.apache.paimon.flink.sink.cdc.CdcRecord; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordEventParser; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.types.RowKind; + +import org.junit.Assert; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** Tests for {@link SyncDatabaseActionBase}. */ +public class SyncDatabaseActionBaseTest { + private static final String ANY_DB = "any_db"; + private static final String WHITE_DB = "white_db"; + private static final String BLACK_DB = "black_db"; + private static final String WHITE_TBL = "white_tbl"; + private static final String BLACK_TBL = "black_tbl"; + + private SyncDatabaseActionBase kafkaSyncDbAction; + private RichCdcMultiplexRecord whiteAnyDbCdcRecord; + private RichCdcMultiplexRecord blackAnyDbCdcRecord; + private RichCdcMultiplexRecord whiteCdcRecord; + private RichCdcMultiplexRecord blackCdcRecord; + private RichCdcMultiplexRecord whiteDbBlackTblCdcRecord; + private RichCdcMultiplexRecord blackDbWhiteTblCdcRecord; + + @TempDir private java.nio.file.Path tmp; + + @BeforeEach + public void setUp() throws Exception { + LocalFileIO localFileIO = new LocalFileIO(); + Path defaultDb = new Path(tmp.toString(), "default.db"); + localFileIO.mkdirs(defaultDb); + + kafkaSyncDbAction = + new KafkaSyncDatabaseAction( + tmp.toString(), "default", new HashMap<>(), new HashMap<>()); + + Map rawData = new HashMap<>(); + rawData.put("field", "value"); + + CdcRecord cdcData = new CdcRecord(RowKind.INSERT, rawData); + whiteAnyDbCdcRecord = + new RichCdcMultiplexRecord( + ANY_DB, WHITE_TBL, Arrays.asList(), Arrays.asList(), cdcData); + blackAnyDbCdcRecord = + new RichCdcMultiplexRecord( + ANY_DB, BLACK_TBL, Arrays.asList(), Arrays.asList(), cdcData); + whiteCdcRecord = + new RichCdcMultiplexRecord( + WHITE_DB, WHITE_TBL, Arrays.asList(), Arrays.asList(), cdcData); + blackCdcRecord = + new RichCdcMultiplexRecord( + BLACK_DB, WHITE_TBL, Arrays.asList(), Arrays.asList(), cdcData); + + whiteDbBlackTblCdcRecord = + new RichCdcMultiplexRecord( + WHITE_DB, BLACK_TBL, Arrays.asList(), Arrays.asList(), cdcData); + blackDbWhiteTblCdcRecord = + new RichCdcMultiplexRecord( + BLACK_DB, WHITE_TBL, Arrays.asList(), Arrays.asList(), cdcData); + } + + @Test + public void testSyncTablesWithoutDbLists() throws NoSuchMethodException, IOException { + + kafkaSyncDbAction.includingTables(WHITE_TBL); + kafkaSyncDbAction.excludingTables(BLACK_TBL); + + RichCdcMultiplexRecordEventParser parser = + (RichCdcMultiplexRecordEventParser) + kafkaSyncDbAction.buildEventParserFactory().create(); + List parsedRecords; + + parser.setRawEvent(whiteAnyDbCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(1, parsedRecords.size()); + + parser.setRawEvent(blackAnyDbCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(0, parsedRecords.size()); + } + + @Test + public void testSyncTablesWithDbList() { + kafkaSyncDbAction.includingDbs(WHITE_DB); + kafkaSyncDbAction.excludingDbs(BLACK_DB); + RichCdcMultiplexRecordEventParser parser = + (RichCdcMultiplexRecordEventParser) + kafkaSyncDbAction.buildEventParserFactory().create(); + List parsedRecords; + + parser.setRawEvent(whiteAnyDbCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(0, parsedRecords.size()); + + parser.setRawEvent(blackAnyDbCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(0, parsedRecords.size()); + + // white db and white table + parser.setRawEvent(whiteCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(1, parsedRecords.size()); + + parser.setRawEvent(blackAnyDbCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(0, parsedRecords.size()); + } + + @Test + public void testSycTablesCrossDB() { + kafkaSyncDbAction.includingDbs(WHITE_DB); + kafkaSyncDbAction.excludingDbs(BLACK_DB); + kafkaSyncDbAction.excludingTables(BLACK_TBL); + RichCdcMultiplexRecordEventParser parser = + (RichCdcMultiplexRecordEventParser) + kafkaSyncDbAction.buildEventParserFactory().create(); + List parsedRecords; + parser.setRawEvent(whiteDbBlackTblCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(0, parsedRecords.size()); + parser.setRawEvent(blackDbWhiteTblCdcRecord); + parsedRecords = parser.parseRecords(); + Assert.assertEquals(0, parsedRecords.size()); + } +} From 32da0356d2a93d033252b0ef998b51b69b8a3f9c Mon Sep 17 00:00:00 2001 From: Jiao Mingye <35512473+mxdzs0612@users.noreply.github.com> Date: Wed, 18 Dec 2024 10:46:39 +0800 Subject: [PATCH 10/17] [core] Introduce CacheStats and expose ScanStats (#4678) --- .../generated/catalog_configuration.html | 2 +- .../apache/paimon/options/CatalogOptions.java | 2 +- .../apache/paimon/catalog/CachingCatalog.java | 66 +++++++++++++++++++ .../operation/AbstractFileStoreScan.java | 2 + .../operation/metrics/CacheMetrics.java | 49 ++++++++++++++ .../paimon/operation/metrics/ScanMetrics.java | 23 +++++-- .../org/apache/paimon/utils/ObjectsCache.java | 12 ++++ .../org/apache/paimon/utils/ObjectsFile.java | 8 +++ .../apache/paimon/utils/SegmentsCache.java | 10 +++ ...stryImplTest.java => MetricGroupTest.java} | 8 +-- .../paimon/metrics/TestMetricRegistry.java} | 4 +- .../operation/metrics/CommitMetricsTest.java | 4 +- .../metrics/CompactionMetricsTest.java | 4 +- .../operation/metrics/ScanMetricsTest.java | 8 ++- .../apache/paimon/utils/ObjectsCacheTest.java | 9 ++- 15 files changed, 189 insertions(+), 22 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/operation/metrics/CacheMetrics.java rename paimon-core/src/test/java/org/apache/paimon/metrics/{MetricRegistryImplTest.java => MetricGroupTest.java} (90%) rename paimon-core/src/{main/java/org/apache/paimon/metrics/MetricRegistryImpl.java => test/java/org/apache/paimon/metrics/TestMetricRegistry.java} (89%) diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html index 6355c9558653..03efd178b8fc 100644 --- a/docs/layouts/shortcodes/generated/catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/catalog_configuration.html @@ -30,7 +30,7 @@
cache-enabled
true Boolean - Controls whether the catalog will cache databases, tables and manifests. + Controls whether the catalog will cache databases, tables, manifests and partitions.
cache.expiration-interval
diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java index b22274e011fb..d0cfbeaf39ed 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java @@ -87,7 +87,7 @@ public class CatalogOptions { .booleanType() .defaultValue(true) .withDescription( - "Controls whether the catalog will cache databases, tables and manifests."); + "Controls whether the catalog will cache databases, tables, manifests and partitions."); public static final ConfigOption CACHE_EXPIRATION_INTERVAL_MS = key("cache.expiration-interval") diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 99540cf0cea5..5c9c7854934a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -312,6 +312,30 @@ public void invalidateTable(Identifier identifier) { } } + public CacheSizes estimatedCacheSizes() { + long databaseCacheSize = databaseCache.estimatedSize(); + long tableCacheSize = tableCache.estimatedSize(); + long manifestCacheSize = 0L; + long manifestCacheBytes = 0L; + if (manifestCache != null) { + manifestCacheSize = manifestCache.getSegmentCacheSize(); + manifestCacheBytes = manifestCache.getSegmentCacheBytes(); + } + long partitionCacheSize = 0L; + if (partitionCache != null) { + for (Map.Entry> entry : + partitionCache.asMap().entrySet()) { + partitionCacheSize += entry.getValue().size(); + } + } + return new CacheSizes( + databaseCacheSize, + tableCacheSize, + manifestCacheSize, + manifestCacheBytes, + partitionCacheSize); + } + // ================================== refresh ================================================ // following caches will affect the latency of table, so refresh method is provided for engine @@ -321,4 +345,46 @@ public void refreshPartitions(Identifier identifier) throws TableNotExistExcepti partitionCache.put(identifier, result); } } + + /** Cache sizes of a caching catalog. */ + public static class CacheSizes { + private final long databaseCacheSize; + private final long tableCacheSize; + private final long manifestCacheSize; + private final long manifestCacheBytes; + private final long partitionCacheSize; + + public CacheSizes( + long databaseCacheSize, + long tableCacheSize, + long manifestCacheSize, + long manifestCacheBytes, + long partitionCacheSize) { + this.databaseCacheSize = databaseCacheSize; + this.tableCacheSize = tableCacheSize; + this.manifestCacheSize = manifestCacheSize; + this.manifestCacheBytes = manifestCacheBytes; + this.partitionCacheSize = partitionCacheSize; + } + + public long databaseCacheSize() { + return databaseCacheSize; + } + + public long tableCacheSize() { + return tableCacheSize; + } + + public long manifestCacheSize() { + return manifestCacheSize; + } + + public long manifestCacheBytes() { + return manifestCacheBytes; + } + + public long partitionCacheSize() { + return partitionCacheSize; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index c73a92062b80..4f8a1f3264de 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -428,6 +428,8 @@ private List readManifest( List entries = manifestFileFactory .create() + .withCacheMetrics( + scanMetrics != null ? scanMetrics.getCacheMetrics() : null) .read( manifest.fileName(), manifest.fileSize(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/CacheMetrics.java b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/CacheMetrics.java new file mode 100644 index 000000000000..fdf9ce6921c1 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/CacheMetrics.java @@ -0,0 +1,49 @@ +/* + * 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.operation.metrics; + +import java.util.concurrent.atomic.AtomicLong; + +/** Metrics for manifest file cache of a caching catalog. */ +public class CacheMetrics { + + private final AtomicLong hitObject; + private final AtomicLong missedObject; + + public CacheMetrics() { + this.hitObject = new AtomicLong(0); + this.missedObject = new AtomicLong(0); + } + + public AtomicLong getHitObject() { + return hitObject; + } + + public void increaseHitObject() { + this.hitObject.incrementAndGet(); + } + + public AtomicLong getMissedObject() { + return missedObject; + } + + public void increaseMissedObject() { + this.missedObject.incrementAndGet(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java index 96f0aec1c0b2..fdc41baf280c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java @@ -30,9 +30,14 @@ public class ScanMetrics { public static final String GROUP_NAME = "scan"; private final MetricGroup metricGroup; + private Histogram durationHistogram; + + private ScanStats latestScan; + private CacheMetrics cacheMetrics; public ScanMetrics(MetricRegistry registry, String tableName) { this.metricGroup = registry.tableMetricGroup(GROUP_NAME, tableName); + this.cacheMetrics = new CacheMetrics(); registerGenericScanMetrics(); } @@ -41,17 +46,13 @@ public MetricGroup getMetricGroup() { return metricGroup; } - private Histogram durationHistogram; - - private ScanStats latestScan; - public static final String LAST_SCAN_DURATION = "lastScanDuration"; public static final String SCAN_DURATION = "scanDuration"; public static final String LAST_SCANNED_MANIFESTS = "lastScannedManifests"; - public static final String LAST_SCAN_SKIPPED_TABLE_FILES = "lastScanSkippedTableFiles"; - public static final String LAST_SCAN_RESULTED_TABLE_FILES = "lastScanResultedTableFiles"; + public static final String MANIFEST_HIT_CACHE = "manifestHitCache"; + public static final String MANIFEST_MISSED_CACHE = "manifestMissedCache"; private void registerGenericScanMetrics() { metricGroup.gauge( @@ -66,10 +67,20 @@ private void registerGenericScanMetrics() { metricGroup.gauge( LAST_SCAN_RESULTED_TABLE_FILES, () -> latestScan == null ? 0L : latestScan.getResultedTableFiles()); + metricGroup.gauge( + MANIFEST_HIT_CACHE, + () -> cacheMetrics == null ? 0L : cacheMetrics.getHitObject().get()); + metricGroup.gauge( + MANIFEST_MISSED_CACHE, + () -> cacheMetrics == null ? 0L : cacheMetrics.getMissedObject().get()); } public void reportScan(ScanStats scanStats) { latestScan = scanStats; durationHistogram.update(scanStats.getDuration()); } + + public CacheMetrics getCacheMetrics() { + return cacheMetrics; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java index 1c9d9664f22f..6f14c78107a2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.memory.MemorySegment; import org.apache.paimon.memory.MemorySegmentSource; +import org.apache.paimon.operation.metrics.CacheMetrics; import org.apache.paimon.types.RowType; import javax.annotation.Nullable; @@ -47,6 +48,7 @@ public class ObjectsCache { private final ThreadLocal formatSerializer; private final FunctionWithIOException fileSizeFunction; private final BiFunctionWithIOE> reader; + private CacheMetrics cacheMetrics; public ObjectsCache( SegmentsCache cache, @@ -71,8 +73,14 @@ public List read( throws IOException { Segments segments = cache.getIfPresents(key); if (segments != null) { + if (cacheMetrics != null) { + cacheMetrics.increaseHitObject(); + } return readFromSegments(segments, readFilter, readVFilter); } else { + if (cacheMetrics != null) { + cacheMetrics.increaseMissedObject(); + } if (fileSize == null) { fileSize = fileSizeFunction.apply(key); } @@ -130,4 +138,8 @@ private Segments readSegments(K key, @Nullable Long fileSize, Filter List readFromIterator( throw new RuntimeException(e); } } + + public ObjectsFile withCacheMetrics(CacheMetrics cacheMetrics) { + if (cache != null) { + cache.withCacheMetrics(cacheMetrics); + } + return this; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java b/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java index d5c2178c8b6a..0ef4b13a3122 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java @@ -90,4 +90,14 @@ public static SegmentsCache create( return new SegmentsCache<>(pageSize, maxMemorySize, maxElementSize); } + + public long getSegmentCacheSize() { + return cache.estimatedSize(); + } + + public long getSegmentCacheBytes() { + return cache.asMap().entrySet().stream() + .mapToLong(entry -> weigh(entry.getKey(), entry.getValue())) + .sum(); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/metrics/MetricRegistryImplTest.java b/paimon-core/src/test/java/org/apache/paimon/metrics/MetricGroupTest.java similarity index 90% rename from paimon-core/src/test/java/org/apache/paimon/metrics/MetricRegistryImplTest.java rename to paimon-core/src/test/java/org/apache/paimon/metrics/MetricGroupTest.java index 27f9e70ecba8..90a063e32415 100644 --- a/paimon-core/src/test/java/org/apache/paimon/metrics/MetricRegistryImplTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/metrics/MetricGroupTest.java @@ -24,12 +24,12 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for the {@link MetricRegistryImpl}. */ -public class MetricRegistryImplTest { +/** Tests for the {@link MetricGroup}. */ +public class MetricGroupTest { @Test public void testGroupRegisterMetrics() { - MetricRegistryImpl registry = new MetricRegistryImpl(); + TestMetricRegistry registry = new TestMetricRegistry(); MetricGroup group = registry.tableMetricGroup("commit", "myTable"); // these will fail is the registration is propagated @@ -50,7 +50,7 @@ public void testGroupRegisterMetrics() { @Test public void testTolerateMetricNameCollisions() { final String name = "abctestname"; - MetricRegistryImpl registry = new MetricRegistryImpl(); + TestMetricRegistry registry = new TestMetricRegistry(); MetricGroup group = registry.tableMetricGroup("commit", "myTable"); Counter counter = group.counter(name); diff --git a/paimon-core/src/main/java/org/apache/paimon/metrics/MetricRegistryImpl.java b/paimon-core/src/test/java/org/apache/paimon/metrics/TestMetricRegistry.java similarity index 89% rename from paimon-core/src/main/java/org/apache/paimon/metrics/MetricRegistryImpl.java rename to paimon-core/src/test/java/org/apache/paimon/metrics/TestMetricRegistry.java index b815364a856d..4b2870041c11 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metrics/MetricRegistryImpl.java +++ b/paimon-core/src/test/java/org/apache/paimon/metrics/TestMetricRegistry.java @@ -20,8 +20,8 @@ import java.util.Map; -/** Default implementation of {@link MetricRegistry}. */ -public class MetricRegistryImpl extends MetricRegistry { +/** Implementation of {@link MetricRegistry} for tests. */ +public class TestMetricRegistry extends MetricRegistry { @Override protected MetricGroup createMetricGroup(String groupName, Map variables) { diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CommitMetricsTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CommitMetricsTest.java index 476790a8c0dc..6a79a0ae5807 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CommitMetricsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CommitMetricsTest.java @@ -23,7 +23,7 @@ import org.apache.paimon.metrics.Gauge; import org.apache.paimon.metrics.Histogram; import org.apache.paimon.metrics.Metric; -import org.apache.paimon.metrics.MetricRegistryImpl; +import org.apache.paimon.metrics.TestMetricRegistry; import org.junit.jupiter.api.Test; @@ -234,6 +234,6 @@ private void reportAgain(CommitMetrics commitMetrics) { } private CommitMetrics getCommitMetrics() { - return new CommitMetrics(new MetricRegistryImpl(), TABLE_NAME); + return new CommitMetrics(new TestMetricRegistry(), TABLE_NAME); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CompactionMetricsTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CompactionMetricsTest.java index 222b99769d06..d2f5e8963d99 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CompactionMetricsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/CompactionMetricsTest.java @@ -22,7 +22,7 @@ import org.apache.paimon.metrics.Counter; import org.apache.paimon.metrics.Gauge; import org.apache.paimon.metrics.Metric; -import org.apache.paimon.metrics.MetricRegistryImpl; +import org.apache.paimon.metrics.TestMetricRegistry; import org.junit.jupiter.api.Test; @@ -33,7 +33,7 @@ public class CompactionMetricsTest { @Test public void testReportMetrics() { - CompactionMetrics metrics = new CompactionMetrics(new MetricRegistryImpl(), "myTable"); + CompactionMetrics metrics = new CompactionMetrics(new TestMetricRegistry(), "myTable"); assertThat(getMetric(metrics, CompactionMetrics.MAX_LEVEL0_FILE_COUNT)).isEqualTo(-1L); assertThat(getMetric(metrics, CompactionMetrics.AVG_LEVEL0_FILE_COUNT)).isEqualTo(-1.0); assertThat(getMetric(metrics, CompactionMetrics.COMPACTION_THREAD_BUSY)).isEqualTo(0.0); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java index a0427d95cab1..7ea86a2800d1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java @@ -22,7 +22,7 @@ import org.apache.paimon.metrics.Histogram; import org.apache.paimon.metrics.Metric; import org.apache.paimon.metrics.MetricGroup; -import org.apache.paimon.metrics.MetricRegistryImpl; +import org.apache.paimon.metrics.TestMetricRegistry; import org.junit.jupiter.api.Test; @@ -48,7 +48,9 @@ public void testGenericMetricsRegistration() { ScanMetrics.SCAN_DURATION, ScanMetrics.LAST_SCANNED_MANIFESTS, ScanMetrics.LAST_SCAN_SKIPPED_TABLE_FILES, - ScanMetrics.LAST_SCAN_RESULTED_TABLE_FILES); + ScanMetrics.LAST_SCAN_RESULTED_TABLE_FILES, + ScanMetrics.MANIFEST_HIT_CACHE, + ScanMetrics.MANIFEST_MISSED_CACHE); } /** Tests that the metrics are updated properly. */ @@ -124,6 +126,6 @@ private void reportAgain(ScanMetrics scanMetrics) { } private ScanMetrics getScanMetrics() { - return new ScanMetrics(new MetricRegistryImpl(), TABLE_NAME); + return new ScanMetrics(new TestMetricRegistry(), TABLE_NAME); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java index 9d3275e3ab48..7e52814a820e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java @@ -21,6 +21,8 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.metrics.TestMetricRegistry; +import org.apache.paimon.operation.metrics.ScanMetrics; import org.apache.paimon.options.MemorySize; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -40,7 +42,7 @@ public class ObjectsCacheTest { @Test - public void test() throws IOException { + public void testObjectsCacheAndMetrics() throws IOException { Map> map = new HashMap<>(); ObjectsCache cache = new ObjectsCache<>( @@ -56,12 +58,15 @@ public void test() throws IOException { .map(r -> (InternalRow) r) .iterator())); + ScanMetrics scanMetrics = new ScanMetrics(new TestMetricRegistry(), "table"); + cache.withCacheMetrics(scanMetrics.getCacheMetrics()); // test empty map.put("k1", Collections.emptyList()); List values = cache.read( "k1", null, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).isEmpty(); + assertThat(scanMetrics.getCacheMetrics().getMissedObject()).hasValue(1); // test values List expect = Arrays.asList("v1", "v2", "v3"); @@ -70,12 +75,14 @@ public void test() throws IOException { cache.read( "k2", null, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactlyElementsOf(expect); + assertThat(scanMetrics.getCacheMetrics().getMissedObject()).hasValue(2); // test cache values = cache.read( "k2", null, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactlyElementsOf(expect); + assertThat(scanMetrics.getCacheMetrics().getHitObject()).hasValue(1); // test filter values = From c93040e38d7b5d6636cc80f3629b8c185e675f80 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 18 Dec 2024 10:48:33 +0800 Subject: [PATCH 11/17] [core] Minor refactor for cache metrics --- .../apache/paimon/catalog/CachingCatalog.java | 33 +++++++++------ .../paimon/operation/metrics/ScanMetrics.java | 41 ++++++++----------- .../org/apache/paimon/utils/ObjectsCache.java | 11 ++--- .../org/apache/paimon/utils/ObjectsFile.java | 14 +++---- .../apache/paimon/utils/SegmentsCache.java | 4 +- 5 files changed, 52 insertions(+), 51 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 5c9c7854934a..34e53f32f267 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -312,14 +312,32 @@ public void invalidateTable(Identifier identifier) { } } + // ================================== Cache Public API + // ================================================ + + /** + * Partition cache will affect the latency of table, so refresh method is provided for compute + * engine. + */ + public void refreshPartitions(Identifier identifier) throws TableNotExistException { + if (partitionCache != null) { + List result = wrapped.listPartitions(identifier); + partitionCache.put(identifier, result); + } + } + + /** + * Cache sizes for compute engine. This method can let the outside know the specific usage of + * cache. + */ public CacheSizes estimatedCacheSizes() { long databaseCacheSize = databaseCache.estimatedSize(); long tableCacheSize = tableCache.estimatedSize(); long manifestCacheSize = 0L; long manifestCacheBytes = 0L; if (manifestCache != null) { - manifestCacheSize = manifestCache.getSegmentCacheSize(); - manifestCacheBytes = manifestCache.getSegmentCacheBytes(); + manifestCacheSize = manifestCache.estimatedSize(); + manifestCacheBytes = manifestCache.totalCacheBytes(); } long partitionCacheSize = 0L; if (partitionCache != null) { @@ -336,18 +354,9 @@ public CacheSizes estimatedCacheSizes() { partitionCacheSize); } - // ================================== refresh ================================================ - // following caches will affect the latency of table, so refresh method is provided for engine - - public void refreshPartitions(Identifier identifier) throws TableNotExistException { - if (partitionCache != null) { - List result = wrapped.listPartitions(identifier); - partitionCache.put(identifier, result); - } - } - /** Cache sizes of a caching catalog. */ public static class CacheSizes { + private final long databaseCacheSize; private final long tableCacheSize; private final long manifestCacheSize; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java index fdc41baf280c..be6b514e04ea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java @@ -28,24 +28,6 @@ public class ScanMetrics { private static final int HISTOGRAM_WINDOW_SIZE = 100; public static final String GROUP_NAME = "scan"; - - private final MetricGroup metricGroup; - private Histogram durationHistogram; - - private ScanStats latestScan; - private CacheMetrics cacheMetrics; - - public ScanMetrics(MetricRegistry registry, String tableName) { - this.metricGroup = registry.tableMetricGroup(GROUP_NAME, tableName); - this.cacheMetrics = new CacheMetrics(); - registerGenericScanMetrics(); - } - - @VisibleForTesting - public MetricGroup getMetricGroup() { - return metricGroup; - } - public static final String LAST_SCAN_DURATION = "lastScanDuration"; public static final String SCAN_DURATION = "scanDuration"; public static final String LAST_SCANNED_MANIFESTS = "lastScannedManifests"; @@ -54,10 +36,18 @@ public MetricGroup getMetricGroup() { public static final String MANIFEST_HIT_CACHE = "manifestHitCache"; public static final String MANIFEST_MISSED_CACHE = "manifestMissedCache"; - private void registerGenericScanMetrics() { + private final MetricGroup metricGroup; + private final Histogram durationHistogram; + private final CacheMetrics cacheMetrics; + + private ScanStats latestScan; + + public ScanMetrics(MetricRegistry registry, String tableName) { + metricGroup = registry.tableMetricGroup(GROUP_NAME, tableName); metricGroup.gauge( LAST_SCAN_DURATION, () -> latestScan == null ? 0L : latestScan.getDuration()); durationHistogram = metricGroup.histogram(SCAN_DURATION, HISTOGRAM_WINDOW_SIZE); + cacheMetrics = new CacheMetrics(); metricGroup.gauge( LAST_SCANNED_MANIFESTS, () -> latestScan == null ? 0L : latestScan.getScannedManifests()); @@ -67,12 +57,13 @@ private void registerGenericScanMetrics() { metricGroup.gauge( LAST_SCAN_RESULTED_TABLE_FILES, () -> latestScan == null ? 0L : latestScan.getResultedTableFiles()); - metricGroup.gauge( - MANIFEST_HIT_CACHE, - () -> cacheMetrics == null ? 0L : cacheMetrics.getHitObject().get()); - metricGroup.gauge( - MANIFEST_MISSED_CACHE, - () -> cacheMetrics == null ? 0L : cacheMetrics.getMissedObject().get()); + metricGroup.gauge(MANIFEST_HIT_CACHE, () -> cacheMetrics.getHitObject().get()); + metricGroup.gauge(MANIFEST_MISSED_CACHE, () -> cacheMetrics.getMissedObject().get()); + } + + @VisibleForTesting + MetricGroup getMetricGroup() { + return metricGroup; } public void reportScan(ScanStats scanStats) { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java index 6f14c78107a2..8fe13943a3b2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java @@ -48,7 +48,8 @@ public class ObjectsCache { private final ThreadLocal formatSerializer; private final FunctionWithIOException fileSizeFunction; private final BiFunctionWithIOE> reader; - private CacheMetrics cacheMetrics; + + @Nullable private CacheMetrics cacheMetrics; public ObjectsCache( SegmentsCache cache, @@ -64,6 +65,10 @@ public ObjectsCache( this.reader = reader; } + public void withCacheMetrics(@Nullable CacheMetrics cacheMetrics) { + this.cacheMetrics = cacheMetrics; + } + public List read( K key, @Nullable Long fileSize, @@ -138,8 +143,4 @@ private Segments readSegments(K key, @Nullable Long fileSize, Filter withCacheMetrics(@Nullable CacheMetrics cacheMetrics) { + if (cache != null) { + cache.withCacheMetrics(cacheMetrics); + } + return this; + } + public FileIO fileIO() { return fileIO; } @@ -208,11 +215,4 @@ public static List readFromIterator( throw new RuntimeException(e); } } - - public ObjectsFile withCacheMetrics(CacheMetrics cacheMetrics) { - if (cache != null) { - cache.withCacheMetrics(cacheMetrics); - } - return this; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java b/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java index 0ef4b13a3122..8cac9f03be12 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/SegmentsCache.java @@ -91,11 +91,11 @@ public static SegmentsCache create( return new SegmentsCache<>(pageSize, maxMemorySize, maxElementSize); } - public long getSegmentCacheSize() { + public long estimatedSize() { return cache.estimatedSize(); } - public long getSegmentCacheBytes() { + public long totalCacheBytes() { return cache.asMap().entrySet().stream() .mapToLong(entry -> weigh(entry.getKey(), entry.getValue())) .sum(); From f3813f075189ecdf2f1fd77e37d7bd32fcd204e5 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 18 Dec 2024 10:59:20 +0800 Subject: [PATCH 12/17] [cdc] Correct equalsIgnoreFieldId in UpdatedDataFieldsProcessFunctionBase --- .../org/apache/paimon/types/ArrayType.java | 2 +- .../org/apache/paimon/types/DataType.java | 27 ++++++++++--------- .../java/org/apache/paimon/types/MapType.java | 2 +- .../java/org/apache/paimon/types/RowType.java | 3 ++- .../cdc/UpdatedDataFieldsProcessFunction.java | 2 +- .../UpdatedDataFieldsProcessFunctionBase.java | 8 +++--- 6 files changed, 25 insertions(+), 19 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java b/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java index 62fb9ce65b69..f4c523534e49 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java @@ -101,7 +101,7 @@ public boolean equals(Object o) { } @Override - public boolean equalsIgnoreFieldId(Object o) { + public boolean equalsIgnoreFieldId(DataType o) { if (this == o) { return true; } diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java index dd9a4685ef5b..ac1e5b2a645e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java @@ -23,8 +23,6 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import javax.annotation.Nonnull; - import java.io.IOException; import java.io.Serializable; import java.util.Arrays; @@ -124,15 +122,6 @@ public final DataType copy() { return copy(isNullable); } - /** - * Compare two data types without nullable. - * - * @param o the target data type - */ - public boolean equalsIgnoreNullable(@Nonnull DataType o) { - return Objects.equals(this.copy(true), o.copy(true)); - } - @Override public boolean equals(Object o) { if (this == o) { @@ -145,7 +134,21 @@ public boolean equals(Object o) { return isNullable == that.isNullable && typeRoot == that.typeRoot; } - public boolean equalsIgnoreFieldId(Object o) { + /** + * Compare two data types without nullable. + * + * @param o the target data type + */ + public boolean equalsIgnoreNullable(DataType o) { + return Objects.equals(this.copy(true), o.copy(true)); + } + + /** + * Compare two data types without field id. + * + * @param o the target data type + */ + public boolean equalsIgnoreFieldId(DataType o) { return equals(o); } diff --git a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java b/paimon-common/src/main/java/org/apache/paimon/types/MapType.java index b715d49284fa..75ea5fbb7cc2 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/MapType.java @@ -110,7 +110,7 @@ public boolean equals(Object o) { } @Override - public boolean equalsIgnoreFieldId(Object o) { + public boolean equalsIgnoreFieldId(DataType o) { if (this == o) { return true; } diff --git a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java b/paimon-common/src/main/java/org/apache/paimon/types/RowType.java index 625a4634b320..fecb5bed9ebd 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/RowType.java @@ -213,7 +213,8 @@ public boolean equals(Object o) { return fields.equals(rowType.fields); } - public boolean equalsIgnoreFieldId(Object o) { + @Override + public boolean equalsIgnoreFieldId(DataType o) { if (this == o) { return true; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java index 504f63105801..e143aabf6c13 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java @@ -77,7 +77,7 @@ public void processElement( extractSchemaChanges(schemaManager, actualUpdatedDataFields)) { applySchemaChange(schemaManager, schemaChange, identifier); } - /** + /* * Here, actualUpdatedDataFields cannot be used to update latestFields because there is a * non-SchemaChange.AddColumn scenario. Otherwise, the previously existing fields cannot be * modified again. diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index d50df23742aa..90edbc034a54 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -219,9 +219,11 @@ protected List extractSchemaChanges( String newFieldName = StringUtils.toLowerCaseIfNeed(newField.name(), caseSensitive); if (oldFields.containsKey(newFieldName)) { DataField oldField = oldFields.get(newFieldName); - // we compare by ignoring nullable, because partition keys and primary keys might be - // nullable in source database, but they can't be null in Paimon - if (oldField.type().equalsIgnoreNullable(newField.type())) { + // 1. we compare by ignoring nullable, because partition keys and primary keys might + // be nullable in source database, but they can't be null in Paimon + // 2. we compare by ignoring field id, the field ID is newly created and may be + // different, we should ignore it + if (oldField.type().copy(true).equalsIgnoreFieldId(newField.type().copy(true))) { // update column comment if (newField.description() != null && !newField.description().equals(oldField.description())) { From a057fd1c46aa08ad1e88e5acf0a56bf816518bf4 Mon Sep 17 00:00:00 2001 From: LsomeYeah <94825748+LsomeYeah@users.noreply.github.com> Date: Wed, 18 Dec 2024 11:30:48 +0800 Subject: [PATCH 13/17] [cdc] add exception message for CdcRecordStoreMultiWriteOperator (#4734) --- .../cdc/CdcRecordStoreMultiWriteOperator.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java index 9387a8293874..a4b4e8284043 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java @@ -174,7 +174,7 @@ public void processElement(StreamRecord element) throws Exce try { write.write(optionalConverted.get()); } catch (Exception e) { - throw new IOException(e); + throw new IOException("Exception occurs for writing record to table: " + tableId, e); } } @@ -235,12 +235,17 @@ protected List prepareCommit(boolean waitCompaction, long for (Map.Entry entry : writes.entrySet()) { Identifier key = entry.getKey(); StoreSinkWrite write = entry.getValue(); - committables.addAll( - write.prepareCommit(waitCompaction, checkpointId).stream() - .map( - committable -> - MultiTableCommittable.fromCommittable(key, committable)) - .collect(Collectors.toList())); + try { + committables.addAll( + write.prepareCommit(waitCompaction, checkpointId).stream() + .map( + committable -> + MultiTableCommittable.fromCommittable( + key, committable)) + .collect(Collectors.toList())); + } catch (Exception e) { + throw new IOException("Failed to prepare commit for table: " + key.toString(), e); + } } return committables; } From 0b684ca6ccd9d648327a06baf75df9eef3e787cc Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Wed, 18 Dec 2024 15:10:57 +0800 Subject: [PATCH 14/17] [core] Fix predicate literals cast in filter pushdown after schema evolution (#4705) --- .../apache/paimon/casting/CastExecutors.java | 55 +++++ .../operation/AppendOnlyFileStoreScan.java | 4 +- .../operation/KeyValueFileStoreScan.java | 2 +- .../paimon/schema/SchemaEvolutionUtil.java | 41 ++- .../paimon/stats/SimpleStatsEvolutions.java | 21 +- .../paimon/utils/FormatReaderMapping.java | 2 +- .../schema/SchemaEvolutionUtilTest.java | 28 +-- .../FilterPushdownWithSchemaChangeITCase.java | 233 ++++++++++++++++++ 8 files changed, 322 insertions(+), 64 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastExecutors.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastExecutors.java index 8134e0118bf8..546066d10aa3 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/CastExecutors.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastExecutors.java @@ -24,8 +24,10 @@ import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -100,6 +102,59 @@ public class CastExecutors { return IDENTITY_CAST_EXECUTOR; } + /** + * If a field type is modified, pushing down a filter of it is dangerous. This method tries to + * cast the literals of filter to its original type. It only cast the literals when the CastRule + * is in whitelist. Otherwise, return Optional.empty(). + */ + public static Optional> castLiteralsWithEvolution( + List literals, DataType predicateType, DataType dataType) { + if (predicateType.equalsIgnoreNullable(dataType)) { + return Optional.of(literals); + } + + CastRule castRule = INSTANCE.internalResolve(predicateType, dataType); + if (castRule == null) { + return Optional.empty(); + } + + if (castRule instanceof NumericPrimitiveCastRule) { + // Ignore float literals because pushing down float filter result is unpredictable. + // For example, (double) 0.1F in Java is 0.10000000149011612. + + if (predicateType.is(DataTypeFamily.INTEGER_NUMERIC) + && dataType.is(DataTypeFamily.INTEGER_NUMERIC)) { + // Ignore input scale < output scale because of overflow. + // For example, alter 383 from INT to TINYINT, the query result is (byte) 383 == + // 127. If we push down filter f = 127, 383 will be filtered out mistakenly. + + if (integerScaleLargerThan(predicateType.getTypeRoot(), dataType.getTypeRoot())) { + CastExecutor castExecutor = + (CastExecutor) castRule.create(predicateType, dataType); + List newLiterals = new ArrayList<>(literals.size()); + for (Object literal : literals) { + Number literalNumber = (Number) literal; + Number newLiteralNumber = castExecutor.cast(literalNumber); + // Ignore if any literal is overflowed. + if (newLiteralNumber.longValue() != literalNumber.longValue()) { + return Optional.empty(); + } + newLiterals.add(newLiteralNumber); + } + return Optional.of(newLiterals); + } + } + } + + return Optional.empty(); + } + + private static boolean integerScaleLargerThan(DataTypeRoot a, DataTypeRoot b) { + return (a == DataTypeRoot.SMALLINT && b == DataTypeRoot.TINYINT) + || (a == DataTypeRoot.INTEGER && b != DataTypeRoot.BIGINT) + || a == DataTypeRoot.BIGINT; + } + // Map> private final Map>> rules = new HashMap<>(); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java index d2ca5da42249..1498e08a2b20 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java @@ -109,7 +109,9 @@ private boolean testFileIndex(@Nullable byte[] embeddedIndexBytes, ManifestEntry Predicate dataPredicate = dataFilterMapping.computeIfAbsent( entry.file().schemaId(), - id -> simpleStatsEvolutions.convertFilter(entry.file().schemaId(), filter)); + id -> + simpleStatsEvolutions.tryDevolveFilter( + entry.file().schemaId(), filter)); try (FileIndexPredicate predicate = new FileIndexPredicate(embeddedIndexBytes, dataRowType)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java index 8d8c51996cfe..e39ad2e3c2e7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java @@ -158,7 +158,7 @@ private boolean filterByFileIndex(@Nullable byte[] embeddedIndexBytes, ManifestE schemaId2DataFilter.computeIfAbsent( entry.file().schemaId(), id -> - fieldValueStatsConverters.convertFilter( + fieldValueStatsConverters.tryDevolveFilter( entry.file().schemaId(), valueFilter)); return predicate.evaluate(dataPredicate).remain(); } catch (IOException e) { diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java index 0ae2798c29e0..cab5dcaeb8ad 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java @@ -276,8 +276,10 @@ public static int[][] createDataProjection( } /** - * Create predicate list from data fields. We will visit all predicate in filters, reset it's - * field index, name and type, and ignore predicate if the field is not exist. + * When pushing down filters after schema evolution, we should devolve the literals from new + * types (in dataFields) to original types (in tableFields). We will visit all predicate in + * filters, reset its field index, name and type, and ignore predicate if the field is not + * exist. * * @param tableFields the table fields * @param dataFields the underlying data fields @@ -285,7 +287,7 @@ public static int[][] createDataProjection( * @return the data filters */ @Nullable - public static List createDataFilters( + public static List devolveDataFilters( List tableFields, List dataFields, List filters) { if (filters == null) { return null; @@ -308,29 +310,16 @@ public static List createDataFilters( return Optional.empty(); } - DataType dataValueType = dataField.type().copy(true); - DataType predicateType = predicate.type().copy(true); - CastExecutor castExecutor = - dataValueType.equals(predicateType) - ? null - : (CastExecutor) - CastExecutors.resolve( - predicate.type(), dataField.type()); - // Convert value from predicate type to underlying data type which may lose - // information, for example, convert double value to int. But it doesn't matter - // because it just for predicate push down and the data will be filtered - // correctly after reading. - List literals = - predicate.literals().stream() - .map(v -> castExecutor == null ? v : castExecutor.cast(v)) - .collect(Collectors.toList()); - return Optional.of( - new LeafPredicate( - predicate.function(), - dataField.type(), - indexOf(dataField, idToDataFields), - dataField.name(), - literals)); + return CastExecutors.castLiteralsWithEvolution( + predicate.literals(), predicate.type(), dataField.type()) + .map( + literals -> + new LeafPredicate( + predicate.function(), + dataField.type(), + indexOf(dataField, idToDataFields), + dataField.name(), + literals)); }; for (Predicate predicate : filters) { diff --git a/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolutions.java b/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolutions.java index a0814b8c04c4..566cae9e6592 100644 --- a/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolutions.java +++ b/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolutions.java @@ -77,15 +77,18 @@ public SimpleStatsEvolution getOrCreate(long dataSchemaId) { }); } - public Predicate convertFilter(long dataSchemaId, Predicate filter) { - return tableSchemaId == dataSchemaId - ? filter - : Objects.requireNonNull( - SchemaEvolutionUtil.createDataFilters( - schemaFields.apply(tableSchemaId), - schemaFields.apply(dataSchemaId), - Collections.singletonList(filter))) - .get(0); + @Nullable + public Predicate tryDevolveFilter(long dataSchemaId, Predicate filter) { + if (tableSchemaId == dataSchemaId) { + return filter; + } + List devolved = + Objects.requireNonNull( + SchemaEvolutionUtil.devolveDataFilters( + schemaFields.apply(tableSchemaId), + schemaFields.apply(dataSchemaId), + Collections.singletonList(filter))); + return devolved.isEmpty() ? null : devolved.get(0); } public List tableDataFields() { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java b/paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java index f6c6287f51b4..00554b233c59 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java @@ -290,7 +290,7 @@ private List readFilters( List dataFilters = tableSchema.id() == dataSchema.id() ? filters - : SchemaEvolutionUtil.createDataFilters( + : SchemaEvolutionUtil.devolveDataFilters( tableSchema.fields(), dataSchema.fields(), filters); // Skip pushing down partition filters to reader. diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaEvolutionUtilTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaEvolutionUtilTest.java index 9d947f76d995..30d844e6c606 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaEvolutionUtilTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaEvolutionUtilTest.java @@ -20,7 +20,6 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.predicate.Equal; import org.apache.paimon.predicate.IsNotNull; import org.apache.paimon.predicate.IsNull; import org.apache.paimon.predicate.LeafPredicate; @@ -263,7 +262,7 @@ public void testCreateDataProjection() { } @Test - public void testCreateDataFilters() { + public void testDevolveDataFilters() { List predicates = new ArrayList<>(); predicates.add( new LeafPredicate( @@ -278,7 +277,7 @@ public void testCreateDataFilters() { IsNull.INSTANCE, DataTypes.INT(), 7, "a", Collections.emptyList())); List filters = - SchemaEvolutionUtil.createDataFilters(tableFields2, dataFields, predicates); + SchemaEvolutionUtil.devolveDataFilters(tableFields2, dataFields, predicates); assert filters != null; assertThat(filters.size()).isEqualTo(1); @@ -287,27 +286,4 @@ public void testCreateDataFilters() { assertThat(child1.fieldName()).isEqualTo("b"); assertThat(child1.index()).isEqualTo(1); } - - @Test - public void testColumnTypeFilter() { - // (1, b, int) in data schema is updated to (1, c, double) in table2 - List predicates = new ArrayList<>(); - predicates.add( - new LeafPredicate( - Equal.INSTANCE, - DataTypes.DOUBLE(), - 0, - "c", - Collections.singletonList(1.0D))); - List filters = - SchemaEvolutionUtil.createDataFilters(tableFields2, dataFields, predicates); - assert filters != null; - assertThat(filters.size()).isEqualTo(1); - - LeafPredicate child = (LeafPredicate) filters.get(0); - // Validate value 1 with index 1 - assertThat(child.test(GenericRow.of(0, 1))).isTrue(); - // Validate value 2 with index 1 - assertThat(child.test(GenericRow.of(1, 2))).isFalse(); - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java new file mode 100644 index 000000000000..3b12ceabe2da --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java @@ -0,0 +1,233 @@ +/* + * 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.flink; + +import org.apache.paimon.casting.CastExecutors; +import org.apache.paimon.testutils.junit.parameterized.ParameterizedTestExtension; +import org.apache.paimon.testutils.junit.parameterized.Parameters; +import org.apache.paimon.types.DataType; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for {@link CastExecutors#castLiteralsWithEvolution(List, DataType, DataType)}. */ +@ExtendWith(ParameterizedTestExtension.class) +public class FilterPushdownWithSchemaChangeITCase extends CatalogITCaseBase { + + private final String fileFormat; + + public FilterPushdownWithSchemaChangeITCase(String fileFormat) { + this.fileFormat = fileFormat; + } + + @SuppressWarnings("unused") + @Parameters(name = "file-format = {0}") + public static List fileFormat() { + return Arrays.asList("parquet", "orc", "avro"); + } + + @TestTemplate + public void testDecimalToDecimal() { + // to higher precision + sql( + "CREATE TABLE T (" + + " id INT," + + " f DECIMAL(5, 2)" + + ") with (" + + " 'file.format' = '%s'" + + ")", + fileFormat); + sql("INSERT INTO T VALUES (1, 111.32)"); + sql("ALTER TABLE T MODIFY (f DECIMAL(6, 3))"); + assertThat(sql("SELECT * FROM T WHERE f < 111.321")) + .containsExactly(Row.of(1, new BigDecimal("111.320"))); + assertThat(sql("SELECT * FROM T WHERE f = 111.321")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f = 111.320")) + .containsExactly(Row.of(1, new BigDecimal("111.320"))); + assertThat(sql("SELECT * FROM T WHERE f <> 111.321")) + .containsExactly(Row.of(1, new BigDecimal("111.320"))); + + sql("DROP TABLE T"); + + // to lower precision + sql( + "CREATE TABLE T (" + + " id INT," + + " f DECIMAL(6, 3)" + + ") with (" + + " 'file.format' = '%s'" + + ")", + fileFormat); + sql("INSERT INTO T VALUES (1, 111.321), (2, 111.331)"); + sql("ALTER TABLE T MODIFY (f DECIMAL(5, 2))"); + assertThat(sql("SELECT * FROM T WHERE f > 111.32")) + .containsExactly(Row.of(2, new BigDecimal("111.33"))); + assertThat(sql("SELECT * FROM T WHERE f = 111.32")) + .containsExactly(Row.of(1, new BigDecimal("111.32"))); + assertThat(sql("SELECT * FROM T WHERE f <> 111.32")) + .containsExactly(Row.of(2, new BigDecimal("111.33"))); + } + + @TestTemplate + public void testNumericPrimitiveToDecimal() { + String ddl = + "CREATE TABLE T (" + + " id INT," + + " f DECIMAL(5, 2)" + + ") with (" + + " 'file.format' = '%s'" + + ")"; + + // to higher precision + sql(ddl, fileFormat); + sql("INSERT INTO T VALUES (1, 111.32)"); + sql("ALTER TABLE T MODIFY (f DOUBLE)"); + assertThat(sql("SELECT * FROM T WHERE f < 111.321")).containsExactly(Row.of(1, 111.32)); + assertThat(sql("SELECT * FROM T WHERE f = 111.321")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f = 111.320")).containsExactly(Row.of(1, 111.32)); + assertThat(sql("SELECT * FROM T WHERE f <> 111.321")).containsExactly(Row.of(1, 111.32)); + + sql("DROP TABLE T"); + + // to lower precision + sql(ddl, fileFormat); + sql("INSERT INTO T VALUES (1, 111.32), (2, 112.33)"); + sql("ALTER TABLE T MODIFY (f INT)"); + assertThat(sql("SELECT * FROM T WHERE f < 112")).containsExactly(Row.of(1, 111)); + assertThat(sql("SELECT * FROM T WHERE f > 112")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f = 111")).containsExactly(Row.of(1, 111)); + assertThat(sql("SELECT * FROM T WHERE f <> 111")).containsExactly(Row.of(2, 112)); + } + + @TestTemplate + public void testDecimalToNumericPrimitive() { + // to higher precision + sql( + "CREATE TABLE T (" + + " id INT," + + " f INT" + + ") with (" + + " 'file.format' = '%s'" + + ")", + fileFormat); + sql("INSERT INTO T VALUES (1, 111)"); + sql("ALTER TABLE T MODIFY (f DECIMAL(5, 2))"); + assertThat(sql("SELECT * FROM T WHERE f < 111.01")) + .containsExactly(Row.of(1, new BigDecimal("111.00"))); + assertThat(sql("SELECT * FROM T WHERE f = 111.01")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f = 111.00")) + .containsExactly(Row.of(1, new BigDecimal("111.00"))); + assertThat(sql("SELECT * FROM T WHERE f <> 111.01")) + .containsExactly(Row.of(1, new BigDecimal("111.00"))); + + sql("DROP TABLE T"); + + // to lower precision + sql( + "CREATE TABLE T (" + + " id INT," + + " f DOUBLE" + + ") with (" + + " 'file.format' = '%s'" + + ")", + fileFormat); + sql("INSERT INTO T VALUES (1, 111.321), (2, 111.331)"); + sql("ALTER TABLE T MODIFY (f DECIMAL(5, 2))"); + assertThat(sql("SELECT * FROM T WHERE f > 111.32")) + .containsExactly(Row.of(2, new BigDecimal("111.33"))); + assertThat(sql("SELECT * FROM T WHERE f = 111.32")) + .containsExactly(Row.of(1, new BigDecimal("111.32"))); + assertThat(sql("SELECT * FROM T WHERE f <> 111.32")) + .containsExactly(Row.of(2, new BigDecimal("111.33"))); + } + + @TestTemplate + public void testNumericPrimitive() { + // no checks for high scale to low scale because we don't pushdown it + + // integer to higher scale integer + sql( + "CREATE TABLE T (" + + " id INT," + + " f TINYINT" + + ") with (" + + " 'file.format' = '%s'" + + ")", + fileFormat); + sql("INSERT INTO T VALUES (1, CAST (127 AS TINYINT))"); + sql("ALTER TABLE T MODIFY (f INT)"); + // (byte) 383 == 127 + assertThat(sql("SELECT * FROM T WHERE f < 128")).containsExactly(Row.of(1, 127)); + assertThat(sql("SELECT * FROM T WHERE f < 383")).containsExactly(Row.of(1, 127)); + assertThat(sql("SELECT * FROM T WHERE f = 127")).containsExactly(Row.of(1, 127)); + assertThat(sql("SELECT * FROM T WHERE f = 383")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f <> 127")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f <> 383")).containsExactly(Row.of(1, 127)); + + sql("DROP TABLE T"); + + // INT to BIGINT + sql( + "CREATE TABLE T (" + + " id INT," + + " f INT" + + ") with (" + + " 'file.format' = '%s'" + + ")", + fileFormat); + // (int) Integer.MAX_VALUE + 1 == Integer.MIN_VALUE -> (int) 2147483648L == -2147483648 + sql("INSERT INTO T VALUES (1, 2147483647), (2, -2147483648)"); + sql("ALTER TABLE T MODIFY (f BIGINT)"); + assertThat(sql("SELECT * FROM T WHERE f < 2147483648")) + .containsExactlyInAnyOrder(Row.of(1, 2147483647L), Row.of(2, -2147483648L)); + assertThat(sql("SELECT * FROM T WHERE f > 2147483648")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f = 2147483647")) + .containsExactly(Row.of(1, 2147483647L)); + assertThat(sql("SELECT * FROM T WHERE f = 2147483648")).isEmpty(); + assertThat(sql("SELECT * FROM T WHERE f <> 2147483647")) + .containsExactly(Row.of(2, -2147483648L)); + assertThat(sql("SELECT * FROM T WHERE f <> 2147483648")) + .containsExactlyInAnyOrder(Row.of(1, 2147483647L), Row.of(2, -2147483648L)); + } + + @TestTemplate + public void testNumericToString() { + // no more string related tests because we don't push down it + sql( + "CREATE TABLE T (" + + " id INT," + + " f STRING" + + ") with (" + + " 'file.format' = '%s'" + + ");", + fileFormat); + sql("INSERT INTO T VALUES (1, '1'), (2, '111')"); + sql("ALTER TABLE T MODIFY (f INT)"); + assertThat(sql("SELECT * FROM T WHERE f > 2")).containsExactly(Row.of(2, 111)); + assertThat(sql("SELECT * FROM T WHERE f = 1")).containsExactly(Row.of(1, 1)); + assertThat(sql("SELECT * FROM T WHERE f <> 1")).containsExactly(Row.of(2, 111)); + } +} From 99f0b23e3aa68eaeaf42ffc8efa4e7ee7fd6f7d7 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Wed, 18 Dec 2024 15:35:08 +0800 Subject: [PATCH 15/17] [hive] Fix sync hms partition with data-file.path-directory (#4735) --- .../java/org/apache/paimon/CoreOptions.java | 1 + .../org/apache/paimon/hive/HiveCatalog.java | 18 ++-- .../paimon/hive/HiveMetastoreClient.java | 26 +++-- .../sql/DDLWithHiveCatalogTestBase.scala | 94 ++++++++++++------- 4 files changed, 88 insertions(+), 51 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index dd5632c18b42..6e1e9bba076b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -184,6 +184,7 @@ public class CoreOptions implements Serializable { .defaultValue("data-") .withDescription("Specify the file name prefix of data files."); + @Immutable public static final ConfigOption DATA_FILE_PATH_DIRECTORY = key("data-file.path-directory") .stringType() diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index a590ede6d2e6..975c1c0b7aed 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -95,6 +95,7 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREWAREHOUSE; import static org.apache.hadoop.hive.serde.serdeConstants.FIELD_DELIM; +import static org.apache.paimon.CoreOptions.DATA_FILE_PATH_DIRECTORY; import static org.apache.paimon.CoreOptions.FILE_FORMAT; import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_TIME; import static org.apache.paimon.CoreOptions.TYPE; @@ -771,22 +772,27 @@ protected void createTableImpl(Identifier identifier, Schema schema) { private Table createHiveTable( Identifier identifier, TableSchema tableSchema, Path location, boolean externalTable) { - checkArgument(Options.fromMap(tableSchema.options()).get(TYPE) != FORMAT_TABLE); + Map options = tableSchema.options(); + checkArgument(Options.fromMap(options).get(TYPE) != FORMAT_TABLE); Map tblProperties; if (syncAllProperties()) { - tblProperties = new HashMap<>(tableSchema.options()); - + tblProperties = new HashMap<>(options); // add primary-key, partition-key to tblproperties tblProperties.putAll(convertToPropertiesTableKey(tableSchema)); } else { - tblProperties = convertToPropertiesPrefixKey(tableSchema.options(), HIVE_PREFIX); - if (tableSchema.options().containsKey(PARTITION_EXPIRATION_TIME.key())) { + tblProperties = convertToPropertiesPrefixKey(options, HIVE_PREFIX); + if (options.containsKey(PARTITION_EXPIRATION_TIME.key())) { // This property will be stored in the 'table_params' table of the HMS database for // querying by other engines or products. tblProperties.put( PARTITION_EXPIRATION_TIME.key(), - tableSchema.options().get(PARTITION_EXPIRATION_TIME.key())); + options.get(PARTITION_EXPIRATION_TIME.key())); + } + if (options.containsKey(DATA_FILE_PATH_DIRECTORY.key())) { + tblProperties.put( + DATA_FILE_PATH_DIRECTORY.key(), + options.get(DATA_FILE_PATH_DIRECTORY.key())); } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java index f7be538c259d..0661988648f4 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java @@ -18,6 +18,7 @@ package org.apache.paimon.hive; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.client.ClientPool; import org.apache.paimon.hive.pool.CachedClientPool; @@ -33,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PartitionEventType; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; import org.apache.thrift.TException; import java.util.ArrayList; @@ -56,19 +58,25 @@ public class HiveMetastoreClient implements MetastoreClient { private final ClientPool clients; private final StorageDescriptor sd; + private final String dataFilePath; HiveMetastoreClient(Identifier identifier, ClientPool clients) throws TException, InterruptedException { this.identifier = identifier; this.clients = clients; - this.sd = - this.clients - .run( - client -> - client.getTable( - identifier.getDatabaseName(), - identifier.getTableName())) - .getSd(); + Table table = + this.clients.run( + client -> + client.getTable( + identifier.getDatabaseName(), identifier.getTableName())); + this.sd = table.getSd(); + this.dataFilePath = + table.getParameters().containsKey(CoreOptions.DATA_FILE_PATH_DIRECTORY.key()) + ? sd.getLocation() + + "/" + + table.getParameters() + .get(CoreOptions.DATA_FILE_PATH_DIRECTORY.key()) + : sd.getLocation(); } @Override @@ -185,7 +193,7 @@ private Partition toHivePartition( Partition hivePartition = new Partition(); StorageDescriptor newSd = new StorageDescriptor(sd); newSd.setLocation( - sd.getLocation() + "/" + PartitionPathUtils.generatePartitionPath(partitionSpec)); + dataFilePath + "/" + PartitionPathUtils.generatePartitionPath(partitionSpec)); hivePartition.setDbName(identifier.getDatabaseName()); hivePartition.setTableName(identifier.getTableName()); hivePartition.setValues(new ArrayList<>(partitionSpec.values())); diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 526e24250751..d51cdce34cb9 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -18,6 +18,7 @@ package org.apache.paimon.spark.sql +import org.apache.paimon.fs.Path import org.apache.paimon.hive.HiveMetastoreClient import org.apache.paimon.spark.PaimonHiveTestBase import org.apache.paimon.table.FileStoreTable @@ -296,43 +297,64 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { test("Paimon DDL with hive catalog: sync partitions to HMS") { Seq(sparkCatalogName, paimonHiveCatalogName).foreach { catalogName => - val dbName = "default" - val tblName = "t" - spark.sql(s"USE $catalogName.$dbName") - withTable(tblName) { - spark.sql(s""" - |CREATE TABLE $tblName (id INT, pt INT) - |USING PAIMON - |TBLPROPERTIES ('metastore.partitioned-table' = 'true') - |PARTITIONED BY (pt) - |""".stripMargin) + Seq("", "data").foreach { + dataFilePathDir => + val dbName = "default" + val tblName = "t" + spark.sql(s"USE $catalogName.$dbName") + withTable(tblName) { + spark.sql(s""" + |CREATE TABLE $tblName (id INT, pt INT) + |USING PAIMON + |TBLPROPERTIES ( + |${if (dataFilePathDir.isEmpty) "" + else s"'data-file.path-directory' = '$dataFilePathDir',"} + |'metastore.partitioned-table' = 'true' + |) + |PARTITIONED BY (pt) + |""".stripMargin) + + val table = loadTable(dbName, tblName) + val metastoreClient = table + .catalogEnvironment() + .metastoreClientFactory() + .create() + .asInstanceOf[HiveMetastoreClient] + .client() + val fileIO = table.fileIO() + + def containsDir(root: Path, targets: Array[String]): Boolean = { + targets.forall(fileIO.listDirectories(root).map(_.getPath.getName).contains) + } + + spark.sql(s"INSERT INTO $tblName VALUES (1, 1), (2, 2), (3, 3)") + // check partitions in paimon + checkAnswer( + spark.sql(s"show partitions $tblName"), + Seq(Row("pt=1"), Row("pt=2"), Row("pt=3"))) + // check partitions in HMS + assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) + // check partitions in filesystem + if (dataFilePathDir.isEmpty) { + assert(containsDir(table.location(), Array("pt=1", "pt=2", "pt=3"))) + } else { + assert(!containsDir(table.location(), Array("pt=1", "pt=2", "pt=3"))) + assert( + containsDir(new Path(table.location(), "data"), Array("pt=1", "pt=2", "pt=3"))) + } - val metastoreClient = loadTable(dbName, tblName) - .catalogEnvironment() - .metastoreClientFactory() - .create() - .asInstanceOf[HiveMetastoreClient] - .client() - - spark.sql(s"INSERT INTO $tblName VALUES (1, 1), (2, 2), (3, 3)") - // check partitions in paimon - checkAnswer( - spark.sql(s"show partitions $tblName"), - Seq(Row("pt=1"), Row("pt=2"), Row("pt=3"))) - // check partitions in HMS - assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) - - spark.sql(s"INSERT INTO $tblName VALUES (4, 3), (5, 4)") - checkAnswer( - spark.sql(s"show partitions $tblName"), - Seq(Row("pt=1"), Row("pt=2"), Row("pt=3"), Row("pt=4"))) - assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 4) - - spark.sql(s"ALTER TABLE $tblName DROP PARTITION (pt=1)") - checkAnswer( - spark.sql(s"show partitions $tblName"), - Seq(Row("pt=2"), Row("pt=3"), Row("pt=4"))) - assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) + spark.sql(s"INSERT INTO $tblName VALUES (4, 3), (5, 4)") + checkAnswer( + spark.sql(s"show partitions $tblName"), + Seq(Row("pt=1"), Row("pt=2"), Row("pt=3"), Row("pt=4"))) + assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 4) + + spark.sql(s"ALTER TABLE $tblName DROP PARTITION (pt=1)") + checkAnswer( + spark.sql(s"show partitions $tblName"), + Seq(Row("pt=2"), Row("pt=3"), Row("pt=4"))) + assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) + } } } } From 85186a65fd242ec5762b1e300f4291a6348092ff Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Wed, 18 Dec 2024 17:51:13 +0800 Subject: [PATCH 16/17] [flink] make warehouse in Flink action optional (#4728) --- .../paimon/flink/FlinkActionITCase.java | 3 +- .../procedure/CompactDatabaseProcedure.java | 3 +- .../flink/procedure/CompactProcedure.java | 4 -- .../flink/procedure/MergeIntoProcedure.java | 6 +- .../cdc/MessageQueueSyncTableActionBase.java | 3 +- .../action/cdc/SyncDatabaseActionBase.java | 2 - .../cdc/SyncDatabaseActionFactoryBase.java | 4 +- .../flink/action/cdc/SyncTableActionBase.java | 2 - .../cdc/SyncTableActionFactoryBase.java | 8 +-- .../action/cdc/SynchronizationActionBase.java | 3 +- .../cdc/SynchronizationActionFactoryBase.java | 9 ++- .../cdc/kafka/KafkaSyncDatabaseAction.java | 7 +- .../kafka/KafkaSyncDatabaseActionFactory.java | 2 +- .../cdc/kafka/KafkaSyncTableAction.java | 9 +-- .../kafka/KafkaSyncTableActionFactory.java | 6 +- .../mongodb/MongoDBSyncDatabaseAction.java | 7 +- .../MongoDBSyncDatabaseActionFactory.java | 2 +- .../cdc/mongodb/MongoDBSyncTableAction.java | 9 +-- .../MongoDBSyncTableActionFactory.java | 6 +- .../cdc/mysql/MySqlSyncDatabaseAction.java | 7 +- .../mysql/MySqlSyncDatabaseActionFactory.java | 2 +- .../cdc/mysql/MySqlSyncTableAction.java | 9 +-- .../mysql/MySqlSyncTableActionFactory.java | 7 +- .../cdc/postgres/PostgresSyncTableAction.java | 9 +-- .../PostgresSyncTableActionFactory.java | 6 +- .../cdc/pulsar/PulsarSyncDatabaseAction.java | 7 +- .../PulsarSyncDatabaseActionFactory.java | 2 +- .../cdc/pulsar/PulsarSyncTableAction.java | 9 +-- .../pulsar/PulsarSyncTableActionFactory.java | 7 +- .../cdc/SyncDatabaseActionBaseTest.java | 5 +- .../paimon/flink/action/ActionBase.java | 4 +- .../paimon/flink/action/ActionFactory.java | 66 ++++++------------- .../paimon/flink/action/CloneAction.java | 11 +--- .../flink/action/CloneActionFactory.java | 17 +++-- .../paimon/flink/action/CompactAction.java | 8 +-- .../flink/action/CompactActionFactory.java | 29 ++------ .../flink/action/CompactDatabaseAction.java | 4 +- .../action/CompactDatabaseActionFactory.java | 5 +- .../flink/action/CreateBranchAction.java | 3 +- .../action/CreateBranchActionFactory.java | 26 ++------ .../CreateOrReplaceTagActionFactory.java | 19 +++--- .../paimon/flink/action/CreateTagAction.java | 3 +- .../flink/action/CreateTagActionFactory.java | 14 +--- .../action/CreateTagFromTimestampAction.java | 13 +++- .../CreateTagFromTimestampActionFactory.java | 17 ++--- .../action/CreateTagFromWatermarkAction.java | 13 +++- .../CreateTagFromWatermarkActionFactory.java | 16 +++-- .../paimon/flink/action/DeleteAction.java | 3 +- .../flink/action/DeleteActionFactory.java | 13 ++-- .../flink/action/DeleteBranchAction.java | 3 +- .../action/DeleteBranchActionFactory.java | 14 ++-- .../paimon/flink/action/DeleteTagAction.java | 3 +- .../flink/action/DeleteTagActionFactory.java | 14 ++-- .../flink/action/DropPartitionAction.java | 3 +- .../action/DropPartitionActionFactory.java | 18 ++--- .../flink/action/ExpirePartitionsAction.java | 3 +- .../action/ExpirePartitionsActionFactory.java | 22 ++----- .../flink/action/ExpireSnapshotsAction.java | 14 ++-- .../action/ExpireSnapshotsActionFactory.java | 13 ++-- .../paimon/flink/action/ExpireTagsAction.java | 9 ++- .../flink/action/ExpireTagsActionFactory.java | 15 ++--- .../flink/action/FastForwardAction.java | 3 +- .../action/FastForwardActionFactory.java | 14 ++-- .../flink/action/MarkPartitionDoneAction.java | 3 +- .../MarkPartitionDoneActionFactory.java | 18 ++--- .../paimon/flink/action/MergeIntoAction.java | 12 +--- .../flink/action/MergeIntoActionFactory.java | 26 +++----- .../flink/action/MigrateDatabaseAction.java | 3 +- .../action/MigrateDatabaseActionFactory.java | 10 +-- .../flink/action/MigrateFileAction.java | 3 +- .../action/MigrateFileActionFactory.java | 4 +- .../flink/action/MigrateTableAction.java | 3 +- .../action/MigrateTableActionFactory.java | 10 +-- .../action/MultipleParameterToolAdapter.java | 11 ++++ .../action/QueryServiceActionFactory.java | 8 +-- .../flink/action/RemoveOrphanFilesAction.java | 3 +- .../RemoveOrphanFilesActionFactory.java | 17 ++--- .../paimon/flink/action/RenameTagAction.java | 3 +- .../flink/action/RenameTagActionFactory.java | 22 ++----- .../paimon/flink/action/RepairAction.java | 4 +- .../flink/action/RepairActionFactory.java | 5 +- .../paimon/flink/action/ReplaceTagAction.java | 3 +- .../flink/action/ReplaceTagActionFactory.java | 13 +--- .../flink/action/ResetConsumerAction.java | 3 +- .../action/ResetConsumerActionFactory.java | 14 ++-- .../flink/action/RewriteFileIndexAction.java | 5 +- .../action/RewriteFileIndexActionFactory.java | 6 +- .../paimon/flink/action/RollbackToAction.java | 3 +- .../flink/action/RollbackToActionFactory.java | 15 ++--- .../action/RollbackToTimestampAction.java | 3 +- .../RollbackToTimestampActionFactory.java | 19 ++---- .../flink/action/SortCompactAction.java | 3 +- .../paimon/flink/action/TableActionBase.java | 8 +-- .../flink/procedure/CloneProcedure.java | 29 ++++++-- .../procedure/CompactDatabaseProcedure.java | 3 +- .../flink/procedure/CompactProcedure.java | 3 - .../flink/procedure/MergeIntoProcedure.java | 6 +- .../flink/PrimaryKeyFileStoreTableITCase.java | 24 ++++++- .../flink/action/CompactActionITCase.java | 18 ++--- .../CreateTagFromTimestampActionITTest.java | 4 +- .../CreateTagFromWatermarkActionITTest.java | 16 +++-- .../flink/action/ExpireTagsActionTest.java | 12 +++- ...rtCompactActionForUnawareBucketITCase.java | 6 +- .../ExpireSnapshotsProcedureITCase.java | 24 ++++--- .../MigrateDatabaseProcedureITCase.java | 10 +-- .../procedure/MigrateFileProcedureITCase.java | 3 +- .../MigrateTableProcedureITCase.java | 10 +-- .../hive/procedure/RepairActionITCase.java | 8 +-- 108 files changed, 404 insertions(+), 629 deletions(-) diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java index 2d743218d40f..0d151e2d31dc 100644 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java +++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java @@ -47,7 +47,8 @@ public void testDeleteAction() throws Exception { batchSql("INSERT INTO T VALUES (1, 'Hi'), (2, 'Hello'), (3, 'World')"); DeleteAction action = - new DeleteAction(path, "default", "T", "k = 1", Collections.emptyMap()); + new DeleteAction( + "default", "T", "k = 1", Collections.singletonMap("warehouse", path)); BlockingIterator iterator = BlockingIterator.of(sEnv.executeSql("SELECT * FROM T").collect()); diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java index ac4340c11336..8935c8547aed 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java @@ -123,10 +123,9 @@ public String[] call( String partitionIdleTime, String compactStrategy) throws Exception { - String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); CompactDatabaseAction action = - new CompactDatabaseAction(warehouse, catalogOptions) + new CompactDatabaseAction(catalogOptions) .includingDatabases(nullable(includingDatabases)) .includingTables(nullable(includingTables)) .excludingTables(nullable(excludingTables)) diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index 560e532a6dbb..18e03e053cdd 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -139,8 +139,6 @@ public String[] call( String partitionIdleTime, String compactStrategy) throws Exception { - - String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); Map tableConf = StringUtils.isNullOrWhitespaceOnly(tableOptions) @@ -152,7 +150,6 @@ public String[] call( if (orderStrategy.isEmpty() && orderByColumns.isEmpty()) { action = new CompactAction( - warehouse, identifier.getDatabaseName(), identifier.getObjectName(), catalogOptions, @@ -171,7 +168,6 @@ public String[] call( "sort compact do not support 'partition_idle_time'."); action = new SortCompactAction( - warehouse, identifier.getDatabaseName(), identifier.getObjectName(), catalogOptions, diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java index acda2afd2e69..87af3adb331f 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java @@ -179,15 +179,11 @@ public String[] call( String notMatchedInsertCondition, String notMatchedInsertValues, String matchedDeleteCondition) { - String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); Identifier identifier = Identifier.fromString(targetTableId); MergeIntoAction action = new MergeIntoAction( - warehouse, - identifier.getDatabaseName(), - identifier.getObjectName(), - catalogOptions); + identifier.getDatabaseName(), identifier.getObjectName(), catalogOptions); action.withTargetAlias(nullable(targetAlias)); if (!sourceSqls.isEmpty()) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java index 3af0957ce6da..1e1671b4a305 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java @@ -54,13 +54,12 @@ public abstract class MessageQueueSyncTableActionBase extends SyncTableActionBase { public MessageQueueSyncTableActionBase( - String warehouse, String database, String table, Map catalogConfig, Map mqConfig, SyncJobHandler.SourceType sourceType) { - super(warehouse, database, table, catalogConfig, mqConfig, sourceType); + super(database, table, catalogConfig, mqConfig, sourceType); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java index 63e29d6a0ed8..fd9892c0f1d5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java @@ -66,13 +66,11 @@ public abstract class SyncDatabaseActionBase extends SynchronizationActionBase { protected Map> partitionKeyMultiple = new HashMap<>(); public SyncDatabaseActionBase( - String warehouse, String database, Map catalogConfig, Map cdcSourceConfig, SyncJobHandler.SourceType sourceType) { super( - warehouse, database, catalogConfig, cdcSourceConfig, diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java index c82039a9a021..4c7bd9cc1b88 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java @@ -42,13 +42,11 @@ public abstract class SyncDatabaseActionFactoryBase extends SynchronizationActionFactoryBase { - protected String warehouse; protected String database; @Override public Optional create(MultipleParameterToolAdapter params) { - this.warehouse = getRequiredValue(params, WAREHOUSE); - this.database = getRequiredValue(params, DATABASE); + this.database = params.getRequired(DATABASE); return super.create(params); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index 6fcdbd44bca2..d997fcc29040 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -59,14 +59,12 @@ public abstract class SyncTableActionBase extends SynchronizationActionBase { protected List computedColumns = new ArrayList<>(); public SyncTableActionBase( - String warehouse, String database, String table, Map catalogConfig, Map cdcSourceConfig, SyncJobHandler.SourceType sourceType) { super( - warehouse, database, catalogConfig, cdcSourceConfig, diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionFactoryBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionFactoryBase.java index 8f67ee16b51c..08b9131f719a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionFactoryBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionFactoryBase.java @@ -22,8 +22,6 @@ import org.apache.paimon.flink.action.ActionFactory; import org.apache.paimon.flink.action.MultipleParameterToolAdapter; -import org.apache.flink.api.java.tuple.Tuple3; - import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -39,11 +37,13 @@ public abstract class SyncTableActionFactoryBase extends SynchronizationActionFactoryBase { - protected Tuple3 tablePath; + protected String database; + protected String table; @Override public Optional create(MultipleParameterToolAdapter params) { - this.tablePath = getTablePath(params); + this.database = params.getRequired(DATABASE); + this.table = params.getRequired(TABLE); return super.create(params); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index d755b200a957..ebc051171de9 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -71,12 +71,11 @@ public abstract class SynchronizationActionBase extends ActionBase { protected CdcMetadataConverter[] metadataConverters = new CdcMetadataConverter[] {}; public SynchronizationActionBase( - String warehouse, String database, Map catalogConfig, Map cdcSourceConfig, SyncJobHandler syncJobHandler) { - super(warehouse, catalogConfig); + super(catalogConfig); this.database = database; this.cdcSourceConfig = Configuration.fromMap(cdcSourceConfig); this.syncJobHandler = syncJobHandler; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionFactoryBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionFactoryBase.java index a6482ab27f1d..0656f97c6216 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionFactoryBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionFactoryBase.java @@ -25,6 +25,8 @@ import java.util.Map; import java.util.Optional; +import static org.apache.paimon.utils.Preconditions.checkArgument; + /** Base {@link ActionFactory} for table/database synchronizing job. */ public abstract class SynchronizationActionFactoryBase implements ActionFactory { @@ -38,8 +40,11 @@ public abstract class SynchronizationActionFactoryBase create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, cdcConfigIdentifier()); - this.catalogConfig = optionalConfigMap(params, CATALOG_CONF); + checkArgument( + params.has(cdcConfigIdentifier()), + "Argument '%s' is required. Run ' --help' for help.", + cdcConfigIdentifier()); + this.catalogConfig = catalogConfigMap(params); this.cdcSourceConfig = optionalConfigMap(params, cdcConfigIdentifier()); T action = createAction(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java index 089bd035c325..d0d9e2b23c25 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java @@ -29,11 +29,8 @@ public class KafkaSyncDatabaseAction extends SyncDatabaseActionBase { public KafkaSyncDatabaseAction( - String warehouse, - String database, - Map catalogConfig, - Map kafkaConfig) { - super(warehouse, database, catalogConfig, kafkaConfig, SyncJobHandler.SourceType.KAFKA); + String database, Map catalogConfig, Map kafkaConfig) { + super(database, catalogConfig, kafkaConfig, SyncJobHandler.SourceType.KAFKA); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionFactory.java index dbf30b056ee3..4c0a14854d8d 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionFactory.java @@ -40,7 +40,7 @@ protected String cdcConfigIdentifier() { @Override public KafkaSyncDatabaseAction createAction() { - return new KafkaSyncDatabaseAction(warehouse, database, catalogConfig, cdcSourceConfig); + return new KafkaSyncDatabaseAction(database, catalogConfig, cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java index 0c2d8540730c..bff61ab0cde1 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java @@ -27,17 +27,10 @@ public class KafkaSyncTableAction extends MessageQueueSyncTableActionBase { public KafkaSyncTableAction( - String warehouse, String database, String table, Map catalogConfig, Map kafkaConfig) { - super( - warehouse, - database, - table, - catalogConfig, - kafkaConfig, - SyncJobHandler.SourceType.KAFKA); + super(database, table, catalogConfig, kafkaConfig, SyncJobHandler.SourceType.KAFKA); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionFactory.java index 1e3d015ed617..530e4af7c368 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionFactory.java @@ -40,11 +40,7 @@ public String cdcConfigIdentifier() { @Override public KafkaSyncTableAction createAction() { return new KafkaSyncTableAction( - this.tablePath.f0, - this.tablePath.f1, - this.tablePath.f2, - this.catalogConfig, - this.cdcSourceConfig); + this.database, this.table, this.catalogConfig, this.cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java index 3166a3c82ae5..31a216525ca8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java @@ -52,11 +52,8 @@ public class MongoDBSyncDatabaseAction extends SyncDatabaseActionBase { public MongoDBSyncDatabaseAction( - String warehouse, - String database, - Map catalogConfig, - Map mongodbConfig) { - super(warehouse, database, catalogConfig, mongodbConfig, SyncJobHandler.SourceType.MONGODB); + String database, Map catalogConfig, Map mongodbConfig) { + super(database, catalogConfig, mongodbConfig, SyncJobHandler.SourceType.MONGODB); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionFactory.java index d768b6740ccf..3d11e798d89b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionFactory.java @@ -40,7 +40,7 @@ protected String cdcConfigIdentifier() { @Override public MongoDBSyncDatabaseAction createAction() { - return new MongoDBSyncDatabaseAction(warehouse, database, catalogConfig, cdcSourceConfig); + return new MongoDBSyncDatabaseAction(database, catalogConfig, cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java index 34128a62fcde..a4ef4e0d6011 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java @@ -51,18 +51,11 @@ public class MongoDBSyncTableAction extends SyncTableActionBase { public MongoDBSyncTableAction( - String warehouse, String database, String table, Map catalogConfig, Map mongodbConfig) { - super( - warehouse, - database, - table, - catalogConfig, - mongodbConfig, - SyncJobHandler.SourceType.MONGODB); + super(database, table, catalogConfig, mongodbConfig, SyncJobHandler.SourceType.MONGODB); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionFactory.java index 79063c606f8c..e9909750beb8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionFactory.java @@ -46,11 +46,7 @@ public String cdcConfigIdentifier() { @Override public SyncTableActionBase createAction() { return new MongoDBSyncTableAction( - this.tablePath.f0, - this.tablePath.f1, - this.tablePath.f2, - this.catalogConfig, - this.cdcSourceConfig); + database, table, this.catalogConfig, this.cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index 0f452e2834be..790467ba0d87 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -102,11 +102,8 @@ public class MySqlSyncDatabaseAction extends SyncDatabaseActionBase { private final List excludedTables = new ArrayList<>(); public MySqlSyncDatabaseAction( - String warehouse, - String database, - Map catalogConfig, - Map mySqlConfig) { - super(warehouse, database, catalogConfig, mySqlConfig, SyncJobHandler.SourceType.MYSQL); + String database, Map catalogConfig, Map mySqlConfig) { + super(database, catalogConfig, mySqlConfig, SyncJobHandler.SourceType.MYSQL); this.mode = DIVIDED; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionFactory.java index 316e3f7822a5..74d0b6f23441 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionFactory.java @@ -49,7 +49,7 @@ protected String cdcConfigIdentifier() { @Override public MySqlSyncDatabaseAction createAction() { - return new MySqlSyncDatabaseAction(warehouse, database, catalogConfig, cdcSourceConfig); + return new MySqlSyncDatabaseAction(database, catalogConfig, cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java index d73d9702f1e1..eee640217559 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java @@ -76,18 +76,11 @@ public class MySqlSyncTableAction extends SyncTableActionBase { private JdbcSchemasInfo mySqlSchemasInfo; public MySqlSyncTableAction( - String warehouse, String database, String table, Map catalogConfig, Map mySqlConfig) { - super( - warehouse, - database, - table, - catalogConfig, - mySqlConfig, - SyncJobHandler.SourceType.MYSQL); + super(database, table, catalogConfig, mySqlConfig, SyncJobHandler.SourceType.MYSQL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionFactory.java index 141b7b73e1f7..f4bb73861754 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionFactory.java @@ -39,12 +39,7 @@ public String cdcConfigIdentifier() { @Override public MySqlSyncTableAction createAction() { - return new MySqlSyncTableAction( - this.tablePath.f0, - this.tablePath.f1, - this.tablePath.f2, - this.catalogConfig, - this.cdcSourceConfig); + return new MySqlSyncTableAction(database, table, this.catalogConfig, this.cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java index 7dc1b019cd02..01755e6d0b18 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java @@ -77,18 +77,11 @@ public class PostgresSyncTableAction extends SyncTableActionBase { private JdbcSchemasInfo postgresSchemasInfo; public PostgresSyncTableAction( - String warehouse, String database, String table, Map catalogConfig, Map postgresConfig) { - super( - warehouse, - database, - table, - catalogConfig, - postgresConfig, - SyncJobHandler.SourceType.POSTGRES); + super(database, table, catalogConfig, postgresConfig, SyncJobHandler.SourceType.POSTGRES); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionFactory.java index b40a480f839a..a68f6366e5d2 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionFactory.java @@ -40,11 +40,7 @@ public String cdcConfigIdentifier() { @Override public PostgresSyncTableAction createAction() { return new PostgresSyncTableAction( - this.tablePath.f0, - this.tablePath.f1, - this.tablePath.f2, - this.catalogConfig, - this.cdcSourceConfig); + database, table, this.catalogConfig, this.cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseAction.java index 0b99040f1319..70dbb1ab01bd 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseAction.java @@ -29,11 +29,8 @@ public class PulsarSyncDatabaseAction extends SyncDatabaseActionBase { public PulsarSyncDatabaseAction( - String warehouse, - String database, - Map catalogConfig, - Map pulsarConfig) { - super(warehouse, database, catalogConfig, pulsarConfig, SyncJobHandler.SourceType.PULSAR); + String database, Map catalogConfig, Map pulsarConfig) { + super(database, catalogConfig, pulsarConfig, SyncJobHandler.SourceType.PULSAR); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionFactory.java index adebef7275c8..26af229c5713 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncDatabaseActionFactory.java @@ -40,7 +40,7 @@ protected String cdcConfigIdentifier() { @Override public PulsarSyncDatabaseAction createAction() { - return new PulsarSyncDatabaseAction(warehouse, database, catalogConfig, cdcSourceConfig); + return new PulsarSyncDatabaseAction(database, catalogConfig, cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableAction.java index c94c4581af53..0b39887f9a88 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableAction.java @@ -27,17 +27,10 @@ public class PulsarSyncTableAction extends MessageQueueSyncTableActionBase { public PulsarSyncTableAction( - String warehouse, String database, String table, Map catalogConfig, Map pulsarConfig) { - super( - warehouse, - database, - table, - catalogConfig, - pulsarConfig, - SyncJobHandler.SourceType.PULSAR); + super(database, table, catalogConfig, pulsarConfig, SyncJobHandler.SourceType.PULSAR); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionFactory.java index f048abac5743..25822a7b87a1 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarSyncTableActionFactory.java @@ -39,12 +39,7 @@ public String cdcConfigIdentifier() { @Override public PulsarSyncTableAction createAction() { - return new PulsarSyncTableAction( - this.tablePath.f0, - this.tablePath.f1, - this.tablePath.f2, - this.catalogConfig, - this.cdcSourceConfig); + return new PulsarSyncTableAction(database, table, this.catalogConfig, this.cdcSourceConfig); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java index 5247225caff4..6ebfdb7550fa 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBaseTest.java @@ -33,6 +33,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -63,7 +64,9 @@ public void setUp() throws Exception { kafkaSyncDbAction = new KafkaSyncDatabaseAction( - tmp.toString(), "default", new HashMap<>(), new HashMap<>()); + "default", + Collections.singletonMap("warehouse", tmp.toString()), + new HashMap<>()); Map rawData = new HashMap<>(); rawData.put("field", "value"); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java index 4490023e7b03..1d7547a358cc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java @@ -24,7 +24,6 @@ import org.apache.paimon.flink.FlinkCatalog; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.LogicalTypeConversion; -import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeCasts; @@ -54,9 +53,8 @@ public abstract class ActionBase implements Action { protected StreamExecutionEnvironment env; protected StreamTableEnvironment batchTEnv; - public ActionBase(String warehouse, Map catalogConfig) { + public ActionBase(Map catalogConfig) { catalogOptions = Options.fromMap(catalogConfig); - catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse); // disable cache to avoid concurrent modification exception if (!catalogOptions.contains(CACHE_ENABLED)) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java index fbf8f12f49eb..71298d492086 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java @@ -18,13 +18,11 @@ package org.apache.paimon.flink.action; -import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.factories.Factory; import org.apache.paimon.factories.FactoryException; import org.apache.paimon.factories.FactoryUtil; -import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.options.CatalogOptions; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.utils.MultipleParameterTool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +48,7 @@ public interface ActionFactory extends Factory { String WAREHOUSE = "warehouse"; String DATABASE = "database"; String TABLE = "table"; - String PATH = "path"; + @Deprecated String PATH = "path"; String CATALOG_CONF = "catalog_conf"; String TABLE_CONF = "table_conf"; String PARTITION = "partition"; @@ -88,6 +86,14 @@ static Optional createAction(String[] args) { return Optional.empty(); } + if (params.has(PATH)) { + throw new UnsupportedOperationException( + String.format( + "Parameter '%s' is deprecated. Please use '--%s %s=' to specify warehouse if needed, " + + "and use '%s' to specify database and '%s' to specify table.", + PATH, CATALOG_CONF, CatalogOptions.WAREHOUSE.key(), DATABASE, TABLE)); + } + return actionFactory.create(params); } @@ -105,39 +111,6 @@ static void printDefaultHelp() { System.out.println("For detailed options of each action, run --help"); } - default Tuple3 getTablePath(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - String database = params.get(DATABASE); - String table = params.get(TABLE); - String path = params.get(PATH); - - Tuple3 tablePath = null; - int count = 0; - if (warehouse != null || database != null || table != null) { - if (warehouse == null || database == null || table == null) { - throw new IllegalArgumentException( - "Warehouse, database and table must be specified all at once to specify a table."); - } - tablePath = Tuple3.of(warehouse, database, table); - count++; - } - if (path != null) { - tablePath = - Tuple3.of( - CatalogUtils.warehouse(path), - CatalogUtils.database(path), - CatalogUtils.table(path)); - count++; - } - - if (count != 1) { - throw new IllegalArgumentException( - "Please specify either \"warehouse, database and table\" or \"path\"."); - } - - return tablePath; - } - default List> getPartitions(MultipleParameterToolAdapter params) { List> partitions = new ArrayList<>(); for (String partition : params.getMultiParameter(PARTITION)) { @@ -160,16 +133,6 @@ default Map optionalConfigMap(MultipleParameterToolAdapter param return config; } - default void checkRequiredArgument(MultipleParameterToolAdapter params, String key) { - Preconditions.checkArgument( - params.has(key), "Argument '%s' is required. Run ' --help' for help.", key); - } - - default String getRequiredValue(MultipleParameterToolAdapter params, String key) { - checkRequiredArgument(params, key); - return params.get(key); - } - default Map> optionalConfigMapList( MultipleParameterToolAdapter params, String key) { if (!params.has(key)) { @@ -182,4 +145,13 @@ default Map> optionalConfigMapList( } return config; } + + default Map catalogConfigMap(MultipleParameterToolAdapter params) { + Map catalogConfig = new HashMap<>(optionalConfigMap(params, CATALOG_CONF)); + String warehouse = params.get(WAREHOUSE); + if (warehouse != null && !catalogConfig.containsKey(WAREHOUSE)) { + catalogConfig.put(WAREHOUSE, warehouse); + } + return catalogConfig; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java index bac030dd0496..15b90ec83411 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java @@ -25,7 +25,6 @@ import org.apache.paimon.flink.clone.SnapshotHintChannelComputer; import org.apache.paimon.flink.clone.SnapshotHintOperator; import org.apache.paimon.flink.sink.FlinkStreamPartitioner; -import org.apache.paimon.options.CatalogOptions; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -37,7 +36,6 @@ import java.util.HashMap; import java.util.Map; -import static org.apache.paimon.utils.Preconditions.checkNotNull; import static org.apache.paimon.utils.StringUtils.isNullOrWhitespaceOnly; /** The Latest Snapshot clone action for Flink. */ @@ -54,19 +52,14 @@ public class CloneAction extends ActionBase { private final String targetTableName; public CloneAction( - String warehouse, String database, String tableName, Map sourceCatalogConfig, - String targetWarehouse, String targetDatabase, String targetTableName, Map targetCatalogConfig, String parallelismStr) { - super(warehouse, sourceCatalogConfig); - - checkNotNull(warehouse, "warehouse must not be null."); - checkNotNull(targetWarehouse, "targetWarehouse must not be null."); + super(sourceCatalogConfig); this.parallelism = isNullOrWhitespaceOnly(parallelismStr) @@ -77,7 +70,6 @@ public CloneAction( if (!sourceCatalogConfig.isEmpty()) { this.sourceCatalogConfig = sourceCatalogConfig; } - this.sourceCatalogConfig.put(CatalogOptions.WAREHOUSE.key(), warehouse); this.database = database; this.tableName = tableName; @@ -85,7 +77,6 @@ public CloneAction( if (!targetCatalogConfig.isEmpty()) { this.targetCatalogConfig = targetCatalogConfig; } - this.targetCatalogConfig.put(CatalogOptions.WAREHOUSE.key(), targetWarehouse); this.targetDatabase = targetDatabase; this.targetTableName = targetTableName; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java index db45c8508447..4fb5081ee8f2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java @@ -18,6 +18,8 @@ package org.apache.paimon.flink.action; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; /** Factory to create {@link CloneAction}. */ @@ -37,16 +39,23 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { + Map catalogConfig = catalogConfigMap(params); + + Map targetCatalogConfig = + new HashMap<>(optionalConfigMap(params, TARGET_CATALOG_CONF)); + String targetWarehouse = params.get(TARGET_WAREHOUSE); + if (targetWarehouse != null && !targetCatalogConfig.containsKey(WAREHOUSE)) { + targetCatalogConfig.put(WAREHOUSE, targetWarehouse); + } + CloneAction cloneAction = new CloneAction( - params.get(WAREHOUSE), params.get(DATABASE), params.get(TABLE), - optionalConfigMap(params, CATALOG_CONF), - params.get(TARGET_WAREHOUSE), + catalogConfig, params.get(TARGET_DATABASE), params.get(TARGET_TABLE), - optionalConfigMap(params, TARGET_CATALOG_CONF), + targetCatalogConfig, params.get(PARALLELISM)); return Optional.of(cloneAction); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java index 84e37a5b10f9..73c96b2c4bb1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java @@ -41,7 +41,6 @@ import javax.annotation.Nullable; import java.time.Duration; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -61,17 +60,12 @@ public class CompactAction extends TableActionBase { private Boolean fullCompaction; - public CompactAction(String warehouse, String database, String tableName) { - this(warehouse, database, tableName, Collections.emptyMap(), Collections.emptyMap()); - } - public CompactAction( - String warehouse, String database, String tableName, Map catalogConfig, Map tableConf) { - super(warehouse, database, tableName, catalogConfig); + super(database, tableName, catalogConfig); if (!(table instanceof FileStoreTable)) { throw new UnsupportedOperationException( String.format( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java index fc60a870eabe..ea0e48db2213 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java @@ -21,8 +21,6 @@ import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.TimeUtils; -import org.apache.flink.api.java.tuple.Tuple3; - import java.util.List; import java.util.Map; import java.util.Optional; @@ -46,9 +44,10 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + String database = params.getRequired(DATABASE); + String table = params.getRequired(TABLE); + Map catalogConfig = catalogConfigMap(params); + Map tableConfig = optionalConfigMap(params, TABLE_CONF); CompactAction action; if (params.has(ORDER_STRATEGY)) { @@ -56,22 +55,11 @@ public Optional create(MultipleParameterToolAdapter params) { !params.has(PARTITION_IDLE_TIME), "sort compact do not support 'partition_idle_time'."); action = - new SortCompactAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - optionalConfigMap(params, TABLE_CONF)) + new SortCompactAction(database, table, catalogConfig, tableConfig) .withOrderStrategy(params.get(ORDER_STRATEGY)) - .withOrderColumns(getRequiredValue(params, ORDER_BY).split(",")); + .withOrderColumns(params.getRequired(ORDER_BY).split(",")); } else { - action = - new CompactAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - optionalConfigMap(params, TABLE_CONF)); + action = new CompactAction(database, table, catalogConfig, tableConfig); if (params.has(PARTITION_IDLE_TIME)) { action.withPartitionIdleTime( TimeUtils.parseDuration(params.get(PARTITION_IDLE_TIME))); @@ -123,7 +111,6 @@ public void printHelp() { System.out.println( " compact --warehouse s3://path/to/warehouse --database " + "--table [--catalog_conf [--catalog_conf ...]]"); - System.out.println(" compact --path [--partition ]"); System.out.println(); System.out.println("Partition name syntax:"); @@ -142,8 +129,6 @@ public void printHelp() { System.out.println("Examples:"); System.out.println( " compact --warehouse hdfs:///path/to/warehouse --database test_db --table test_table"); - System.out.println( - " compact --path hdfs:///path/to/warehouse/test_db.db/test_table --partition dt=20221126,hh=08"); System.out.println( " compact --warehouse hdfs:///path/to/warehouse --database test_db --table test_table " + "--partition dt=20221126,hh=08 --partition dt=20221127,hh=09"); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java index 124d3ca68776..457e04bfd8ff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java @@ -76,8 +76,8 @@ public class CompactDatabaseAction extends ActionBase { private boolean isStreaming; - public CompactDatabaseAction(String warehouse, Map catalogConfig) { - super(warehouse, catalogConfig); + public CompactDatabaseAction(Map catalogConfig) { + super(catalogConfig); } public CompactDatabaseAction includingDatabases(@Nullable String includingDatabases) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java index 5672f99dc30f..8fa5a1cf46ec 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java @@ -42,10 +42,7 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - CompactDatabaseAction action = - new CompactDatabaseAction( - getRequiredValue(params, WAREHOUSE), - optionalConfigMap(params, CATALOG_CONF)); + CompactDatabaseAction action = new CompactDatabaseAction(catalogConfigMap(params)); action.includingDatabases(params.get(INCLUDING_DATABASES)) .includingTables(params.get(INCLUDING_TABLES)) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchAction.java index aa8cc697ae36..c8eb1d83a41b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchAction.java @@ -28,13 +28,12 @@ public class CreateBranchAction extends TableActionBase { private final String tagName; public CreateBranchAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String branchName, String tagName) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.branchName = branchName; this.tagName = tagName; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchActionFactory.java index d1071d0870ad..a95a1d5a36ae 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateBranchActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link CreateBranchAction}. */ @@ -38,26 +35,13 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, BRANCH_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - - String tagName = null; - if (params.has(TAG_NAME)) { - tagName = params.get(TAG_NAME); - } - - String branchName = params.get(BRANCH_NAME); - CreateBranchAction action = new CreateBranchAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - branchName, - tagName); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(BRANCH_NAME), + params.get(TAG_NAME)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java index fecb6895b682..d6ff60c3fabf 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java @@ -20,8 +20,6 @@ import org.apache.paimon.utils.TimeUtils; -import org.apache.flink.api.java.tuple.Tuple3; - import java.time.Duration; import java.util.Map; import java.util.Optional; @@ -35,11 +33,8 @@ public abstract class CreateOrReplaceTagActionFactory implements ActionFactory { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, TAG_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String tagName = params.get(TAG_NAME); + Map catalogConfig = catalogConfigMap(params); + String tagName = params.getRequired(TAG_NAME); Long snapshot = null; if (params.has(SNAPSHOT)) { @@ -53,11 +48,17 @@ public Optional create(MultipleParameterToolAdapter params) { return Optional.of( createOrReplaceTagAction( - tablePath, catalogConfig, tagName, snapshot, timeRetained)); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfig, + tagName, + snapshot, + timeRetained)); } abstract Action createOrReplaceTagAction( - Tuple3 tablePath, + String database, + String table, Map catalogConfig, String tagName, Long snapshot, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagAction.java index cfc9b558b40c..7628fda10618 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagAction.java @@ -31,14 +31,13 @@ public class CreateTagAction extends TableActionBase { private final @Nullable Duration timeRetained; public CreateTagAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String tagName, @Nullable Long snapshotId, @Nullable Duration timeRetained) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.tagName = tagName; this.timeRetained = timeRetained; this.snapshotId = snapshotId; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java index c525943122bc..76669d31401a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java @@ -18,8 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - import java.time.Duration; import java.util.Map; @@ -35,19 +33,13 @@ public String identifier() { @Override Action createOrReplaceTagAction( - Tuple3 tablePath, + String database, + String table, Map catalogConfig, String tagName, Long snapshot, Duration timeRetained) { - return new CreateTagAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - tagName, - snapshot, - timeRetained); + return new CreateTagAction(database, table, catalogConfig, tagName, snapshot, timeRetained); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampAction.java index 632c41fc0f66..14a4e01fd1ea 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampAction.java @@ -26,19 +26,22 @@ /** Create tag from timestamp action for Flink. */ public class CreateTagFromTimestampAction extends ActionBase { + + private final String database; private final String table; private final String tag; private final Long timestamp; private final String timeRetained; public CreateTagFromTimestampAction( - String warehouse, + String database, String table, String tag, Long timestamp, String timeRetained, Map catalogConfig) { - super(warehouse, catalogConfig); + super(catalogConfig); + this.database = database; this.table = table; this.tag = tag; this.timestamp = timestamp; @@ -51,6 +54,10 @@ public void run() throws Exception { new CreateTagFromTimestampProcedure(); createTagFromTimestampProcedure.withCatalog(catalog); createTagFromTimestampProcedure.call( - new DefaultProcedureContext(env), table, tag, timestamp, timeRetained); + new DefaultProcedureContext(env), + database + "." + table, + tag, + timestamp, + timeRetained); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionFactory.java index 2d2fae73925d..0c5f477f30c5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionFactory.java @@ -25,8 +25,6 @@ public class CreateTagFromTimestampActionFactory implements ActionFactory { public static final String IDENTIFIER = "create_tag_from_timestamp"; - private static final String TABLE = "table"; - private static final String TAG = "tag"; private static final String TIMESTAMP = "timestamp"; @@ -40,16 +38,18 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - String table = params.get(TABLE); - String tag = params.get(TAG); Long timestamp = Long.parseLong(params.get(TIMESTAMP)); String timeRetained = params.get(TIME_RETAINED); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); CreateTagFromTimestampAction createTagFromTimestampAction = new CreateTagFromTimestampAction( - warehouse, table, tag, timestamp, timeRetained, catalogConfig); + params.getRequired(DATABASE), + params.getRequired(TABLE), + params.getRequired(TAG), + timestamp, + timeRetained, + catalogConfig); return Optional.of(createTagFromTimestampAction); } @@ -61,7 +61,8 @@ public void printHelp() { System.out.println("Syntax:"); System.out.println( " create_tag_from_timestamp --warehouse " - + "--table " + + "--database " + + "--table " + "--tag " + "--timestamp " + "[--timeRetained ] " diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkAction.java index 8afa54082f29..af75865eef02 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkAction.java @@ -26,19 +26,22 @@ /** Create tag from watermark action for Flink. */ public class CreateTagFromWatermarkAction extends ActionBase { + + private final String database; private final String table; private final String tag; private final Long watermark; private final String timeRetained; public CreateTagFromWatermarkAction( - String warehouse, + String database, String table, String tag, Long watermark, String timeRetained, Map catalogConfig) { - super(warehouse, catalogConfig); + super(catalogConfig); + this.database = database; this.table = table; this.tag = tag; this.watermark = watermark; @@ -51,6 +54,10 @@ public void run() throws Exception { new CreateTagFromWatermarkProcedure(); createTagFromWatermarkProcedure.withCatalog(catalog); createTagFromWatermarkProcedure.call( - new DefaultProcedureContext(env), table, tag, watermark, timeRetained); + new DefaultProcedureContext(env), + database + "." + table, + tag, + watermark, + timeRetained); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionFactory.java index 1fb86bde7ebb..282ce67246b3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionFactory.java @@ -26,8 +26,6 @@ public class CreateTagFromWatermarkActionFactory implements ActionFactory { public static final String IDENTIFIER = "create_tag_from_watermark"; - private static final String TABLE = "table"; - private static final String TAG = "tag"; private static final String WATERMARK = "watermark"; @@ -41,16 +39,19 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - String table = params.get(TABLE); String tag = params.get(TAG); Long watermark = Long.parseLong(params.get(WATERMARK)); String timeRetained = params.get(TIME_RETAINED); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); CreateTagFromWatermarkAction createTagFromWatermarkAction = new CreateTagFromWatermarkAction( - warehouse, table, tag, watermark, timeRetained, catalogConfig); + params.getRequired(DATABASE), + params.getRequired(TABLE), + tag, + watermark, + timeRetained, + catalogConfig); return Optional.of(createTagFromWatermarkAction); } @@ -62,7 +63,8 @@ public void printHelp() { System.out.println("Syntax:"); System.out.println( " create_tag_from_watermark --warehouse " - + "--table " + + "--database " + + "--table " + "--tag " + "--watermark " + "[--timeRetained ] " diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteAction.java index c275ce6f1fe0..ed9423cf528c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteAction.java @@ -44,12 +44,11 @@ public class DeleteAction extends TableActionBase { private final String filter; public DeleteAction( - String warehouse, String databaseName, String tableName, String filter, Map catalogConfig) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.filter = filter; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteActionFactory.java index 7f8f63f7f601..1f116e6097be 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link DeleteAction}. */ @@ -37,18 +34,18 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - String filter = params.get(WHERE); if (filter == null) { throw new IllegalArgumentException( "Please specify deletion filter. If you want to delete all records, please use overwrite (see doc)."); } - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - DeleteAction action = - new DeleteAction(tablePath.f0, tablePath.f1, tablePath.f2, filter, catalogConfig); + new DeleteAction( + params.getRequired(DATABASE), + params.getRequired(TABLE), + filter, + catalogConfigMap(params)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchAction.java index 7373f8fff773..9cec62b99a53 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchAction.java @@ -26,12 +26,11 @@ public class DeleteBranchAction extends TableActionBase { private final String branchNames; public DeleteBranchAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String branchNames) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.branchNames = branchNames; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchActionFactory.java index 33f1c7990683..eedc16023437 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteBranchActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link DeleteBranchAction}. */ @@ -37,15 +34,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, BRANCH_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String branchName = params.get(BRANCH_NAME); - DeleteBranchAction action = new DeleteBranchAction( - tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig, branchName); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(BRANCH_NAME)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagAction.java index 73cf21033d64..3e73a7f18265 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagAction.java @@ -26,12 +26,11 @@ public class DeleteTagAction extends TableActionBase { private final String tagNameStr; public DeleteTagAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String tagNameStr) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.tagNameStr = tagNameStr; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagActionFactory.java index 56b724c0ee29..fcb3c77a359a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DeleteTagActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link DeleteTagAction}. */ @@ -37,15 +34,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, TAG_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String tagName = params.get(TAG_NAME); - DeleteTagAction action = new DeleteTagAction( - tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig, tagName); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(TAG_NAME)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionAction.java index 2ead85001d6d..6f568dfe61bc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionAction.java @@ -34,12 +34,11 @@ public class DropPartitionAction extends TableActionBase { private final FileStoreCommit commit; public DropPartitionAction( - String warehouse, String databaseName, String tableName, List> partitions, Map catalogConfig) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); if (!(table instanceof FileStoreTable)) { throw new UnsupportedOperationException( String.format( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionActionFactory.java index e4b29dde0758..e70dcc4a77d1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/DropPartitionActionFactory.java @@ -18,12 +18,12 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - import java.util.List; import java.util.Map; import java.util.Optional; +import static org.apache.paimon.utils.Preconditions.checkArgument; + /** Factory to create {@link DropPartitionAction}. */ public class DropPartitionActionFactory implements ActionFactory { @@ -36,16 +36,18 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - checkRequiredArgument(params, PARTITION); + checkArgument( + params.has(PARTITION), + "Argument '%s' is required. Run ' --help' for help.", + PARTITION); List> partitions = getPartitions(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - return Optional.of( new DropPartitionAction( - tablePath.f0, tablePath.f1, tablePath.f2, partitions, catalogConfig)); + params.getRequired(DATABASE), + params.getRequired(TABLE), + partitions, + catalogConfigMap(params))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java index 0fa17e1a8ddb..0dc96bd93a7e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java @@ -37,7 +37,6 @@ public class ExpirePartitionsAction extends TableActionBase { private final PartitionExpire partitionExpire; public ExpirePartitionsAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, @@ -45,7 +44,7 @@ public ExpirePartitionsAction( String timestampFormatter, String timestampPattern, String expireStrategy) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); if (!(table instanceof FileStoreTable)) { throw new UnsupportedOperationException( String.format( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsActionFactory.java index 3d0dfc265983..2cf63d3bc775 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link ExpirePartitionsAction}. */ @@ -35,25 +32,16 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - checkRequiredArgument(params, EXPIRATIONTIME); - checkRequiredArgument(params, TIMESTAMPFORMATTER); - String expirationTime = params.get(EXPIRATIONTIME); - String timestampFormatter = params.get(TIMESTAMPFORMATTER); String expireStrategy = params.get(EXPIRE_STRATEGY); String timestampPattern = params.get(TIMESTAMP_PATTERN); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - return Optional.of( new ExpirePartitionsAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - expirationTime, - timestampFormatter, + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(EXPIRATIONTIME), + params.getRequired(TIMESTAMPFORMATTER), timestampPattern, expireStrategy)); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java index 92848c804852..5ae5c486d7ce 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java @@ -27,22 +27,24 @@ /** Expire snapshots action for Flink. */ public class ExpireSnapshotsAction extends ActionBase { - private final String identifier; + private final String database; + private final String table; private final Integer retainMax; private final Integer retainMin; private final String olderThan; private final Integer maxDeletes; public ExpireSnapshotsAction( - String warehouse, - String identifier, + String database, + String table, Map catalogConfig, Integer retainMax, Integer retainMin, String olderThan, Integer maxDeletes) { - super(warehouse, catalogConfig); - this.identifier = identifier; + super(catalogConfig); + this.database = database; + this.table = table; this.retainMax = retainMax; this.retainMin = retainMin; this.olderThan = olderThan; @@ -54,7 +56,7 @@ public void run() throws Exception { expireSnapshotsProcedure.withCatalog(catalog); expireSnapshotsProcedure.call( new DefaultProcedureContext(env), - identifier, + database + "." + table, retainMax, retainMin, olderThan, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsActionFactory.java index 1f29d3a71326..24e25844718a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsActionFactory.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.action; -import java.util.Map; import java.util.Optional; /** Factory to create {@link ExpireSnapshotsAction}. */ @@ -40,10 +39,6 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String identifier = params.get(IDENTIFIER_KEY); - Integer retainMax = params.has(RETAIN_MAX) ? Integer.parseInt(params.get(RETAIN_MAX)) : null; Integer retainMin = @@ -54,9 +49,9 @@ public Optional create(MultipleParameterToolAdapter params) { ExpireSnapshotsAction action = new ExpireSnapshotsAction( - warehouse, - identifier, - catalogConfig, + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), retainMax, retainMin, olderThan, @@ -72,6 +67,6 @@ public void printHelp() { System.out.println("Syntax:"); System.out.println( - " expire_snapshots --warehouse --identifier --retain_max --retain_min --older_than --max_delete "); + " expire_snapshots --warehouse --database --table --retain_max --retain_min --older_than --max_delete "); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java index c1231ed3ad54..9e26e2121939 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java @@ -27,12 +27,14 @@ /** Expire tags action for Flink. */ public class ExpireTagsAction extends ActionBase { + private final String database; private final String table; private final String olderThan; public ExpireTagsAction( - String warehouse, String table, String olderThan, Map catalogConfig) { - super(warehouse, catalogConfig); + String database, String table, String olderThan, Map catalogConfig) { + super(catalogConfig); + this.database = database; this.table = table; this.olderThan = olderThan; } @@ -41,6 +43,7 @@ public ExpireTagsAction( public void run() throws Exception { ExpireTagsProcedure expireTagsProcedure = new ExpireTagsProcedure(); expireTagsProcedure.withCatalog(catalog); - expireTagsProcedure.call(new DefaultProcedureContext(env), table, olderThan); + expireTagsProcedure.call( + new DefaultProcedureContext(env), database + "." + table, olderThan); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.java index e9bbb0a3bdc7..01f7a2ecd633 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.action; -import java.util.Map; import java.util.Optional; /** Factory to create {@link ExpireTagsAction}. */ @@ -35,13 +34,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - String table = params.get(TABLE); - String olderThan = params.get(OLDER_THAN); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - ExpireTagsAction expireTagsAction = - new ExpireTagsAction(warehouse, table, olderThan, catalogConfig); + new ExpireTagsAction( + params.getRequired(DATABASE), + params.getRequired(TABLE), + params.get(OLDER_THAN), + catalogConfigMap(params)); return Optional.of(expireTagsAction); } @@ -53,7 +51,8 @@ public void printHelp() { System.out.println("Syntax:"); System.out.println( " expire_tags --warehouse " - + "--table " + + "--database " + + "--table
" + "[--older_than ]"); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardAction.java index b13c004c0d96..0aaa44e6e356 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardAction.java @@ -25,12 +25,11 @@ public class FastForwardAction extends TableActionBase { private final String branchName; public FastForwardAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String branchName) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.branchName = branchName; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardActionFactory.java index b1e2228c6592..9d984f3c9505 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FastForwardActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link FastForwardAction}. */ @@ -37,15 +34,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, BRANCH_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String branchName = params.get(BRANCH_NAME); - FastForwardAction action = new FastForwardAction( - tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig, branchName); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(BRANCH_NAME)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneAction.java index 9fd906ee44dc..25cd14af2195 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneAction.java @@ -35,12 +35,11 @@ public class MarkPartitionDoneAction extends TableActionBase { private final List> partitions; public MarkPartitionDoneAction( - String warehouse, String databaseName, String tableName, List> partitions, Map catalogConfig) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); if (!(table instanceof FileStoreTable)) { throw new UnsupportedOperationException( String.format( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneActionFactory.java index 8de5a7130b14..78929e1b76c4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MarkPartitionDoneActionFactory.java @@ -18,12 +18,12 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - import java.util.List; import java.util.Map; import java.util.Optional; +import static org.apache.paimon.utils.Preconditions.checkArgument; + /** Factory to create {@link MarkPartitionDoneAction}. */ public class MarkPartitionDoneActionFactory implements ActionFactory { @@ -36,16 +36,18 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - checkRequiredArgument(params, PARTITION); + checkArgument( + params.has(PARTITION), + "Argument '%s' is required. Run ' --help' for help.", + PARTITION); List> partitions = getPartitions(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - return Optional.of( new MarkPartitionDoneAction( - tablePath.f0, tablePath.f1, tablePath.f2, partitions, catalogConfig)); + params.getRequired(DATABASE), + params.getRequired(TABLE), + partitions, + catalogConfigMap(params))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoAction.java index f99b2297392c..1ecd23ea6246 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoAction.java @@ -120,16 +120,8 @@ public class MergeIntoAction extends TableActionBase { @Nullable private String notMatchedInsertCondition; @Nullable private String notMatchedInsertValues; - public MergeIntoAction(String warehouse, String database, String tableName) { - this(warehouse, database, tableName, Collections.emptyMap()); - } - - public MergeIntoAction( - String warehouse, - String database, - String tableName, - Map catalogConfig) { - super(warehouse, database, tableName, catalogConfig); + public MergeIntoAction(String database, String tableName, Map catalogConfig) { + super(database, tableName, catalogConfig); if (!(table instanceof FileStoreTable)) { throw new UnsupportedOperationException( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoActionFactory.java index 0ff0db3271ee..830c0708d911 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MergeIntoActionFactory.java @@ -18,12 +18,9 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -62,12 +59,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); MergeIntoAction action = - new MergeIntoAction(tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig); + new MergeIntoAction( + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params)); if (params.has(TARGET_AS)) { action.withTargetAlias(params.get(TARGET_AS)); @@ -78,26 +75,22 @@ public Optional create(MultipleParameterToolAdapter params) { action.withSourceSqls(sourceSqls.toArray(new String[0])); } - checkRequiredArgument(params, SOURCE_TABLE); - action.withSourceTable(params.get(SOURCE_TABLE)); + action.withSourceTable(params.getRequired(SOURCE_TABLE)); - checkRequiredArgument(params, ON); - action.withMergeCondition(params.get(ON)); + action.withMergeCondition(params.getRequired(ON)); List actions = Arrays.stream(params.get(MERGE_ACTIONS).split(",")) .map(String::trim) .collect(Collectors.toList()); if (actions.contains(MATCHED_UPSERT)) { - checkRequiredArgument(params, MATCHED_UPSERT_SET); action.withMatchedUpsert( - params.get(MATCHED_UPSERT_CONDITION), params.get(MATCHED_UPSERT_SET)); + params.get(MATCHED_UPSERT_CONDITION), params.getRequired(MATCHED_UPSERT_SET)); } if (actions.contains(NOT_MATCHED_BY_SOURCE_UPSERT)) { - checkRequiredArgument(params, NOT_MATCHED_BY_SOURCE_UPSERT_SET); action.withNotMatchedBySourceUpsert( params.get(NOT_MATCHED_BY_SOURCE_UPSERT_CONDITION), - params.get(NOT_MATCHED_BY_SOURCE_UPSERT_SET)); + params.getRequired(NOT_MATCHED_BY_SOURCE_UPSERT_SET)); } if (actions.contains(MATCHED_DELETE)) { action.withMatchedDelete(params.get(MATCHED_DELETE_CONDITION)); @@ -106,10 +99,9 @@ public Optional create(MultipleParameterToolAdapter params) { action.withNotMatchedBySourceDelete(params.get(NOT_MATCHED_BY_SOURCE_DELETE_CONDITION)); } if (actions.contains(NOT_MATCHED_INSERT)) { - checkRequiredArgument(params, NOT_MATCHED_INSERT_VALUES); action.withNotMatchedInsert( params.get(NOT_MATCHED_INSERT_CONDITION), - params.get(NOT_MATCHED_INSERT_VALUES)); + params.getRequired(NOT_MATCHED_INSERT_VALUES)); } action.validate(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseAction.java index 6a8afd206091..c7f5445c19be 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseAction.java @@ -33,12 +33,11 @@ public class MigrateDatabaseAction extends ActionBase { public MigrateDatabaseAction( String connector, - String warehouse, String hiveDatabaseName, Map catalogConfig, String tableProperties, Integer parallelism) { - super(warehouse, catalogConfig); + super(catalogConfig); this.connector = connector; this.hiveDatabaseName = hiveDatabaseName; this.tableProperties = tableProperties; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseActionFactory.java index 8ce33f58e644..33a4524ed701 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateDatabaseActionFactory.java @@ -37,21 +37,15 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); String connector = params.get(SOURCE_TYPE); String sourceHiveDatabase = params.get(DATABASE); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); String tableConf = params.get(OPTIONS); Integer parallelism = Integer.parseInt(params.get(PARALLELISM)); MigrateDatabaseAction migrateDatabaseAction = new MigrateDatabaseAction( - connector, - warehouse, - sourceHiveDatabase, - catalogConfig, - tableConf, - parallelism); + connector, sourceHiveDatabase, catalogConfig, tableConf, parallelism); return Optional.of(migrateDatabaseAction); } 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..e874536e788f 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 @@ -36,14 +36,13 @@ public class MigrateFileAction extends ActionBase { public MigrateFileAction( String connector, - String warehouse, String sourceTable, String targetTable, boolean deleteOrigin, Map catalogConfig, String tableProperties, Integer parallelism) { - super(warehouse, catalogConfig); + super(catalogConfig); this.connector = connector; this.sourceTable = sourceTable; this.targetTable = targetTable; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileActionFactory.java index 34df99cfdf5b..c2d6aa2993ef 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MigrateFileActionFactory.java @@ -44,19 +44,17 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); String connector = params.get(SOURCE_TYPE); String sourceHiveTable = params.get(SOURCE_TABLE); String targetTable = params.get(TARGET_TABLE); boolean deleteOrigin = Boolean.parseBoolean(params.get(DELETE_ORIGIN)); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); String tableConf = params.get(OPTIONS); Integer parallelism = Integer.parseInt(params.get(PARALLELISM)); MigrateFileAction migrateFileAction = new MigrateFileAction( connector, - warehouse, sourceHiveTable, targetTable, deleteOrigin, 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..b12cb5f862fe 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 @@ -34,12 +34,11 @@ public class MigrateTableAction extends ActionBase { public MigrateTableAction( String connector, - String warehouse, String hiveTableFullName, Map catalogConfig, String tableProperties, Integer parallelism) { - super(warehouse, catalogConfig); + super(catalogConfig); this.connector = connector; this.hiveTableFullName = hiveTableFullName; this.tableProperties = tableProperties; 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..4ff060fb4665 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 @@ -37,21 +37,15 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); String connector = params.get(SOURCE_TYPE); String sourceHiveTable = params.get(TABLE); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); String tableConf = params.get(OPTIONS); Integer parallelism = Integer.parseInt(params.get(PARALLELISM)); MigrateTableAction migrateTableAction = new MigrateTableAction( - connector, - warehouse, - sourceHiveTable, - catalogConfig, - tableConf, - parallelism); + connector, sourceHiveTable, catalogConfig, tableConf, parallelism); return Optional.of(migrateTableAction); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MultipleParameterToolAdapter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MultipleParameterToolAdapter.java index e03b8cd698f2..7755cf7b40af 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MultipleParameterToolAdapter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/MultipleParameterToolAdapter.java @@ -48,4 +48,15 @@ public Collection getMultiParameter(String key) { public String fallback(String key) { return key.replaceAll("_", "-"); } + + public String getRequired(String key) { + String value = get(key); + if (value == null) { + throw new IllegalArgumentException( + "Argument '" + + key + + "' is required. Run ' --help' for more information."); + } + return value; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/QueryServiceActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/QueryServiceActionFactory.java index fecbc91e6270..060933676dbb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/QueryServiceActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/QueryServiceActionFactory.java @@ -20,8 +20,6 @@ import org.apache.paimon.flink.service.QueryService; -import org.apache.flink.api.java.tuple.Tuple3; - import java.util.Map; import java.util.Optional; @@ -39,13 +37,13 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); Map tableConfig = optionalConfigMap(params, TABLE_CONF); String parallStr = params.get(PARALLELISM); int parallelism = parallStr == null ? 1 : Integer.parseInt(parallStr); Action action = - new TableActionBase(tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig) { + new TableActionBase( + params.getRequired(DATABASE), params.getRequired(TABLE), catalogConfig) { @Override public void run() throws Exception { QueryService.build(env, table.copy(tableConfig), parallelism); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesAction.java index 5c7e6967c319..318089b30be5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesAction.java @@ -37,12 +37,11 @@ public class RemoveOrphanFilesAction extends ActionBase { private boolean dryRun = false; public RemoveOrphanFilesAction( - String warehouse, String databaseName, @Nullable String tableName, @Nullable String parallelism, Map catalogConfig) { - super(warehouse, catalogConfig); + super(catalogConfig); this.databaseName = databaseName; this.tableName = tableName; this.parallelism = parallelism; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionFactory.java index ed567510d143..279bc87baf65 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionFactory.java @@ -18,11 +18,8 @@ package org.apache.paimon.flink.action; -import java.util.Map; import java.util.Optional; -import static org.apache.paimon.utils.Preconditions.checkNotNull; - /** Factory to create {@link RemoveOrphanFilesAction}. */ public class RemoveOrphanFilesActionFactory implements ActionFactory { @@ -38,16 +35,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - checkNotNull(warehouse); - String database = params.get(DATABASE); - checkNotNull(database); - String table = params.get(TABLE); - String parallelism = params.get(PARALLELISM); - - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); RemoveOrphanFilesAction action = - new RemoveOrphanFilesAction(warehouse, database, table, parallelism, catalogConfig); + new RemoveOrphanFilesAction( + params.getRequired(DATABASE), + params.getRequired(TABLE), + params.get(PARALLELISM), + catalogConfigMap(params)); if (params.has(OLDER_THAN)) { action.olderThan(params.get(OLDER_THAN)); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java index 3b12b4c119a1..69ef7aa12a22 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java @@ -28,13 +28,12 @@ public class RenameTagAction extends TableActionBase { private final String targetTagName; public RenameTagAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String tagName, String targetTagName) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.tagName = tagName; this.targetTagName = targetTagName; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java index 84f174d39bbb..9f43146cfa05 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link RenameTagActionFactory}. */ @@ -38,22 +35,13 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, TAG_NAME); - checkRequiredArgument(params, TARGET_TAG_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String tagName = params.get(TAG_NAME); - String targetTagName = params.get(TARGET_TAG_NAME); - RenameTagAction action = new RenameTagAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - tagName, - targetTagName); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(TAG_NAME), + params.getRequired(TARGET_TAG_NAME)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairAction.java index 69fe5b409796..52a068777b36 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairAction.java @@ -29,8 +29,8 @@ public class RepairAction extends ActionBase { private final String identifier; - public RepairAction(String warehouse, String identifier, Map catalogConfig) { - super(warehouse, catalogConfig); + public RepairAction(String identifier, Map catalogConfig) { + super(catalogConfig); this.identifier = identifier; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairActionFactory.java index 2f201f675cc8..6431e00d2357 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RepairActionFactory.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.action; -import java.util.Map; import java.util.Optional; /** Factory to create {@link RepairAction}. */ @@ -35,10 +34,8 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); String identifier = params.get(IDENTIFIER_KEY); - RepairAction action = new RepairAction(warehouse, identifier, catalogConfig); + RepairAction action = new RepairAction(identifier, catalogConfigMap(params)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java index 09a85fe8a25a..a2b81bcc1e25 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java @@ -31,14 +31,13 @@ public class ReplaceTagAction extends TableActionBase { private final @Nullable Duration timeRetained; public ReplaceTagAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String tagName, @Nullable Long snapshotId, @Nullable Duration timeRetained) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.tagName = tagName; this.timeRetained = timeRetained; this.snapshotId = snapshotId; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java index a734e9cfbdc5..147e47af6dda 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java @@ -18,8 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - import java.time.Duration; import java.util.Map; @@ -35,19 +33,14 @@ public String identifier() { @Override Action createOrReplaceTagAction( - Tuple3 tablePath, + String database, + String table, Map catalogConfig, String tagName, Long snapshot, Duration timeRetained) { return new ReplaceTagAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - tagName, - snapshot, - timeRetained); + database, table, catalogConfig, tagName, snapshot, timeRetained); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java index 6db8ab4fef75..9c8f800650e6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java @@ -32,12 +32,11 @@ public class ResetConsumerAction extends TableActionBase { private Long nextSnapshotId; protected ResetConsumerAction( - String warehouse, String databaseName, String tableName, Map catalogConfig, String consumerId) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.consumerId = consumerId; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerActionFactory.java index acd85ae60167..dd93d2c7770c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link ResetConsumerAction}. */ @@ -38,15 +35,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, CONSUMER_ID); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String consumerId = params.get(CONSUMER_ID); - ResetConsumerAction action = new ResetConsumerAction( - tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig, consumerId); + params.getRequired(DATABASE), + params.getRequired(TABLE), + catalogConfigMap(params), + params.getRequired(CONSUMER_ID)); if (params.has(NEXT_SNAPSHOT)) { action.withNextSnapshotIds(Long.parseLong(params.get(NEXT_SNAPSHOT))); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexAction.java index f8ed73fd2025..dd2544093ead 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexAction.java @@ -29,9 +29,8 @@ public class RewriteFileIndexAction extends ActionBase { private String identifier; - public RewriteFileIndexAction( - String warehouse, String identifier, Map catalogConfig) { - super(warehouse, catalogConfig); + public RewriteFileIndexAction(String identifier, Map catalogConfig) { + super(catalogConfig); this.identifier = identifier; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexActionFactory.java index 910e7b8d9280..9cb178025471 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RewriteFileIndexActionFactory.java @@ -35,12 +35,10 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - String warehouse = params.get(WAREHOUSE); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + Map catalogConfig = catalogConfigMap(params); String identifier = params.get(IDENTIFIER_KEY); - RewriteFileIndexAction action = - new RewriteFileIndexAction(warehouse, identifier, catalogConfig); + RewriteFileIndexAction action = new RewriteFileIndexAction(identifier, catalogConfig); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToAction.java index 431383d38215..8afc75aa3fde 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToAction.java @@ -33,12 +33,11 @@ public class RollbackToAction extends TableActionBase { private final String version; public RollbackToAction( - String warehouse, String databaseName, String tableName, String version, Map catalogConfig) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.version = version; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToActionFactory.java index 077c608acab5..1a6a33bb1165 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link RollbackToAction}. */ @@ -37,16 +34,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - checkRequiredArgument(params, VERSION); - String version = params.get(VERSION); - - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - RollbackToAction action = new RollbackToAction( - tablePath.f0, tablePath.f1, tablePath.f2, version, catalogConfig); + params.getRequired(DATABASE), + params.getRequired(TABLE), + params.getRequired(VERSION), + catalogConfigMap(params)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java index ca706101cc1d..3a91b8a65a77 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java @@ -36,12 +36,11 @@ public class RollbackToTimestampAction extends TableActionBase { private final Long timestamp; public RollbackToTimestampAction( - String warehouse, String databaseName, String tableName, Long timestamp, Map catalogConfig) { - super(warehouse, databaseName, tableName, catalogConfig); + super(databaseName, tableName, catalogConfig); this.timestamp = timestamp; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java index c694ac0041b5..f4462a160451 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java @@ -18,9 +18,6 @@ package org.apache.paimon.flink.action; -import org.apache.flink.api.java.tuple.Tuple3; - -import java.util.Map; import java.util.Optional; /** Factory to create {@link RollbackToTimestampAction}. */ @@ -37,20 +34,12 @@ public String identifier() { @Override public Optional create(MultipleParameterToolAdapter params) { - Tuple3 tablePath = getTablePath(params); - - checkRequiredArgument(params, TIMESTAMP); - String timestamp = params.get(TIMESTAMP); - - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - RollbackToTimestampAction action = new RollbackToTimestampAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - Long.parseLong(timestamp), - catalogConfig); + params.getRequired(DATABASE), + params.getRequired(TABLE), + Long.parseLong(params.getRequired(TIMESTAMP)), + catalogConfigMap(params)); return Optional.of(action); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java index 2b12aa7a0430..eb545ea4377e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java @@ -51,12 +51,11 @@ public class SortCompactAction extends CompactAction { private List orderColumns; public SortCompactAction( - String warehouse, String database, String tableName, Map catalogConfig, Map tableConf) { - super(warehouse, database, tableName, catalogConfig, tableConf); + super(database, tableName, catalogConfig, tableConf); table = table.copy(Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java index c525b133b949..870d22012bf2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java @@ -41,12 +41,8 @@ public abstract class TableActionBase extends ActionBase { protected Table table; protected final Identifier identifier; - TableActionBase( - String warehouse, - String databaseName, - String tableName, - Map catalogConfig) { - super(warehouse, catalogConfig); + TableActionBase(String databaseName, String tableName, Map catalogConfig) { + super(catalogConfig); identifier = new Identifier(databaseName, tableName); try { table = catalog.getTable(identifier); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java index 8b3bc99567c8..45a89f1a5530 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java @@ -19,19 +19,26 @@ package org.apache.paimon.flink.procedure; import org.apache.paimon.flink.action.CloneAction; +import org.apache.paimon.utils.StringUtils; 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 java.util.HashMap; +import java.util.Map; + /** Clone Procedure. */ public class CloneProcedure extends ProcedureBase { public static final String IDENTIFIER = "clone"; @ProcedureHint( argument = { - @ArgumentHint(name = "warehouse", type = @DataTypeHint("STRING")), + @ArgumentHint( + name = "warehouse", + type = @DataTypeHint("STRING"), + isOptional = true), @ArgumentHint(name = "database", type = @DataTypeHint("STRING"), isOptional = true), @ArgumentHint(name = "table", type = @DataTypeHint("STRING"), isOptional = true), @ArgumentHint( @@ -65,16 +72,28 @@ public String[] call( String targetCatalogConfigStr, Integer parallelismStr) throws Exception { + Map sourceCatalogConfig = + new HashMap<>(optionalConfigMap(sourceCatalogConfigStr)); + if (!StringUtils.isNullOrWhitespaceOnly(warehouse) + && !sourceCatalogConfig.containsKey("warehouse")) { + sourceCatalogConfig.put("warehouse", warehouse); + } + + Map targetCatalogConfig = + new HashMap<>(optionalConfigMap(targetCatalogConfigStr)); + if (!StringUtils.isNullOrWhitespaceOnly(warehouse) + && !targetCatalogConfig.containsKey("warehouse")) { + targetCatalogConfig.put("warehouse", targetWarehouse); + } + CloneAction cloneAction = new CloneAction( - warehouse, database, tableName, - optionalConfigMap(sourceCatalogConfigStr), - targetWarehouse, + sourceCatalogConfig, targetDatabase, targetTableName, - optionalConfigMap(targetCatalogConfigStr), + targetCatalogConfig, parallelismStr == null ? null : Integer.toString(parallelismStr)); return execute(procedureContext, cloneAction, "Clone Job"); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java index 80602b755aa5..eaa8b4baf604 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java @@ -101,10 +101,9 @@ public String[] call( String compactStrategy) throws Exception { partitionIdleTime = notnull(partitionIdleTime); - String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); CompactDatabaseAction action = - new CompactDatabaseAction(warehouse, catalogOptions) + new CompactDatabaseAction(catalogOptions) .includingDatabases(nullable(includingDatabases)) .includingTables(nullable(includingTables)) .excludingTables(nullable(excludingTables)) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index 282f5af34043..b80693b39ec9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -77,7 +77,6 @@ public String[] call( String partitionIdleTime, String compactStrategy) throws Exception { - String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); Map tableConf = isNullOrWhitespaceOnly(tableOptions) @@ -89,7 +88,6 @@ public String[] call( if (isNullOrWhitespaceOnly(orderStrategy) && isNullOrWhitespaceOnly(orderByColumns)) { action = new CompactAction( - warehouse, identifier.getDatabaseName(), identifier.getObjectName(), catalogOptions, @@ -109,7 +107,6 @@ public String[] call( "sort compact do not support 'partition_idle_time'."); action = new SortCompactAction( - warehouse, identifier.getDatabaseName(), identifier.getObjectName(), catalogOptions, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java index e297c0bdbb4c..b5cf67a484d0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java @@ -144,15 +144,11 @@ public String[] call( notMatchedBySourceUpsertSetting = notnull(notMatchedBySourceUpsertSetting); notMatchedBySourceDeleteCondition = notnull(notMatchedBySourceDeleteCondition); - String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); Identifier identifier = Identifier.fromString(targetTableId); MergeIntoAction action = new MergeIntoAction( - warehouse, - identifier.getDatabaseName(), - identifier.getObjectName(), - catalogOptions); + identifier.getDatabaseName(), identifier.getObjectName(), catalogOptions); action.withTargetAlias(nullable(targetAlias)); if (!sourceSqls.isEmpty()) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java index 4ee539c4fd27..4a90415c191a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java @@ -172,7 +172,13 @@ public void testFullCompactionWithLongCheckpointInterval() throws Exception { .checkpointIntervalMs(2000) .build(); env.setParallelism(1); - new CompactAction(path, "default", "T").withStreamExecutionEnvironment(env).build(); + new CompactAction( + "default", + "T", + Collections.singletonMap("warehouse", path), + Collections.emptyMap()) + .withStreamExecutionEnvironment(env) + .build(); JobClient client = env.executeAsync(); // write records for a while @@ -834,7 +840,13 @@ private void testStandAloneFullCompactJobRandom( .parallelism(2) .allowRestart() .build(); - new CompactAction(path, "default", "T").withStreamExecutionEnvironment(env).build(); + new CompactAction( + "default", + "T", + Collections.singletonMap("warehouse", path), + Collections.emptyMap()) + .withStreamExecutionEnvironment(env) + .build(); env.executeAsync(); } @@ -873,7 +885,13 @@ private void testStandAloneLookupJobRandom( .allowRestart() .build(); env.setParallelism(2); - new CompactAction(path, "default", "T").withStreamExecutionEnvironment(env).build(); + new CompactAction( + "default", + "T", + Collections.singletonMap("warehouse", path), + Collections.emptyMap()) + .withStreamExecutionEnvironment(env) + .build(); env.executeAsync(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java index 2c4fb64f331c..280b5d71a6e6 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java @@ -409,17 +409,17 @@ private void runAction(boolean isStreaming, boolean unawareBucket) throws Except } ArrayList baseArgs = - Lists.newArrayList( - "compact", - "--warehouse", - warehouse, - "--database", - database, - "--table", - tableName); + Lists.newArrayList("compact", "--database", database, "--table", tableName); + ThreadLocalRandom random = ThreadLocalRandom.current(); + if (random.nextBoolean()) { + baseArgs.addAll(Lists.newArrayList("--warehouse", warehouse)); + } else { + baseArgs.addAll(Lists.newArrayList("--catalog_conf", "warehouse=" + warehouse)); + } + if (unawareBucket) { - if (true) { + if (random.nextBoolean()) { baseArgs.addAll(Lists.newArrayList("--where", "k=1")); } else { baseArgs.addAll(Lists.newArrayList("--partition", "k=1")); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionITTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionITTest.java index fef369b69040..a93d04e4674f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionITTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromTimestampActionITTest.java @@ -67,8 +67,10 @@ public void testCreateTagFromTimestampAction() throws Exception { "create_tag_from_timestamp", "--warehouse", warehouse, + "--database", + database, "--table", - database + "." + tableName, + tableName, "--tag", tag, "--timestamp", diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionITTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionITTest.java index b20974e7f44e..1198eb47f4b9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionITTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CreateTagFromWatermarkActionITTest.java @@ -70,8 +70,10 @@ public void testCreateTagsFromSnapshotsWatermark() throws Exception { "create_tag_from_watermark", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T", + "T", "--tag", "tag2", "--watermark", @@ -86,8 +88,10 @@ public void testCreateTagsFromSnapshotsWatermark() throws Exception { "create_tag_from_watermark", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T", + "T", "--tag", "tag3", "--watermark", @@ -144,8 +148,10 @@ public void testCreateTagsFromTagsWatermark() throws Exception { "create_tag_from_watermark", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T", + "T", "--tag", "tag2", "--watermark", @@ -161,8 +167,10 @@ public void testCreateTagsFromTagsWatermark() throws Exception { "create_tag_from_watermark", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T", + "T", "--tag", "tag3", "--watermark", diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java index 5a156ced25be..7e204ca88492 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java @@ -63,8 +63,10 @@ public void testExpireTags() throws Exception { "expire_tags", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T") + "T") .run(); // no tags expired assertThat(table.tagManager().tags().size()).isEqualTo(3); @@ -79,8 +81,10 @@ public void testExpireTags() throws Exception { "expire_tags", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T") + "T") .run(); // tag-4,tag-5 expires assertThat(table.tagManager().tags().size()).isEqualTo(3); @@ -96,8 +100,10 @@ public void testExpireTags() throws Exception { "expire_tags", "--warehouse", warehouse, + "--database", + database, "--table", - database + ".T", + "T", "--older_than", timestamp.toString()) .run(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java index dabb8bb2c990..f53686df07fb 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java @@ -285,10 +285,9 @@ public void testTableConf() throws Exception { createTable(); SortCompactAction sortCompactAction = new SortCompactAction( - warehouse, database, tableName, - Collections.emptyMap(), + Collections.singletonMap("warehouse", warehouse), Collections.singletonMap( FlinkConnectorOptions.SINK_PARALLELISM.key(), "20")) .withOrderStrategy("zorder") @@ -323,10 +322,9 @@ public void testSortCompactionOnEmptyData() throws Exception { createTable(); SortCompactAction sortCompactAction = new SortCompactAction( - warehouse, database, tableName, - Collections.emptyMap(), + Collections.singletonMap("warehouse", warehouse), Collections.emptyMap()) .withOrderStrategy("zorder") .withOrderColumns(Collections.singletonList("f0")); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireSnapshotsProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireSnapshotsProcedureITCase.java index c29e6edd435a..cb49cd0f1076 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireSnapshotsProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireSnapshotsProcedureITCase.java @@ -99,8 +99,10 @@ public void testExpireSnapshotsAction() throws Exception { "expire_snapshots", "--warehouse", path, - "--identifier", - "default.word_count", + "--database", + "default", + "--table", + "word_count", "--retain_max", "5") .withStreamExecutionEnvironment(env) @@ -114,8 +116,10 @@ public void testExpireSnapshotsAction() throws Exception { "expire_snapshots", "--warehouse", path, - "--identifier", - "default.word_count", + "--database", + "default", + "--table", + "word_count", "--older_than", ts6.toString(), "--max_deletes", @@ -129,8 +133,10 @@ public void testExpireSnapshotsAction() throws Exception { "expire_snapshots", "--warehouse", path, - "--identifier", - "default.word_count", + "--database", + "default", + "--table", + "word_count", "--older_than", ts6.toString(), "--retain_min", @@ -145,8 +151,10 @@ public void testExpireSnapshotsAction() throws Exception { "expire_snapshots", "--warehouse", path, - "--identifier", - "default.word_count", + "--database", + "default", + "--table", + "word_count", "--older_than", ts6.toString()) .withStreamExecutionEnvironment(env) diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateDatabaseProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateDatabaseProcedureITCase.java index 0acb7b8398ed..6bd57e0129bc 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateDatabaseProcedureITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateDatabaseProcedureITCase.java @@ -251,14 +251,10 @@ public void testMigrateDatabaseAction(String format) throws Exception { Map catalogConf = new HashMap<>(); catalogConf.put("metastore", "hive"); catalogConf.put("uri", "thrift://localhost:" + PORT); + catalogConf.put( + "warehouse", System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); MigrateDatabaseAction migrateDatabaseAction = - new MigrateDatabaseAction( - "hive", - System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname), - "my_database", - catalogConf, - "", - 6); + new MigrateDatabaseAction("hive", "my_database", catalogConf, "", 6); migrateDatabaseAction.run(); tEnv.executeSql( diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateFileProcedureITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateFileProcedureITCase.java index a50bdca3d58e..7ffc2ae54f04 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateFileProcedureITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/MigrateFileProcedureITCase.java @@ -161,10 +161,11 @@ public void testMigrateFileAction(String format, boolean isNamedArgument) throws Map catalogConf = new HashMap<>(); catalogConf.put("metastore", "hive"); catalogConf.put("uri", "thrift://localhost:" + PORT); + catalogConf.put( + "warehouse", System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); MigrateFileAction migrateFileAction = new MigrateFileAction( "hive", - System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname), "default.hivetable02", "default.paimontable02", false, 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..8d6ded69dc99 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 @@ -185,14 +185,10 @@ public void testMigrateAction(String format) throws Exception { Map catalogConf = new HashMap<>(); catalogConf.put("metastore", "hive"); catalogConf.put("uri", "thrift://localhost:" + PORT); + catalogConf.put( + "warehouse", System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); MigrateTableAction migrateTableAction = - new MigrateTableAction( - "hive", - System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname), - "default.hivetable", - catalogConf, - "", - 6); + new MigrateTableAction("hive", "default.hivetable", catalogConf, "", 6); migrateTableAction.run(); tEnv.executeSql( diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/RepairActionITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/RepairActionITCase.java index fdaa243dfa5d..5101425ded54 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/RepairActionITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/RepairActionITCase.java @@ -83,11 +83,9 @@ public void testRepairTableAction() throws Exception { Map catalogConf = new HashMap<>(); catalogConf.put("metastore", "hive"); catalogConf.put("uri", "thrift://localhost:" + PORT); - RepairAction repairAction = - new RepairAction( - System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname), - "test_db.t_repair_hive", - catalogConf); + catalogConf.put( + "warehouse", System.getProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + RepairAction repairAction = new RepairAction("test_db.t_repair_hive", catalogConf); repairAction.run(); List ret = From 2db0281696570a6909637854a682e56f19f2c512 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Wed, 18 Dec 2024 19:06:18 +0800 Subject: [PATCH 17/17] [core] Skip case checking in catalog (#4730) --- .../paimon/catalog/AbstractCatalog.java | 30 ---------- .../apache/paimon/catalog/CatalogUtils.java | 25 --------- .../paimon/catalog/FileSystemCatalogTest.java | 11 +--- .../apache/paimon/jdbc/JdbcCatalogTest.java | 16 ++---- .../action/cdc/SyncDatabaseActionBase.java | 8 --- .../flink/action/cdc/SyncTableActionBase.java | 7 --- .../action/cdc/SynchronizationActionBase.java | 4 -- .../apache/paimon/hive/HiveCatalogTest.java | 17 ++---- .../paimon/hive/HiveCatalogITCaseBase.java | 48 +++------------- .../org/apache/paimon/spark/SparkCatalog.java | 16 +----- .../paimon/spark/SparkGenericCatalog.java | 13 +---- .../SparkGenericCatalogWithHiveTest.java | 56 +------------------ .../sql/DDLWithHiveCatalogTestBase.scala | 47 ++++++++++++++++ 13 files changed, 73 insertions(+), 225 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index 51cb346d4bd7..d7447c37dd79 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -48,7 +48,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -282,8 +281,6 @@ public void createTable(Identifier identifier, Schema schema, boolean ignoreIfEx throws TableAlreadyExistException, DatabaseNotExistException { checkNotBranch(identifier, "createTable"); checkNotSystemTable(identifier, "createTable"); - validateIdentifierNameCaseInsensitive(identifier); - validateFieldNameCaseInsensitive(schema.rowType().getFieldNames()); validateAutoCreateClose(schema.options()); validateCustomTablePath(schema.options()); @@ -339,7 +336,6 @@ public void renameTable(Identifier fromTable, Identifier toTable, boolean ignore checkNotBranch(toTable, "renameTable"); checkNotSystemTable(fromTable, "renameTable"); checkNotSystemTable(toTable, "renameTable"); - validateIdentifierNameCaseInsensitive(toTable); try { getTable(fromTable); @@ -366,8 +362,6 @@ public void alterTable( Identifier identifier, List changes, boolean ignoreIfNotExists) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { checkNotSystemTable(identifier, "alterTable"); - validateIdentifierNameCaseInsensitive(identifier); - validateFieldNameCaseInsensitiveInSchemaChange(changes); try { getTable(identifier); @@ -571,30 +565,6 @@ protected void checkNotSystemDatabase(String database) { } } - protected void validateIdentifierNameCaseInsensitive(Identifier identifier) { - CatalogUtils.validateCaseInsensitive( - caseSensitive(), "Database", identifier.getDatabaseName()); - CatalogUtils.validateCaseInsensitive(caseSensitive(), "Table", identifier.getObjectName()); - } - - private void validateFieldNameCaseInsensitiveInSchemaChange(List changes) { - List fieldNames = new ArrayList<>(); - for (SchemaChange change : changes) { - if (change instanceof SchemaChange.AddColumn) { - SchemaChange.AddColumn addColumn = (SchemaChange.AddColumn) change; - fieldNames.addAll(Arrays.asList(addColumn.fieldNames())); - } else if (change instanceof SchemaChange.RenameColumn) { - SchemaChange.RenameColumn rename = (SchemaChange.RenameColumn) change; - fieldNames.add(rename.newName()); - } - } - validateFieldNameCaseInsensitive(fieldNames); - } - - protected void validateFieldNameCaseInsensitive(List fieldNames) { - CatalogUtils.validateCaseInsensitive(caseSensitive(), "Field", fieldNames); - } - private void validateAutoCreateClose(Map options) { checkArgument( !Boolean.parseBoolean( diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java index bae23c627607..043da0504d7f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java @@ -21,14 +21,10 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.schema.SchemaManager; -import java.util.Arrays; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static org.apache.paimon.catalog.Catalog.TABLE_DEFAULT_OPTION_PREFIX; import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; -import static org.apache.paimon.utils.Preconditions.checkArgument; /** Utils for {@link Catalog}. */ public class CatalogUtils { @@ -64,25 +60,4 @@ public static String table(String path) { public static Map tableDefaultOptions(Map options) { return convertToPropertiesPrefixKey(options, TABLE_DEFAULT_OPTION_PREFIX); } - - /** Validate database, table and field names must be lowercase when not case-sensitive. */ - public static void validateCaseInsensitive( - boolean caseSensitive, String type, String... names) { - validateCaseInsensitive(caseSensitive, type, Arrays.asList(names)); - } - - /** Validate database, table and field names must be lowercase when not case-sensitive. */ - public static void validateCaseInsensitive( - boolean caseSensitive, String type, List names) { - if (caseSensitive) { - return; - } - List illegalNames = - names.stream().filter(f -> !f.equals(f.toLowerCase())).collect(Collectors.toList()); - checkArgument( - illegalNames.isEmpty(), - String.format( - "%s name %s cannot contain upper case in the catalog.", - type, illegalNames)); - } } diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index 7045daca8e86..dcd27a91eda5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -19,7 +19,6 @@ package org.apache.paimon.catalog; import org.apache.paimon.fs.Path; -import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataTypes; @@ -29,7 +28,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link FileSystemCatalog}. */ @@ -39,14 +37,13 @@ public class FileSystemCatalogTest extends CatalogTestBase { public void setUp() throws Exception { super.setUp(); Options catalogOptions = new Options(); - catalogOptions.set(CatalogOptions.CASE_SENSITIVE, false); catalog = new FileSystemCatalog(fileIO, new Path(warehouse), catalogOptions); } @Test public void testCreateTableCaseSensitive() throws Exception { catalog.createDatabase("test_db", false); - Identifier identifier = Identifier.create("test_db", "new_table"); + Identifier identifier = Identifier.create("test_db", "new_TABLE"); Schema schema = Schema.newBuilder() .column("Pk1", DataTypes.INT()) @@ -64,11 +61,7 @@ public void testCreateTableCaseSensitive() throws Exception { .partitionKeys("Pk1", "pk2") .primaryKey("Pk1", "pk2", "pk3") .build(); - - // Create table throws Exception if using uppercase when 'allow-upper-case' is false - assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> catalog.createTable(identifier, schema, false)) - .withMessage("Field name [Pk1, Col1] cannot contain upper case in the catalog."); + catalog.createTable(identifier, schema, false); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index f01a46fd6bb4..51e2bf5c779d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -18,6 +18,7 @@ package org.apache.paimon.jdbc; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogTestBase; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.options.CatalogOptions; @@ -87,7 +88,7 @@ public void testCleanTimeoutLockAndAcquireLock() throws SQLException, Interrupte } @Test - public void testCheckIdentifierUpperCase() throws Exception { + public void testUpperCase() throws Exception { catalog.createDatabase("test_db", false); assertThatThrownBy( () -> @@ -95,17 +96,10 @@ public void testCheckIdentifierUpperCase() throws Exception { Identifier.create("TEST_DB", "new_table"), DEFAULT_TABLE_SCHEMA, false)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Database name [TEST_DB] cannot contain upper case in the catalog."); + .isInstanceOf(Catalog.DatabaseNotExistException.class) + .hasMessage("Database TEST_DB does not exist."); - assertThatThrownBy( - () -> - catalog.createTable( - Identifier.create("test_db", "NEW_TABLE"), - DEFAULT_TABLE_SCHEMA, - false)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Table name [NEW_TABLE] cannot contain upper case in the catalog."); + catalog.createTable(Identifier.create("test_db", "new_TABLE"), DEFAULT_TABLE_SCHEMA, false); } @Test diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java index fd9892c0f1d5..d6d85e59bba8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.flink.action.Action; import org.apache.paimon.flink.action.MultiTablesSinkMode; import org.apache.paimon.flink.sink.cdc.EventParser; @@ -166,13 +165,6 @@ public SyncDatabaseActionBase withPrimaryKeys(String... primaryKeys) { return this; } - @Override - protected void validateCaseSensitivity() { - CatalogUtils.validateCaseInsensitive(caseSensitive, "Database", database); - CatalogUtils.validateCaseInsensitive(caseSensitive, "Table prefix", tablePrefix); - CatalogUtils.validateCaseInsensitive(caseSensitive, "Table suffix", tableSuffix); - } - @Override protected FlatMapFunction recordParse() { return syncJobHandler.provideRecordParser( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index d997fcc29040..ae4f5346b24c 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -20,7 +20,6 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.action.Action; @@ -111,12 +110,6 @@ protected Schema buildPaimonSchema(Schema retrievedSchema) { true); } - @Override - protected void validateCaseSensitivity() { - CatalogUtils.validateCaseInsensitive(caseSensitive, "Database", database); - CatalogUtils.validateCaseInsensitive(caseSensitive, "Table", table); - } - @Override protected void beforeBuildingSourceSink() throws Exception { Identifier identifier = new Identifier(database, table); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index ebc051171de9..ce2766b9410b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -113,8 +113,6 @@ public void build() throws Exception { catalog.createDatabase(database, true); - validateCaseSensitivity(); - beforeBuildingSourceSink(); DataStream input = @@ -125,8 +123,6 @@ public void build() throws Exception { buildSink(input, parserFactory); } - protected abstract void validateCaseSensitivity(); - protected void beforeBuildingSourceSink() throws Exception {} protected Source buildSource() { diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index e3b48f02a696..bf6eb02f3e55 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -85,23 +85,18 @@ public void testListDatabasesWhenNoDatabases() { @Test public void testCheckIdentifierUpperCase() throws Exception { catalog.createDatabase("test_db", false); - assertThatThrownBy( - () -> - catalog.createTable( - Identifier.create("TEST_DB", "new_table"), - DEFAULT_TABLE_SCHEMA, - false)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Database name [TEST_DB] cannot contain upper case in the catalog."); - + assertThatThrownBy(() -> catalog.createDatabase("TEST_DB", false)) + .isInstanceOf(Catalog.DatabaseAlreadyExistException.class) + .hasMessage("Database TEST_DB already exists."); + catalog.createTable(Identifier.create("TEST_DB", "new_table"), DEFAULT_TABLE_SCHEMA, false); assertThatThrownBy( () -> catalog.createTable( Identifier.create("test_db", "NEW_TABLE"), DEFAULT_TABLE_SCHEMA, false)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Table name [NEW_TABLE] cannot contain upper case in the catalog."); + .isInstanceOf(Catalog.TableAlreadyExistException.class) + .hasMessage("Table test_db.NEW_TABLE already exists."); } private static final String HADOOP_CONF_DIR = diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java index 2266a8484d9d..c39b85cb3d22 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java @@ -577,8 +577,7 @@ public void testCreateInsensitiveTable() throws Exception { " 'uri' = '',", " 'warehouse' = '" + path + "',", " 'lock.enabled' = 'true',", - " 'table.type' = 'EXTERNAL',", - " 'allow-upper-case' = 'true'", + " 'table.type' = 'EXTERNAL'", ")")) .await(); tEnv.executeSql("USE CATALOG paimon_catalog_01").await(); @@ -593,30 +592,6 @@ public void testCreateInsensitiveTable() throws Exception { tEnv.executeSql("DROP TABLE t").await(); Path tablePath = new Path(path, "test_db.db/t"); assertThat(tablePath.getFileSystem().exists(tablePath)).isTrue(); - - tEnv.executeSql( - String.join( - "\n", - "CREATE CATALOG paimon_catalog_02 WITH (", - " 'type' = 'paimon',", - " 'metastore' = 'hive',", - " 'uri' = '',", - " 'warehouse' = '" + path + "',", - " 'lock.enabled' = 'true',", - " 'table.type' = 'EXTERNAL',", - " 'allow-upper-case' = 'false'", - ")")) - .await(); - tEnv.executeSql("USE CATALOG paimon_catalog_02").await(); - tEnv.executeSql("USE test_db").await(); - - // set case-sensitive = false would throw exception out - assertThatThrownBy( - () -> - tEnv.executeSql( - "CREATE TABLE t1 ( aa INT, Bb STRING ) WITH ( 'file.format' = 'avro' )") - .await()) - .isInstanceOf(RuntimeException.class); } @Test @@ -1006,7 +981,8 @@ public void testRenameTable() throws Exception { // the target table name has upper case. assertThatThrownBy(() -> tEnv.executeSql("ALTER TABLE t1 RENAME TO T1")) - .hasMessage("Table name [T1] cannot contain upper case in the catalog."); + .hasMessage( + "Could not execute ALTER TABLE my_hive.test_db.t1 RENAME TO my_hive.test_db.T1"); tEnv.executeSql("ALTER TABLE t1 RENAME TO t3").await(); @@ -1160,24 +1136,16 @@ public void testHiveLock() throws InterruptedException, Catalog.TableNotExistExc @Test public void testUpperCase() { + tEnv.executeSql("CREATE TABLE T (a INT, b STRING ) WITH ( 'file.format' = 'avro' )"); + tEnv.executeSql( + "CREATE TABLE tT (A INT, b STRING, C STRING) WITH ( 'file.format' = 'avro')"); assertThatThrownBy( () -> tEnv.executeSql( - "CREATE TABLE T ( a INT, b STRING ) WITH ( 'file.format' = 'avro' )") - .await()) - .hasRootCauseMessage( - String.format( - "Table name [%s] cannot contain upper case in the catalog.", "T")); - - assertThatThrownBy( - () -> - tEnv.executeSql( - "CREATE TABLE t (A INT, b STRING, C STRING) WITH ( 'file.format' = 'avro')") + "CREATE TABLE tt ( A INT, b STRING, C STRING) WITH ( 'file.format' = 'avro' )") .await()) .hasRootCauseMessage( - String.format( - "Field name %s cannot contain upper case in the catalog.", - "[A, C]")); + "Table (or view) test_db.tt already exists in Catalog my_hive."); } @Test diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index 12023cb84779..f32b87603f44 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -52,7 +52,6 @@ import org.apache.spark.sql.execution.datasources.v2.csv.CSVTable; import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable; import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable; -import org.apache.spark.sql.internal.SessionState; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -70,7 +69,6 @@ import static org.apache.paimon.CoreOptions.FILE_FORMAT; import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.TableType.FORMAT_TABLE; -import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE; import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf; @@ -91,18 +89,10 @@ public class SparkCatalog extends SparkBaseCatalog implements SupportFunction, S @Override public void initialize(String name, CaseInsensitiveStringMap options) { this.catalogName = name; - Map newOptions = new HashMap<>(options.asCaseSensitiveMap()); - SessionState sessionState = SparkSession.active().sessionState(); - CatalogContext catalogContext = - CatalogContext.create(Options.fromMap(options), sessionState.newHadoopConf()); - - // if spark is case-insensitive, set case-sensitive to catalog - if (!sessionState.conf().caseSensitiveAnalysis()) { - newOptions.put(CASE_SENSITIVE.key(), "true"); - } - options = new CaseInsensitiveStringMap(newOptions); - + CatalogContext.create( + Options.fromMap(options), + SparkSession.active().sessionState().newHadoopConf()); this.catalog = CatalogFactory.createCatalog(catalogContext); this.defaultDatabase = options.getOrDefault(DEFAULT_DATABASE.key(), DEFAULT_DATABASE.defaultValue()); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index b57228fa44f0..ac1543f2fe17 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -62,7 +62,6 @@ import java.util.Map; import java.util.concurrent.Callable; -import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.options.CatalogOptions.METASTORE; import static org.apache.paimon.options.CatalogOptions.WAREHOUSE; import static org.apache.paimon.spark.SparkCatalogOptions.CREATE_UNDERLYING_SESSION_CATALOG; @@ -242,7 +241,6 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { SparkSession sparkSession = SparkSession.active(); SessionState sessionState = sparkSession.sessionState(); Configuration hadoopConf = sessionState.newHadoopConf(); - SparkConf sparkConf = new SparkConf(); if (options.containsKey(METASTORE.key()) && options.get(METASTORE.key()).equalsIgnoreCase("hive")) { String uri = options.get(CatalogOptions.URI.key()); @@ -257,11 +255,6 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { } } } - if ("in-memory" - .equals(sparkSession.conf().get(StaticSQLConf.CATALOG_IMPLEMENTATION().key()))) { - LOG.warn("InMemoryCatalog here may cause bad effect."); - } - this.catalogName = name; this.sparkCatalog = new SparkCatalog(); @@ -273,6 +266,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { CREATE_UNDERLYING_SESSION_CATALOG.key(), CREATE_UNDERLYING_SESSION_CATALOG.defaultValue())) { this.underlyingSessionCatalogEnabled = true; + SparkConf sparkConf = new SparkConf(); for (Map.Entry entry : options.entrySet()) { sparkConf.set("spark.hadoop." + entry.getKey(), entry.getValue()); hadoopConf.set(entry.getKey(), entry.getValue()); @@ -330,11 +324,6 @@ private void fillCommonConfigurations(Map options, SQLConf sqlCo } else { options.put(DEFAULT_DATABASE.key(), sessionCatalogDefaultDatabase); } - - // if spark is case-insensitive, set case-sensitive to catalog - if (!sqlConf.caseSensitiveAnalysis()) { - options.put(CASE_SENSITIVE.key(), "true"); - } } @Override diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java index 84ea1ab5cba2..f3d4ba878905 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java @@ -31,7 +31,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.Assertions.assertThrows; /** Base tests for spark read. */ public class SparkGenericCatalogWithHiveTest { @@ -48,59 +47,6 @@ public static void closeMetastore() throws Exception { testHiveMetastore.stop(); } - @Test - public void testCreateTableCaseSensitive(@TempDir java.nio.file.Path tempDir) { - // firstly, we use hive metastore to creata table, and check the result. - Path warehousePath = new Path("file:" + tempDir.toString()); - SparkSession spark = - SparkSession.builder() - .config("spark.sql.warehouse.dir", warehousePath.toString()) - // with case-sensitive false - .config("spark.sql.caseSensitive", "false") - // with hive metastore - .config("spark.sql.catalogImplementation", "hive") - .config( - "spark.sql.catalog.spark_catalog", - SparkGenericCatalog.class.getName()) - .master("local[2]") - .getOrCreate(); - - spark.sql("CREATE DATABASE IF NOT EXISTS my_db1"); - spark.sql("USE my_db1"); - spark.sql( - "CREATE TABLE IF NOT EXISTS t2 (a INT, Bb INT, c STRING) USING paimon TBLPROPERTIES" - + " ('file.format'='avro')"); - - assertThat( - spark.sql("SHOW TABLES").collectAsList().stream() - .map(s -> s.get(1)) - .map(Object::toString)) - .containsExactlyInAnyOrder("t2"); - spark.close(); - - SparkSession spark1 = - SparkSession.builder() - .config("spark.sql.warehouse.dir", warehousePath.toString()) - // with case-sensitive true - .config("spark.sql.caseSensitive", "true") - // with hive metastore - .config("spark.sql.catalogImplementation", "hive") - .config( - "spark.sql.catalog.spark_catalog", - SparkGenericCatalog.class.getName()) - .master("local[2]") - .getOrCreate(); - - spark1.sql("USE my_db1"); - assertThrows( - RuntimeException.class, - () -> - spark1.sql( - "CREATE TABLE IF NOT EXISTS t3 (a INT, Bb INT, c STRING) USING paimon TBLPROPERTIES" - + " ('file.format'='avro')")); - spark1.close(); - } - @Test public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { // firstly, we use hive metastore to create table, and check the result. @@ -123,7 +69,7 @@ public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { + " ('file.format'='avro')"); assertThat(spark.sql("SHOW NAMESPACES").collectAsList().stream().map(Object::toString)) - .containsExactlyInAnyOrder("[default]", "[my_db]", "[my_db1]"); + .containsExactlyInAnyOrder("[default]", "[my_db]"); assertThat( spark.sql("SHOW TABLES").collectAsList().stream() diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index d51cdce34cb9..5311c586a59c 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -553,6 +553,53 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: case sensitive") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + Seq(false, true).foreach { + caseSensitive => + withSparkSQLConf("spark.sql.caseSensitive" -> caseSensitive.toString) { + spark.sql(s"USE $catalogName") + withDatabase("paimon_case_sensitive_DB") { + spark.sql(s"CREATE DATABASE paimon_case_sensitive_DB") + + // check create db + // note: db name is always lower case in hive + intercept[Exception](spark.sql("CREATE DATABASE paimon_case_sensitive_db")) + + spark.sql(s"USE paimon_case_sensitive_DB") + withTable("tT", "tt") { + spark.sql("CREATE TABLE tT (aA INT) USING paimon") + spark.sql("INSERT INTO tT VALUES 1") + + // check select + checkAnswer(spark.sql("SELECT aA FROM tT"), Row(1)) + if (caseSensitive) { + intercept[Exception](spark.sql(s"SELECT aa FROM tT")) + } else { + checkAnswer(spark.sql("SELECT aa FROM tT"), Row(1)) + } + + // check alter table rename + // note: table name is always lower case in hive + intercept[Exception](spark.sql(s"ALTER TABLE tT RENAME TO tt")) + + // check alter table rename column + // note: col name can be upper case in hive + if (caseSensitive) { + spark.sql("ALTER TABLE tT RENAME COLUMN aA TO aa") + checkAnswer(spark.sql("SELECT aa FROM tT"), Row(1)) + intercept[Exception](spark.sql(s"SELECT aA FROM tT")) + } else { + intercept[Exception](spark.sql("ALTER TABLE tT RENAME COLUMN aA TO aa")) + } + } + } + } + } + } + } + def getDatabaseProp(dbName: String, propertyName: String): String = { spark .sql(s"DESC DATABASE EXTENDED $dbName")