This is an automated email from the ASF dual-hosted git repository.

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new 5ec8f4efe9 [core] Introduce paimon-api module for light rest api, part 
1 (#5605)
5ec8f4efe9 is described below

commit 5ec8f4efe9355080d87196a259cc195b8e32ee4d
Author: Jingsong Lee <[email protected]>
AuthorDate: Thu May 15 13:37:31 2025 +0800

    [core] Introduce paimon-api module for light rest api, part 1 (#5605)
---
 docs/content/program-api/catalog-api.md            | 88 +++++++++++-----------
 docs/content/program-api/flink-api.md              |  2 +-
 docs/content/program-api/java-api.md               | 20 ++---
 paimon-api/README.md                               |  7 ++
 paimon-api/pom.xml                                 | 74 ++++++++++++++++++
 .../main/java/org/apache/paimon/CoreOptions.java   | 58 +++-----------
 .../src/main/java/org/apache/paimon/TableType.java |  0
 .../org/apache/paimon/annotation/ConfigGroup.java  |  0
 .../org/apache/paimon/annotation/ConfigGroups.java |  0
 .../apache/paimon/annotation/Documentation.java    |  4 +-
 .../org/apache/paimon/annotation/Experimental.java |  0
 .../java/org/apache/paimon/annotation/Public.java  |  0
 .../paimon/annotation/VisibleForTesting.java       |  0
 .../java/org/apache/paimon/catalog/Identifier.java | 25 +++---
 .../apache/paimon/compression/CompressOptions.java |  0
 .../apache/paimon/fileindex/FileIndexOptions.java  |  0
 .../src/main/java/org/apache/paimon/fs/Path.java   |  6 +-
 .../java/org/apache/paimon/function/Function.java  |  0
 .../org/apache/paimon/function/FunctionChange.java |  0
 .../apache/paimon/function/FunctionDefinition.java |  0
 .../org/apache/paimon/function/FunctionImpl.java   | 12 ---
 .../org/apache/paimon/lookup/LookupStrategy.java   |  0
 .../org/apache/paimon/options/CatalogOptions.java  |  0
 .../org/apache/paimon/options/ConfigOption.java    |  0
 .../org/apache/paimon/options/ConfigOptions.java   |  0
 .../org/apache/paimon/options/ExpireConfig.java    |  0
 .../org/apache/paimon/options/FallbackKey.java     |  0
 .../java/org/apache/paimon/options/MemorySize.java |  0
 .../java/org/apache/paimon/options/Options.java    |  0
 .../org/apache/paimon/options/OptionsUtils.java    |  0
 .../paimon/options/StructuredOptionsSplitter.java  |  0
 .../paimon/options/description/BlockElement.java   |  0
 .../paimon/options/description/DescribedEnum.java  |  0
 .../paimon/options/description/Description.java    |  0
 .../options/description/DescriptionElement.java    |  0
 .../paimon/options/description/Formatter.java      |  0
 .../paimon/options/description/HtmlFormatter.java  |  0
 .../paimon/options/description/InlineElement.java  |  0
 .../options/description/LineBreakElement.java      |  0
 .../paimon/options/description/LinkElement.java    |  0
 .../paimon/options/description/ListElement.java    |  0
 .../paimon/options/description/TextElement.java    |  0
 .../java/org/apache/paimon/rest/RESTMessage.java   |  0
 .../java/org/apache/paimon/rest/RESTRequest.java   |  0
 .../paimon/rest/requests/AlterTableRequest.java    |  0
 .../rest/requests/CreateDatabaseRequest.java       |  0
 .../paimon/rest/requests/CreateTableRequest.java   |  0
 .../main/java/org/apache/paimon/schema/Schema.java |  0
 .../org/apache/paimon/schema/SchemaChange.java     |  0
 .../org/apache/paimon/table/CatalogTableType.java  |  0
 .../org/apache/paimon/table/SpecialFields.java     |  0
 .../java/org/apache/paimon/types/ArrayType.java    |  0
 .../java/org/apache/paimon/types/BigIntType.java   |  0
 .../java/org/apache/paimon/types/BinaryType.java   |  0
 .../java/org/apache/paimon/types/BooleanType.java  |  0
 .../java/org/apache/paimon/types/CharType.java     |  0
 .../java/org/apache/paimon/types/DataField.java    |  0
 .../java/org/apache/paimon/types/DataType.java     |  4 -
 .../org/apache/paimon/types/DataTypeCasts.java     |  0
 .../org/apache/paimon/types/DataTypeChecks.java    |  0
 .../paimon/types/DataTypeDefaultVisitor.java       |  0
 .../org/apache/paimon/types/DataTypeFamily.java    |  0
 .../apache/paimon/types/DataTypeJsonParser.java    |  0
 .../java/org/apache/paimon/types/DataTypeRoot.java |  0
 .../org/apache/paimon/types/DataTypeVisitor.java   |  0
 .../java/org/apache/paimon/types/DataTypes.java    |  0
 .../java/org/apache/paimon/types/DateType.java     |  0
 .../java/org/apache/paimon/types/DecimalType.java  | 10 ++-
 .../java/org/apache/paimon/types/DoubleType.java   |  0
 .../java/org/apache/paimon/types/FloatType.java    |  0
 .../main/java/org/apache/paimon/types/IntType.java |  0
 .../paimon/types/LocalZonedTimestampType.java      |  0
 .../main/java/org/apache/paimon/types/MapType.java |  0
 .../java/org/apache/paimon/types/MultisetType.java |  0
 .../org/apache/paimon/types/ReassignFieldId.java   |  0
 .../main/java/org/apache/paimon/types/RowKind.java |  0
 .../main/java/org/apache/paimon/types/RowType.java | 14 ----
 .../java/org/apache/paimon/types/SmallIntType.java |  0
 .../java/org/apache/paimon/types/TimeType.java     |  0
 .../org/apache/paimon/types/TimestampType.java     |  0
 .../java/org/apache/paimon/types/TinyIntType.java  |  0
 .../org/apache/paimon/types/VarBinaryType.java     |  0
 .../java/org/apache/paimon/types/VarCharType.java  |  0
 .../java/org/apache/paimon/types/VariantType.java  |  3 +-
 .../java/org/apache/paimon/utils/ArrayUtils.java   |  0
 .../org/apache/paimon/utils/EncodingUtils.java     |  0
 .../java/org/apache/paimon/utils/MathUtils.java    |  0
 .../main/java/org/apache/paimon/utils/Pair.java    |  0
 .../org/apache/paimon/utils/Preconditions.java     |  0
 .../java/org/apache/paimon/utils/StringUtils.java  | 42 +----------
 .../java/org/apache/paimon/utils/TimeUtils.java    |  0
 .../src/main/java/org/apache/paimon/view/View.java |  0
 .../java/org/apache/paimon/view/ViewChange.java    |  0
 .../main/java/org/apache/paimon/view/ViewImpl.java |  0
 .../java/org/apache/paimon/view/ViewSchema.java    |  0
 paimon-bundle/pom.xml                              | 14 ++--
 paimon-common/pom.xml                              |  6 ++
 .../main/java/org/apache/paimon/data/Decimal.java  |  6 +-
 .../java/org/apache/paimon/format/FileFormat.java  | 12 ++-
 .../org/apache/paimon/utils/BinaryStringUtils.java | 35 ++++++++-
 .../paimon/utils/InternalRowPartitionComputer.java |  6 +-
 .../apache/paimon/format/FormatReadWriteTest.java  |  6 +-
 .../java/org/apache/paimon/AbstractFileStore.java  |  7 +-
 .../org/apache/paimon/catalog/AbstractCatalog.java |  2 +-
 .../java/org/apache/paimon/catalog/Catalog.java    |  4 +-
 .../apache/paimon/consumer/ConsumerManager.java    |  2 +-
 .../compact/aggregate/FieldListaggAgg.java         |  5 +-
 .../paimon/operation/BaseAppendFileStoreWrite.java |  4 +-
 .../paimon/operation/KeyValueFileStoreWrite.java   |  3 +-
 .../apache/paimon/operation/OrphanFilesClean.java  |  4 +-
 .../postpone/PostponeBucketFileStoreWrite.java     |  5 +-
 .../java/org/apache/paimon/rest/RESTCatalog.java   |  3 +-
 .../paimon/rest/responses/GetFunctionResponse.java | 14 ++++
 .../org/apache/paimon/schema/SchemaManager.java    |  2 +-
 .../paimon/table/source/AbstractDataTableScan.java | 35 ++++++++-
 .../table/source/snapshot/TimeTravelUtil.java      | 60 ++++++++++-----
 .../table/system/AggregationFieldsTable.java       |  2 +-
 .../apache/paimon/table/system/AuditLogTable.java  |  2 +-
 .../apache/paimon/table/system/BinlogTable.java    | 11 ++-
 .../apache/paimon/table/system/BranchesTable.java  |  2 +-
 .../apache/paimon/table/system/BucketsTable.java   |  2 +-
 .../apache/paimon/table/system/ConsumersTable.java |  2 +-
 .../org/apache/paimon/table/system/FilesTable.java |  2 +-
 .../apache/paimon/table/system/ManifestsTable.java |  5 +-
 .../apache/paimon/table/system/OptionsTable.java   |  2 +-
 .../paimon/table/system/PartitionsTable.java       |  2 +-
 .../paimon/table/system/ReadOptimizedTable.java    |  2 +-
 .../apache/paimon/table/system/SchemasTable.java   |  2 +-
 .../apache/paimon/table/system/SnapshotsTable.java |  2 +-
 .../apache/paimon/table/system/StatisticTable.java |  2 +-
 .../paimon/table/system/TableIndexesTable.java     |  2 +-
 .../org/apache/paimon/table/system/TagsTable.java  |  2 +-
 .../org/apache/paimon/utils/BranchManager.java     |  2 +-
 .../java/org/apache/paimon/utils/TagManager.java   |  2 +-
 .../java/org/apache/paimon/FileFormatTest.java     |  4 +-
 .../test/java/org/apache/paimon/SnapshotTest.java  |  2 +-
 .../manifest/IndexManifestFileHandlerTest.java     |  5 +-
 .../apache/paimon/table/SimpleTableTestBase.java   |  3 +-
 .../table/system/AggregationFieldsTableTest.java   |  4 +-
 .../paimon/table/system/AuditLogTableTest.java     |  4 +-
 .../paimon/table/system/BucketsTableTest.java      |  2 +-
 .../apache/paimon/table/system/FilesTableTest.java |  5 +-
 .../paimon/table/system/OptionsTableTest.java      |  4 +-
 .../paimon/table/system/PartitionsTableTest.java   |  4 +-
 .../paimon/table/system/SnapshotsTableTest.java    |  4 +-
 .../apache/paimon/utils/SnapshotManagerTest.java   |  2 +-
 .../configuration/ConfigOptionsDocGenerator.java   |  5 +-
 .../flink/lookup/FileStoreLookupFunction.java      |  8 +-
 .../org/apache/paimon/flink/FileStoreITCase.java   |  4 +-
 .../parquet/reader/VectorizedColumnReader.java     |  1 -
 .../reader/FileTypeNotMatchReadTypeTest.java       |  1 -
 .../java/org/apache/paimon/hive/HiveCatalog.java   |  2 +-
 pom.xml                                            |  1 +
 153 files changed, 415 insertions(+), 302 deletions(-)

diff --git a/docs/content/program-api/catalog-api.md 
b/docs/content/program-api/catalog-api.md
index 5142a2f752..c7fcf46aba 100644
--- a/docs/content/program-api/catalog-api.md
+++ b/docs/content/program-api/catalog-api.md
@@ -250,30 +250,30 @@ public class AlterTable {
 
         try {
             catalog.createTable(
-                identifier,
-                new Schema(
-                    Lists.newArrayList(
-                        new DataField(0, "col1", DataTypes.STRING(), "field1"),
-                        new DataField(1, "col2", DataTypes.STRING(), "field2"),
-                        new DataField(2, "col3", DataTypes.STRING(), "field3"),
-                        new DataField(3, "col4", DataTypes.BIGINT(), "field4"),
-                        new DataField(
-                            4,
-                            "col5",
-                            DataTypes.ROW(
-                                new DataField(
-                                    5, "f1", DataTypes.STRING(), "f1"),
-                                new DataField(
-                                    6, "f2", DataTypes.STRING(), "f2"),
-                                new DataField(
-                                    7, "f3", DataTypes.STRING(), "f3")),
-                            "field5"),
-                        new DataField(8, "col6", DataTypes.STRING(), 
"field6")),
-                    Lists.newArrayList("col1"), // partition keys
-                    Lists.newArrayList("col1", "col2"), // primary key
-                    options,
-                    "table comment"),
-                false);
+                    identifier,
+                    new Schema(
+                            Lists.newArrayList(
+                                    new DataField(0, "col1", 
DataTypes.STRING(), "field1"),
+                                    new DataField(1, "col2", 
DataTypes.STRING(), "field2"),
+                                    new DataField(2, "col3", 
DataTypes.STRING(), "field3"),
+                                    new DataField(3, "col4", 
DataTypes.BIGINT(), "field4"),
+                                    new DataField(
+                                            4,
+                                            "col5",
+                                            DataTypes.ROW(
+                                                    new DataField(
+                                                            5, "f1", 
DataTypes.STRING(), "f1"),
+                                                    new DataField(
+                                                            6, "f2", 
DataTypes.STRING(), "f2"),
+                                                    new DataField(
+                                                            7, "f3", 
DataTypes.STRING(), "f3")),
+                                            "field5"),
+                                    new DataField(8, "col6", 
DataTypes.STRING(), "field6")),
+                            Lists.newArrayList("col1"), // partition keys
+                            Lists.newArrayList("col1", "col2"), // primary key
+                            options,
+                            "table comment"),
+                    false);
         } catch (Catalog.TableAlreadyExistException e) {
             // do something
         } catch (Catalog.DatabaseNotExistException e) {
@@ -287,44 +287,44 @@ public class AlterTable {
         // add a column after col1
         SchemaChange.Move after = SchemaChange.Move.after("col1_after", 
"col1");
         SchemaChange addColumnAfterField =
-            SchemaChange.addColumn("col7", DataTypes.STRING(), "", after);
+                SchemaChange.addColumn("col7", DataTypes.STRING(), "", after);
         // rename column
         SchemaChange renameColumn = SchemaChange.renameColumn("col3", 
"col3_new_name");
         // drop column
         SchemaChange dropColumn = SchemaChange.dropColumn("col6");
         // update column comment
         SchemaChange updateColumnComment =
-            SchemaChange.updateColumnComment(new String[] {"col4"}, "col4 
field");
+                SchemaChange.updateColumnComment(new String[]{"col4"}, "col4 
field");
         // update nested column comment
         SchemaChange updateNestedColumnComment =
-            SchemaChange.updateColumnComment(new String[] {"col5", "f1"}, 
"col5 f1 field");
+                SchemaChange.updateColumnComment(new String[]{"col5", "f1"}, 
"col5 f1 field");
         // update column type
         SchemaChange updateColumnType = SchemaChange.updateColumnType("col4", 
DataTypes.DOUBLE());
         // update column position, you need to pass in a parameter of type Move
         SchemaChange updateColumnPosition =
-            SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4"));
+                
SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4"));
         // update column nullability
         SchemaChange updateColumnNullability =
-            SchemaChange.updateColumnNullability(new String[] {"col4"}, false);
+                SchemaChange.updateColumnNullability(new String[]{"col4"}, 
false);
         // update nested column nullability
         SchemaChange updateNestedColumnNullability =
-            SchemaChange.updateColumnNullability(new String[] {"col5", "f2"}, 
false);
+                SchemaChange.updateColumnNullability(new String[]{"col5", 
"f2"}, false);
 
         SchemaChange[] schemaChanges =
-            new SchemaChange[] {
-                addOption,
-                removeOption,
-                addColumn,
-                addColumnAfterField,
-                renameColumn,
-                dropColumn,
-                updateColumnComment,
-                updateNestedColumnComment,
-                updateColumnType,
-                updateColumnPosition,
-                updateColumnNullability,
-                updateNestedColumnNullability
-            };
+                new SchemaChange[]{
+                        addOption,
+                        removeOption,
+                        addColumn,
+                        addColumnAfterField,
+                        renameColumn,
+                        dropColumn,
+                        updateColumnComment,
+                        updateNestedColumnComment,
+                        updateColumnType,
+                        updateColumnPosition,
+                        updateColumnNullability,
+                        updateNestedColumnNullability
+                };
         try {
             catalog.alterTable(identifier, Arrays.asList(schemaChanges), 
false);
         } catch (Catalog.TableNotExistException e) {
diff --git a/docs/content/program-api/flink-api.md 
b/docs/content/program-api/flink-api.md
index 14e43aae8b..cf3170fedb 100644
--- a/docs/content/program-api/flink-api.md
+++ b/docs/content/program-api/flink-api.md
@@ -217,7 +217,7 @@ public class WriteCdcToTable {
         Identifier identifier = Identifier.create("my_db", "T");
         Options catalogOptions = new Options();
         catalogOptions.set("warehouse", "/path/to/warehouse");
-        CatalogLoader catalogLoader = 
+        CatalogLoader catalogLoader =
                 () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions);
         Table table = catalogLoader.load().getTable(identifier);
 
diff --git a/docs/content/program-api/java-api.md 
b/docs/content/program-api/java-api.md
index 1bf0ce82f6..09e9b7b9cd 100644
--- a/docs/content/program-api/java-api.md
+++ b/docs/content/program-api/java-api.md
@@ -175,16 +175,16 @@ public class ReadTable {
         Table table = GetTable.getTable();
 
         PredicateBuilder builder =
-            new PredicateBuilder(RowType.of(DataTypes.STRING(), 
DataTypes.INT()));
+                new PredicateBuilder(RowType.of(DataTypes.STRING(), 
DataTypes.INT()));
         Predicate notNull = builder.isNotNull(0);
         Predicate greaterOrEqual = builder.greaterOrEqual(1, 12);
 
-        int[] projection = new int[] {0, 1};
+        int[] projection = new int[]{0, 1};
 
         ReadBuilder readBuilder =
-            table.newReadBuilder()
-                .withProjection(projection)
-                .withFilter(Lists.newArrayList(notNull, greaterOrEqual));
+                table.newReadBuilder()
+                        .withProjection(projection)
+                        .withFilter(Lists.newArrayList(notNull, 
greaterOrEqual));
 
         // 2. Plan splits in 'Coordinator' (or named 'Driver')
         List<Split> splits = readBuilder.newScan().plan().splits();
@@ -283,16 +283,16 @@ public class StreamReadTable {
         Table table = GetTable.getTable();
 
         PredicateBuilder builder =
-            new PredicateBuilder(RowType.of(DataTypes.STRING(), 
DataTypes.INT()));
+                new PredicateBuilder(RowType.of(DataTypes.STRING(), 
DataTypes.INT()));
         Predicate notNull = builder.isNotNull(0);
         Predicate greaterOrEqual = builder.greaterOrEqual(1, 12);
 
-        int[] projection = new int[] {0, 1};
+        int[] projection = new int[]{0, 1};
 
         ReadBuilder readBuilder =
-            table.newReadBuilder()
-                .withProjection(projection)
-                .withFilter(Lists.newArrayList(notNull, greaterOrEqual));
+                table.newReadBuilder()
+                        .withProjection(projection)
+                        .withFilter(Lists.newArrayList(notNull, 
greaterOrEqual));
 
         // 2. Plan splits in 'Coordinator' (or named 'Driver')
         StreamTableScan scan = readBuilder.newStreamScan();
diff --git a/paimon-api/README.md b/paimon-api/README.md
new file mode 100644
index 0000000000..e347cbb122
--- /dev/null
+++ b/paimon-api/README.md
@@ -0,0 +1,7 @@
+## Paimon API
+
+This module is for light API, aims to simplify dependencies as much as 
possible and avoid
+introducing dependencies such as Hadoop, including:
+
+1. Including types, table, view, function and etc.
+2. Including http client REST API.
diff --git a/paimon-api/pom.xml b/paimon-api/pom.xml
new file mode 100644
index 0000000000..367e13f619
--- /dev/null
+++ b/paimon-api/pom.xml
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.paimon</groupId>
+        <artifactId>paimon-parent</artifactId>
+        <version>1.2-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>paimon-api</artifactId>
+    <name>Paimon : API</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-shade-jackson-2</artifactId>
+            
<version>${paimon.shade.jackson.version}-${paimon.shade.version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>shade-paimon</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <artifactSet>
+                                <includes combine.children="append">
+                                    
<include>org.apache.paimon:paimon-shade-jackson-2</include>
+                                </includes>
+                            </artifactSet>
+                            <filters>
+                                <!-- Another copy of the Apache license, which 
we don't need. -->
+                                <filter>
+                                    <artifact>*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/LICENSE.txt</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java 
b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
similarity index 98%
rename from paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
rename to paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
index eefe1a8a24..ae8e097bd3 100644
--- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
+++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
@@ -24,7 +24,6 @@ import org.apache.paimon.annotation.Documentation.Immutable;
 import org.apache.paimon.annotation.VisibleForTesting;
 import org.apache.paimon.compression.CompressOptions;
 import org.apache.paimon.fileindex.FileIndexOptions;
-import org.apache.paimon.format.FileFormat;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.lookup.LookupStrategy;
 import org.apache.paimon.options.ConfigOption;
@@ -35,7 +34,6 @@ import org.apache.paimon.options.Options;
 import org.apache.paimon.options.description.DescribedEnum;
 import org.apache.paimon.options.description.Description;
 import org.apache.paimon.options.description.InlineElement;
-import org.apache.paimon.utils.DateTimeUtils;
 import org.apache.paimon.utils.MathUtils;
 import org.apache.paimon.utils.Pair;
 import org.apache.paimon.utils.StringUtils;
@@ -55,7 +53,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
-import java.util.TimeZone;
 import java.util.UUID;
 import java.util.stream.Collectors;
 
@@ -1780,16 +1777,12 @@ public class CoreOptions implements Serializable {
         return normalizeFileFormat(options.get(FILE_FORMAT));
     }
 
-    public FileFormat fileFormat() {
-        return createFileFormat(options, FILE_FORMAT);
-    }
-
     public String fileFormatString() {
         return normalizeFileFormat(options.get(FILE_FORMAT));
     }
 
-    public FileFormat manifestFormat() {
-        return createFileFormat(options, MANIFEST_FORMAT);
+    public String manifestFormatString() {
+        return normalizeFileFormat(options.get(MANIFEST_FORMAT));
     }
 
     public String manifestCompression() {
@@ -1824,11 +1817,6 @@ public class CoreOptions implements Serializable {
         return options.get(SORT_COMPACTION_SAMPLE_MAGNIFICATION);
     }
 
-    public static FileFormat createFileFormat(Options options, 
ConfigOption<String> formatOption) {
-        String formatIdentifier = 
normalizeFileFormat(options.get(formatOption));
-        return FileFormat.fromIdentifier(formatIdentifier, options);
-    }
-
     public String objectLocation() {
         checkArgument(type() == TableType.OBJECT_TABLE, "Only object table has 
object location!");
         return options.get(OBJECT_LOCATION);
@@ -1859,7 +1847,7 @@ public class CoreOptions implements Serializable {
                 .collect(Collectors.toMap(e -> Integer.valueOf(e.getKey()), 
Map.Entry::getValue));
     }
 
-    private static String normalizeFileFormat(String fileFormat) {
+    public static String normalizeFileFormat(String fileFormat) {
         return fileFormat.toLowerCase();
     }
 
@@ -2296,13 +2284,7 @@ public class CoreOptions implements Serializable {
     }
 
     public Long scanTimestampMills() {
-        String timestampStr = scanTimestamp();
-        Long timestampMillis = options.get(SCAN_TIMESTAMP_MILLIS);
-        if (timestampMillis == null && timestampStr != null) {
-            return DateTimeUtils.parseTimestampData(timestampStr, 3, 
TimeZone.getDefault())
-                    .getMillisecond();
-        }
-        return timestampMillis;
+        return options.get(SCAN_TIMESTAMP_MILLIS);
     }
 
     public String scanTimestamp() {
@@ -2345,32 +2327,8 @@ public class CoreOptions implements Serializable {
         return Pair.of(split[0], split[1]);
     }
 
-    public Pair<Long, Long> incrementalBetweenTimestamp() {
-        String str = options.get(INCREMENTAL_BETWEEN_TIMESTAMP);
-        String[] split = str.split(",");
-        if (split.length != 2) {
-            throw new IllegalArgumentException(
-                    "The incremental-between-timestamp must specific 
start(exclusive) and end timestamp. But is: "
-                            + str);
-        }
-
-        try {
-            return Pair.of(Long.parseLong(split[0]), Long.parseLong(split[1]));
-        } catch (NumberFormatException nfe) {
-            try {
-                long startTimestamp =
-                        DateTimeUtils.parseTimestampData(split[0], 3, 
TimeZone.getDefault())
-                                .getMillisecond();
-                long endTimestamp =
-                        DateTimeUtils.parseTimestampData(split[1], 3, 
TimeZone.getDefault())
-                                .getMillisecond();
-                return Pair.of(startTimestamp, endTimestamp);
-            } catch (Exception e) {
-                throw new IllegalArgumentException(
-                        "The incremental-between-timestamp must specific 
start(exclusive) and end timestamp. But is: "
-                                + str);
-            }
-        }
+    public String incrementalBetweenTimestamp() {
+        return options.get(INCREMENTAL_BETWEEN_TIMESTAMP);
     }
 
     public IncrementalBetweenScanMode incrementalBetweenScanMode() {
@@ -2638,6 +2596,10 @@ public class CoreOptions implements Serializable {
         return options.get(DELETION_VECTORS_ENABLED);
     }
 
+    public boolean forceLookup() {
+        return options.get(FORCE_LOOKUP);
+    }
+
     public boolean batchScanSkipLevel0() {
         return deletionVectorsEnabled() || mergeEngine() == FIRST_ROW;
     }
diff --git a/paimon-common/src/main/java/org/apache/paimon/TableType.java 
b/paimon-api/src/main/java/org/apache/paimon/TableType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/TableType.java
rename to paimon-api/src/main/java/org/apache/paimon/TableType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/annotation/ConfigGroup.java 
b/paimon-api/src/main/java/org/apache/paimon/annotation/ConfigGroup.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/annotation/ConfigGroup.java
rename to paimon-api/src/main/java/org/apache/paimon/annotation/ConfigGroup.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/annotation/ConfigGroups.java 
b/paimon-api/src/main/java/org/apache/paimon/annotation/ConfigGroups.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/annotation/ConfigGroups.java
rename to 
paimon-api/src/main/java/org/apache/paimon/annotation/ConfigGroups.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/annotation/Documentation.java 
b/paimon-api/src/main/java/org/apache/paimon/annotation/Documentation.java
similarity index 96%
rename from 
paimon-common/src/main/java/org/apache/paimon/annotation/Documentation.java
rename to 
paimon-api/src/main/java/org/apache/paimon/annotation/Documentation.java
index d7d917a942..bea0e7599b 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/annotation/Documentation.java
+++ b/paimon-api/src/main/java/org/apache/paimon/annotation/Documentation.java
@@ -18,8 +18,6 @@
 
 package org.apache.paimon.annotation;
 
-import org.apache.paimon.options.ConfigOption;
-
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
@@ -35,7 +33,7 @@ public final class Documentation {
         String value();
     }
 
-    /** Annotation used on {@link ConfigOption} fields to exclude it from 
schema change. */
+    /** Annotation used on {@code ConfigOption} fields to exclude it from 
schema change. */
     @Target(ElementType.FIELD)
     @Retention(RetentionPolicy.RUNTIME)
     public @interface Immutable {}
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/annotation/Experimental.java 
b/paimon-api/src/main/java/org/apache/paimon/annotation/Experimental.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/annotation/Experimental.java
rename to 
paimon-api/src/main/java/org/apache/paimon/annotation/Experimental.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/annotation/Public.java 
b/paimon-api/src/main/java/org/apache/paimon/annotation/Public.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/annotation/Public.java
rename to paimon-api/src/main/java/org/apache/paimon/annotation/Public.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/annotation/VisibleForTesting.java
 b/paimon-api/src/main/java/org/apache/paimon/annotation/VisibleForTesting.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/annotation/VisibleForTesting.java
rename to 
paimon-api/src/main/java/org/apache/paimon/annotation/VisibleForTesting.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java 
b/paimon-api/src/main/java/org/apache/paimon/catalog/Identifier.java
similarity index 89%
rename from paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java
rename to paimon-api/src/main/java/org/apache/paimon/catalog/Identifier.java
index 7b932f5ee5..f88431924e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java
+++ b/paimon-api/src/main/java/org/apache/paimon/catalog/Identifier.java
@@ -22,7 +22,6 @@ import org.apache.paimon.annotation.Public;
 import org.apache.paimon.types.DataField;
 import org.apache.paimon.types.DataTypes;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.BranchManager;
 import org.apache.paimon.utils.Preconditions;
 import org.apache.paimon.utils.StringUtils;
 
@@ -41,7 +40,7 @@ import java.util.Objects;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 
 /**
- * Identifies an object in a catalog.
+ * Identifies an object.
  *
  * @since 0.4.0
  */
@@ -54,6 +53,10 @@ public class Identifier implements Serializable {
     private static final String FIELD_DATABASE_NAME = "database";
     private static final String FIELD_OBJECT_NAME = "object";
 
+    public static final String SYSTEM_TABLE_SPLITTER = "$";
+    public static final String SYSTEM_BRANCH_PREFIX = "branch_";
+    public static final String DEFAULT_MAIN_BRANCH = "main";
+
     public static final RowType SCHEMA =
             new RowType(
                     false,
@@ -93,12 +96,10 @@ public class Identifier implements Serializable {
 
         StringBuilder builder = new StringBuilder(table);
         if (branch != null && !"main".equalsIgnoreCase(branch)) {
-            builder.append(Catalog.SYSTEM_TABLE_SPLITTER)
-                    .append(Catalog.SYSTEM_BRANCH_PREFIX)
-                    .append(branch);
+            
builder.append(SYSTEM_TABLE_SPLITTER).append(SYSTEM_BRANCH_PREFIX).append(branch);
         }
         if (systemTable != null) {
-            builder.append(Catalog.SYSTEM_TABLE_SPLITTER).append(systemTable);
+            builder.append(SYSTEM_TABLE_SPLITTER).append(systemTable);
         }
         this.object = builder.toString();
 
@@ -139,7 +140,7 @@ public class Identifier implements Serializable {
     @JsonIgnore
     public String getBranchNameOrDefault() {
         String branch = getBranchName();
-        return branch == null ? BranchManager.DEFAULT_MAIN_BRANCH : branch;
+        return branch == null ? DEFAULT_MAIN_BRANCH : branch;
     }
 
     @JsonIgnore
@@ -158,15 +159,15 @@ public class Identifier implements Serializable {
             return;
         }
 
-        String[] splits = StringUtils.split(object, 
Catalog.SYSTEM_TABLE_SPLITTER, -1, true);
+        String[] splits = StringUtils.split(object, SYSTEM_TABLE_SPLITTER, -1, 
true);
         if (splits.length == 1) {
             table = object;
             branch = null;
             systemTable = null;
         } else if (splits.length == 2) {
             table = splits[0];
-            if (splits[1].startsWith(Catalog.SYSTEM_BRANCH_PREFIX)) {
-                branch = 
splits[1].substring(Catalog.SYSTEM_BRANCH_PREFIX.length());
+            if (splits[1].startsWith(SYSTEM_BRANCH_PREFIX)) {
+                branch = splits[1].substring(SYSTEM_BRANCH_PREFIX.length());
                 systemTable = null;
             } else {
                 branch = null;
@@ -174,10 +175,10 @@ public class Identifier implements Serializable {
             }
         } else if (splits.length == 3) {
             Preconditions.checkArgument(
-                    splits[1].startsWith(Catalog.SYSTEM_BRANCH_PREFIX),
+                    splits[1].startsWith(SYSTEM_BRANCH_PREFIX),
                     "System table can only contain one '$' separator, but this 
is: " + object);
             table = splits[0];
-            branch = 
splits[1].substring(Catalog.SYSTEM_BRANCH_PREFIX.length());
+            branch = splits[1].substring(SYSTEM_BRANCH_PREFIX.length());
             systemTable = splits[2];
         } else {
             throw new IllegalArgumentException("Invalid object name: " + 
object);
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/compression/CompressOptions.java
 b/paimon-api/src/main/java/org/apache/paimon/compression/CompressOptions.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/compression/CompressOptions.java
rename to 
paimon-api/src/main/java/org/apache/paimon/compression/CompressOptions.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexOptions.java 
b/paimon-api/src/main/java/org/apache/paimon/fileindex/FileIndexOptions.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexOptions.java
rename to 
paimon-api/src/main/java/org/apache/paimon/fileindex/FileIndexOptions.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/Path.java 
b/paimon-api/src/main/java/org/apache/paimon/fs/Path.java
similarity index 97%
rename from paimon-common/src/main/java/org/apache/paimon/fs/Path.java
rename to paimon-api/src/main/java/org/apache/paimon/fs/Path.java
index 3cfeac5b6d..7dddf0e0ff 100644
--- a/paimon-common/src/main/java/org/apache/paimon/fs/Path.java
+++ b/paimon-api/src/main/java/org/apache/paimon/fs/Path.java
@@ -27,12 +27,8 @@ import java.net.URISyntaxException;
 import java.util.UUID;
 import java.util.regex.Pattern;
 
-/* This file is based on source code from the Hadoop Project 
(http://hadoop.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. */
-
 /**
- * Names a file or directory in a {@link FileIO}. Path strings use slash as 
the directory separator.
+ * Names a file or directory in a {@code FileIO}. Path strings use slash as 
the directory separator.
  *
  * @since 0.4.0
  */
diff --git a/paimon-core/src/main/java/org/apache/paimon/function/Function.java 
b/paimon-api/src/main/java/org/apache/paimon/function/Function.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/function/Function.java
rename to paimon-api/src/main/java/org/apache/paimon/function/Function.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/function/FunctionChange.java 
b/paimon-api/src/main/java/org/apache/paimon/function/FunctionChange.java
similarity index 100%
rename from 
paimon-core/src/main/java/org/apache/paimon/function/FunctionChange.java
rename to 
paimon-api/src/main/java/org/apache/paimon/function/FunctionChange.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/function/FunctionDefinition.java 
b/paimon-api/src/main/java/org/apache/paimon/function/FunctionDefinition.java
similarity index 100%
rename from 
paimon-core/src/main/java/org/apache/paimon/function/FunctionDefinition.java
rename to 
paimon-api/src/main/java/org/apache/paimon/function/FunctionDefinition.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/function/FunctionImpl.java 
b/paimon-api/src/main/java/org/apache/paimon/function/FunctionImpl.java
similarity index 84%
rename from 
paimon-core/src/main/java/org/apache/paimon/function/FunctionImpl.java
rename to paimon-api/src/main/java/org/apache/paimon/function/FunctionImpl.java
index 683d23d173..85ad987de9 100644
--- a/paimon-core/src/main/java/org/apache/paimon/function/FunctionImpl.java
+++ b/paimon-api/src/main/java/org/apache/paimon/function/FunctionImpl.java
@@ -18,7 +18,6 @@
 
 package org.apache.paimon.function;
 
-import org.apache.paimon.rest.responses.GetFunctionResponse;
 import org.apache.paimon.types.DataField;
 
 import java.util.List;
@@ -62,17 +61,6 @@ public class FunctionImpl implements Function {
         this.options = options;
     }
 
-    public FunctionImpl(GetFunctionResponse response) {
-        this.uuid = response.uuid();
-        this.name = response.name();
-        this.inputParams = response.inputParams();
-        this.returnParams = response.returnParams();
-        this.deterministic = response.isDeterministic();
-        this.definitions = response.definitions();
-        this.comment = response.comment();
-        this.options = response.options();
-    }
-
     @Override
     public String uuid() {
         return this.uuid;
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/lookup/LookupStrategy.java 
b/paimon-api/src/main/java/org/apache/paimon/lookup/LookupStrategy.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/lookup/LookupStrategy.java
rename to paimon-api/src/main/java/org/apache/paimon/lookup/LookupStrategy.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java 
b/paimon-api/src/main/java/org/apache/paimon/options/CatalogOptions.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
rename to paimon-api/src/main/java/org/apache/paimon/options/CatalogOptions.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/ConfigOption.java 
b/paimon-api/src/main/java/org/apache/paimon/options/ConfigOption.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/ConfigOption.java
rename to paimon-api/src/main/java/org/apache/paimon/options/ConfigOption.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java 
b/paimon-api/src/main/java/org/apache/paimon/options/ConfigOptions.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java
rename to paimon-api/src/main/java/org/apache/paimon/options/ConfigOptions.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/ExpireConfig.java 
b/paimon-api/src/main/java/org/apache/paimon/options/ExpireConfig.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/ExpireConfig.java
rename to paimon-api/src/main/java/org/apache/paimon/options/ExpireConfig.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/FallbackKey.java 
b/paimon-api/src/main/java/org/apache/paimon/options/FallbackKey.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/FallbackKey.java
rename to paimon-api/src/main/java/org/apache/paimon/options/FallbackKey.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/MemorySize.java 
b/paimon-api/src/main/java/org/apache/paimon/options/MemorySize.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/MemorySize.java
rename to paimon-api/src/main/java/org/apache/paimon/options/MemorySize.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/options/Options.java 
b/paimon-api/src/main/java/org/apache/paimon/options/Options.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/options/Options.java
rename to paimon-api/src/main/java/org/apache/paimon/options/Options.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/options/OptionsUtils.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java
rename to paimon-api/src/main/java/org/apache/paimon/options/OptionsUtils.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/StructuredOptionsSplitter.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/StructuredOptionsSplitter.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/StructuredOptionsSplitter.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/StructuredOptionsSplitter.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/BlockElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/BlockElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/BlockElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/BlockElement.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/DescribedEnum.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/DescribedEnum.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/DescribedEnum.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/DescribedEnum.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/Description.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/Description.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/Description.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/Description.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/DescriptionElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/DescriptionElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/DescriptionElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/DescriptionElement.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/Formatter.java
 b/paimon-api/src/main/java/org/apache/paimon/options/description/Formatter.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/Formatter.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/Formatter.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/HtmlFormatter.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/HtmlFormatter.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/HtmlFormatter.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/HtmlFormatter.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/InlineElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/InlineElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/InlineElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/InlineElement.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/LineBreakElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/LineBreakElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/LineBreakElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/LineBreakElement.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/LinkElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/LinkElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/LinkElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/LinkElement.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/ListElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/ListElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/ListElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/ListElement.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/options/description/TextElement.java
 
b/paimon-api/src/main/java/org/apache/paimon/options/description/TextElement.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/options/description/TextElement.java
rename to 
paimon-api/src/main/java/org/apache/paimon/options/description/TextElement.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/RESTMessage.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java
rename to paimon-api/src/main/java/org/apache/paimon/rest/RESTMessage.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/RESTRequest.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java
rename to paimon-api/src/main/java/org/apache/paimon/rest/RESTRequest.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterTableRequest.java
 
b/paimon-api/src/main/java/org/apache/paimon/rest/requests/AlterTableRequest.java
similarity index 100%
rename from 
paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterTableRequest.java
rename to 
paimon-api/src/main/java/org/apache/paimon/rest/requests/AlterTableRequest.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
 
b/paimon-api/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
similarity index 100%
rename from 
paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
rename to 
paimon-api/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
 
b/paimon-api/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
similarity index 100%
rename from 
paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
rename to 
paimon-api/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java 
b/paimon-api/src/main/java/org/apache/paimon/schema/Schema.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/schema/Schema.java
rename to paimon-api/src/main/java/org/apache/paimon/schema/Schema.java
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java 
b/paimon-api/src/main/java/org/apache/paimon/schema/SchemaChange.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java
rename to paimon-api/src/main/java/org/apache/paimon/schema/SchemaChange.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/table/CatalogTableType.java 
b/paimon-api/src/main/java/org/apache/paimon/table/CatalogTableType.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/table/CatalogTableType.java
rename to paimon-api/src/main/java/org/apache/paimon/table/CatalogTableType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java 
b/paimon-api/src/main/java/org/apache/paimon/table/SpecialFields.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java
rename to paimon-api/src/main/java/org/apache/paimon/table/SpecialFields.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/ArrayType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/ArrayType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/BigIntType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/BigIntType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/BigIntType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/BigIntType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/BinaryType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/BinaryType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/BinaryType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/BinaryType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/BooleanType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/BooleanType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/BooleanType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/BooleanType.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/CharType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/CharType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/CharType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/CharType.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataField.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/DataField.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataField.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataType.java
similarity index 95%
rename from paimon-common/src/main/java/org/apache/paimon/types/DataType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataType.java
index ac1e5b2a64..860463ce4b 100644
--- a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java
+++ b/paimon-api/src/main/java/org/apache/paimon/types/DataType.java
@@ -29,10 +29,6 @@ import java.util.Arrays;
 import java.util.Objects;
 import java.util.Set;
 
-/* This file is based on source code of Apache Flink Project 
(https://flink.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. */
-
 /**
  * Describes the data type in the paimon ecosystem.
  *
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeCasts.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeCasts.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeCasts.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataTypeCasts.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeChecks.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeChecks.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeChecks.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataTypeChecks.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java
 b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java
rename to 
paimon-api/src/main/java/org/apache/paimon/types/DataTypeDefaultVisitor.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeFamily.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeFamily.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeFamily.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataTypeFamily.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java
rename to 
paimon-api/src/main/java/org/apache/paimon/types/DataTypeJsonParser.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeRoot.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeRoot.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeRoot.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataTypeRoot.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DataTypeVisitor.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypeVisitor.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/DataTypeVisitor.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataTypeVisitor.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataTypes.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DataTypes.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/DataTypes.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DataTypes.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DateType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DateType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/DateType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DateType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DecimalType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DecimalType.java
similarity index 92%
rename from paimon-common/src/main/java/org/apache/paimon/types/DecimalType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DecimalType.java
index 88e5c8300f..d0a4d82b1e 100644
--- a/paimon-common/src/main/java/org/apache/paimon/types/DecimalType.java
+++ b/paimon-api/src/main/java/org/apache/paimon/types/DecimalType.java
@@ -19,7 +19,6 @@
 package org.apache.paimon.types;
 
 import org.apache.paimon.annotation.Public;
-import org.apache.paimon.data.Decimal;
 
 import java.util.Objects;
 
@@ -33,6 +32,8 @@ public class DecimalType extends DataType {
 
     private static final long serialVersionUID = 1L;
 
+    public static final int MAX_COMPACT_PRECISION = 18;
+
     public static final int MIN_PRECISION = 1;
 
     public static final int MAX_PRECISION = 38;
@@ -89,7 +90,7 @@ public class DecimalType extends DataType {
 
     @Override
     public int defaultSize() {
-        return Decimal.isCompact(precision) ? 8 : 16;
+        return isCompact(precision) ? 8 : 16;
     }
 
     @Override
@@ -126,4 +127,9 @@ public class DecimalType extends DataType {
     public <R> R accept(DataTypeVisitor<R> visitor) {
         return visitor.visit(this);
     }
+
+    /** Returns whether the decimal value is small enough to be stored in a 
long. */
+    public static boolean isCompact(int precision) {
+        return precision <= MAX_COMPACT_PRECISION;
+    }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/DoubleType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/DoubleType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/DoubleType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/DoubleType.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/FloatType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/FloatType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/FloatType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/FloatType.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/IntType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/IntType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/IntType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/IntType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/LocalZonedTimestampType.java
 b/paimon-api/src/main/java/org/apache/paimon/types/LocalZonedTimestampType.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/LocalZonedTimestampType.java
rename to 
paimon-api/src/main/java/org/apache/paimon/types/LocalZonedTimestampType.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/MapType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/MapType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/MapType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/MultisetType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/MultisetType.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/MultisetType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/MultisetType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/ReassignFieldId.java 
b/paimon-api/src/main/java/org/apache/paimon/types/ReassignFieldId.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/ReassignFieldId.java
rename to paimon-api/src/main/java/org/apache/paimon/types/ReassignFieldId.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/RowKind.java 
b/paimon-api/src/main/java/org/apache/paimon/types/RowKind.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/RowKind.java
rename to paimon-api/src/main/java/org/apache/paimon/types/RowKind.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/RowType.java
similarity index 96%
rename from paimon-common/src/main/java/org/apache/paimon/types/RowType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/RowType.java
index d0c0881b6a..5aa58fb375 100644
--- a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java
+++ b/paimon-api/src/main/java/org/apache/paimon/types/RowType.java
@@ -19,7 +19,6 @@
 package org.apache.paimon.types;
 
 import org.apache.paimon.annotation.Public;
-import org.apache.paimon.data.InternalRow;
 import org.apache.paimon.table.SpecialFields;
 import org.apache.paimon.utils.Preconditions;
 import org.apache.paimon.utils.StringUtils;
@@ -38,7 +37,6 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
-import java.util.stream.IntStream;
 
 /**
  * Data type of a sequence of fields. A field consists of a field name, field 
type, and an optional
@@ -59,7 +57,6 @@ public final class RowType extends DataType {
     public static final String FORMAT = "ROW<%s>";
 
     private final List<DataField> fields;
-    private InternalRow.FieldGetter[] fieldGetters;
 
     public RowType(boolean isNullable, List<DataField> fields) {
         super(isNullable, DataTypeRoot.ROW);
@@ -420,15 +417,4 @@ public final class RowType extends DataType {
             return new RowType(isNullable, fields);
         }
     }
-
-    public InternalRow.FieldGetter[] fieldGetters() {
-        if (fieldGetters == null) {
-            fieldGetters =
-                    IntStream.range(0, getFieldCount())
-                            .mapToObj(i -> 
InternalRow.createFieldGetter(getTypeAt(i), i))
-                            .toArray(InternalRow.FieldGetter[]::new);
-        }
-
-        return fieldGetters;
-    }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/SmallIntType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/SmallIntType.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/SmallIntType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/SmallIntType.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/types/TimeType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/TimeType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/TimeType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/TimeType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/TimestampType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/TimestampType.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/TimestampType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/TimestampType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/TinyIntType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/TinyIntType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/TinyIntType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/TinyIntType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/VarBinaryType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/VarBinaryType.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/types/VarBinaryType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/VarBinaryType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/VarCharType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/VarCharType.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/types/VarCharType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/VarCharType.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/types/VariantType.java 
b/paimon-api/src/main/java/org/apache/paimon/types/VariantType.java
similarity index 95%
rename from paimon-common/src/main/java/org/apache/paimon/types/VariantType.java
rename to paimon-api/src/main/java/org/apache/paimon/types/VariantType.java
index 3537dd6e15..17dbf82ef5 100644
--- a/paimon-common/src/main/java/org/apache/paimon/types/VariantType.java
+++ b/paimon-api/src/main/java/org/apache/paimon/types/VariantType.java
@@ -19,10 +19,9 @@
 package org.apache.paimon.types;
 
 import org.apache.paimon.annotation.Public;
-import org.apache.paimon.data.variant.Variant;
 
 /**
- * Data type of {@link Variant}.
+ * Data type of {@code Variant}.
  *
  * @since 1.1.0
  */
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/ArrayUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/ArrayUtils.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/utils/ArrayUtils.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/ArrayUtils.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/EncodingUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/EncodingUtils.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/utils/EncodingUtils.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/EncodingUtils.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/MathUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/MathUtils.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/utils/MathUtils.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/MathUtils.java
diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/Pair.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/Pair.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/utils/Pair.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/Pair.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/Preconditions.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/Preconditions.java
similarity index 100%
rename from 
paimon-common/src/main/java/org/apache/paimon/utils/Preconditions.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/Preconditions.java
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java
similarity index 93%
rename from paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java
index 5aa4e65f4c..2581c9ac82 100644
--- a/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java
+++ b/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java
@@ -18,24 +18,17 @@
 
 package org.apache.paimon.utils;
 
-import org.apache.paimon.data.BinaryString;
-import org.apache.paimon.memory.MemorySegmentUtils;
-
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 
-import static org.apache.paimon.data.BinaryString.fromBytes;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 import static org.apache.paimon.utils.Preconditions.checkNotNull;
 
-/**
- * Utils for {@link BinaryString} and utility class to convert objects into 
strings in vice-versa.
- */
+/** Utils for strings. */
 public class StringUtils {
 
     private static final char[] HEX_CHARS = {
@@ -52,39 +45,6 @@ public class StringUtils {
     /** The empty String {@code ""}. */
     public static final String EMPTY = "";
 
-    /**
-     * Concatenates input strings together into a single string. Returns NULL 
if any argument is
-     * NULL.
-     */
-    public static BinaryString concat(BinaryString... inputs) {
-        return concat(Arrays.asList(inputs));
-    }
-
-    public static BinaryString concat(Iterable<BinaryString> inputs) {
-        // Compute the total length of the result.
-        int totalLength = 0;
-        for (BinaryString input : inputs) {
-            if (input == null) {
-                return null;
-            }
-
-            totalLength += input.getSizeInBytes();
-        }
-
-        // Allocate a new byte array, and copy the inputs one by one into it.
-        final byte[] result = new byte[totalLength];
-        int offset = 0;
-        for (BinaryString input : inputs) {
-            if (input != null) {
-                int len = input.getSizeInBytes();
-                MemorySegmentUtils.copyToBytes(
-                        input.getSegments(), input.getOffset(), result, 
offset, len);
-                offset += len;
-            }
-        }
-        return fromBytes(result);
-    }
-
     /**
      * Checks if the string is null, empty, or contains only whitespace 
characters. A whitespace
      * character is defined via {@link Character#isWhitespace(char)}.
diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/TimeUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/TimeUtils.java
similarity index 100%
rename from paimon-common/src/main/java/org/apache/paimon/utils/TimeUtils.java
rename to paimon-api/src/main/java/org/apache/paimon/utils/TimeUtils.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/view/View.java 
b/paimon-api/src/main/java/org/apache/paimon/view/View.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/view/View.java
rename to paimon-api/src/main/java/org/apache/paimon/view/View.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/view/ViewChange.java 
b/paimon-api/src/main/java/org/apache/paimon/view/ViewChange.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/view/ViewChange.java
rename to paimon-api/src/main/java/org/apache/paimon/view/ViewChange.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/view/ViewImpl.java 
b/paimon-api/src/main/java/org/apache/paimon/view/ViewImpl.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/view/ViewImpl.java
rename to paimon-api/src/main/java/org/apache/paimon/view/ViewImpl.java
diff --git a/paimon-core/src/main/java/org/apache/paimon/view/ViewSchema.java 
b/paimon-api/src/main/java/org/apache/paimon/view/ViewSchema.java
similarity index 100%
rename from paimon-core/src/main/java/org/apache/paimon/view/ViewSchema.java
rename to paimon-api/src/main/java/org/apache/paimon/view/ViewSchema.java
diff --git a/paimon-bundle/pom.xml b/paimon-bundle/pom.xml
index a9d04baf42..3abaded69a 100644
--- a/paimon-bundle/pom.xml
+++ b/paimon-bundle/pom.xml
@@ -35,6 +35,12 @@ under the License.
 
     <dependencies>
 
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-common</artifactId>
@@ -91,12 +97,6 @@ under the License.
 
         <!-- other runtime dependencies -->
 
-        <dependency>
-            <groupId>org.apache.paimon</groupId>
-            <artifactId>paimon-shade-jackson-2</artifactId>
-            
<version>${paimon.shade.jackson.version}-${paimon.shade.version}</version>
-        </dependency>
-
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-shade-guava-30</artifactId>
@@ -126,11 +126,11 @@ under the License.
                             <artifactSet>
                                 <includes combine.children="append">
                                     <!-- Paimon-shade -->
-                                    
<include>org.apache.paimon:paimon-shade-jackson-2</include>
                                     
<include>org.apache.paimon:paimon-shade-guava-30</include>
                                     
<include>org.apache.paimon:paimon-shade-caffeine-2</include>
 
                                     <!-- Paimon -->
+                                    
<include>org.apache.paimon:paimon-api</include>
                                     
<include>org.apache.paimon:paimon-common</include>
                                     
<include>org.apache.paimon:paimon-core</include>
                                     
<include>org.apache.paimon:paimon-format</include>
diff --git a/paimon-common/pom.xml b/paimon-common/pom.xml
index bd60d00940..ad86d48d3e 100644
--- a/paimon-common/pom.xml
+++ b/paimon-common/pom.xml
@@ -32,6 +32,12 @@ under the License.
     <name>Paimon : Common</name>
 
     <dependencies>
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-shade-jackson-2</artifactId>
diff --git a/paimon-common/src/main/java/org/apache/paimon/data/Decimal.java 
b/paimon-common/src/main/java/org/apache/paimon/data/Decimal.java
index 71a484665d..2d81647653 100644
--- a/paimon-common/src/main/java/org/apache/paimon/data/Decimal.java
+++ b/paimon-common/src/main/java/org/apache/paimon/data/Decimal.java
@@ -29,6 +29,7 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.math.RoundingMode;
 
+import static org.apache.paimon.types.DecimalType.MAX_COMPACT_PRECISION;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 
 /**
@@ -44,11 +45,6 @@ public final class Decimal implements Comparable<Decimal>, 
Serializable {
 
     private static final long serialVersionUID = 1L;
 
-    // member fields and static fields are package-visible,
-    // in order to be accessible for DecimalUtils
-
-    static final int MAX_COMPACT_PRECISION = 18;
-
     /** Maximum number of decimal digits a Long can represent. (1e18 < 
Long.MaxValue < 1e19) */
     static final int MAX_LONG_DIGITS = 18;
 
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java 
b/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java
index e1391e7f53..bef47edbf3 100644
--- a/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java
+++ b/paimon-common/src/main/java/org/apache/paimon/format/FileFormat.java
@@ -34,6 +34,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
+import static org.apache.paimon.CoreOptions.normalizeFileFormat;
+
 /**
  * Factory class which creates reader and writer factories for specific file 
format.
  *
@@ -77,7 +79,7 @@ public abstract class FileFormat {
 
     public static FileFormat fromIdentifier(String identifier, Options 
options) {
         return fromIdentifier(
-                identifier,
+                normalizeFileFormat(identifier),
                 new FormatContext(
                         options,
                         options.get(CoreOptions.READ_BATCH_SIZE),
@@ -103,4 +105,12 @@ public abstract class FileFormat {
         }
         return new Options(result);
     }
+
+    public static FileFormat fileFormat(CoreOptions options) {
+        return FileFormat.fromIdentifier(options.fileFormatString(), 
options.toConfiguration());
+    }
+
+    public static FileFormat manifestFormat(CoreOptions options) {
+        return FileFormat.fromIdentifier(options.manifestFormatString(), 
options.toConfiguration());
+    }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java 
b/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java
index 299823d35d..43ab156f17 100644
--- a/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java
+++ b/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java
@@ -356,7 +356,7 @@ public class BinaryStringUtils {
         } else if (strData.numChars() < targetLength && targetCharType) {
             int padLength = targetLength - strData.numChars();
             BinaryString padString = BinaryString.blankString(padLength);
-            return StringUtils.concat(strData, padString);
+            return concat(strData, padString);
         }
         return strData;
     }
@@ -377,4 +377,37 @@ public class BinaryStringUtils {
             }
         }
     }
+
+    /**
+     * Concatenates input strings together into a single string. Returns NULL 
if any argument is
+     * NULL.
+     */
+    public static BinaryString concat(BinaryString... inputs) {
+        return concat(Arrays.asList(inputs));
+    }
+
+    public static BinaryString concat(Iterable<BinaryString> inputs) {
+        // Compute the total length of the result.
+        int totalLength = 0;
+        for (BinaryString input : inputs) {
+            if (input == null) {
+                return null;
+            }
+
+            totalLength += input.getSizeInBytes();
+        }
+
+        // Allocate a new byte array, and copy the inputs one by one into it.
+        final byte[] result = new byte[totalLength];
+        int offset = 0;
+        for (BinaryString input : inputs) {
+            if (input != null) {
+                int len = input.getSizeInBytes();
+                MemorySegmentUtils.copyToBytes(
+                        input.getSegments(), input.getOffset(), result, 
offset, len);
+                offset += len;
+            }
+        }
+        return BinaryString.fromBytes(result);
+    }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java
 
b/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java
index f4aad8f03f..edefce3189 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java
@@ -31,6 +31,7 @@ import org.apache.paimon.types.VarCharType;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.IntStream;
 
 import static 
org.apache.paimon.utils.InternalRowUtils.createNullCheckingFieldGetter;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
@@ -122,7 +123,10 @@ public class InternalRowPartitionComputer {
 
     public static String partToSimpleString(
             RowType partitionType, BinaryRow partition, String delimiter, int 
maxLength) {
-        FieldGetter[] getters = partitionType.fieldGetters();
+        FieldGetter[] getters =
+                IntStream.range(0, partitionType.getFieldCount())
+                        .mapToObj(i -> 
InternalRow.createFieldGetter(partitionType.getTypeAt(i), i))
+                        .toArray(InternalRow.FieldGetter[]::new);
         StringBuilder builder = new StringBuilder();
         for (int i = 0; i < getters.length; i++) {
             Object part = getters[i].getFieldOrNull(partition);
diff --git 
a/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java 
b/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java
index 632be0cf59..55eade1898 100644
--- 
a/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java
+++ 
b/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java
@@ -52,6 +52,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.IntStream;
 
 import static org.apache.paimon.data.BinaryString.fromString;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -353,7 +354,10 @@ public abstract class FormatReadWriteTest {
 
     private void validateFullTypesResult(InternalRow actual, InternalRow 
expected) {
         RowType rowType = rowTypeForFullTypesTest();
-        InternalRow.FieldGetter[] fieldGetters = rowType.fieldGetters();
+        InternalRow.FieldGetter[] fieldGetters =
+                IntStream.range(0, rowType.getFieldCount())
+                        .mapToObj(i -> 
InternalRow.createFieldGetter(rowType.getTypeAt(i), i))
+                        .toArray(InternalRow.FieldGetter[]::new);
         for (int i = 0; i < fieldGetters.length; i++) {
             String name = rowType.getFieldNames().get(i);
             Object actualField = fieldGetters[i].getFieldOrNull(actual);
diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java 
b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
index c0fb464092..1b0f07efb3 100644
--- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
@@ -23,6 +23,7 @@ import org.apache.paimon.catalog.RenamingSnapshotCommit;
 import org.apache.paimon.catalog.SnapshotCommit;
 import org.apache.paimon.data.InternalRow;
 import org.apache.paimon.deletionvectors.DeletionVectorsIndexFile;
+import org.apache.paimon.format.FileFormat;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.iceberg.IcebergCommitCallback;
@@ -199,7 +200,7 @@ abstract class AbstractFileStore<T> implements FileStore<T> 
{
                 fileIO,
                 schemaManager,
                 partitionType,
-                options.manifestFormat(),
+                FileFormat.manifestFormat(options),
                 options.manifestCompression(),
                 pathFactory(),
                 options.manifestTargetSize().getBytes(),
@@ -214,7 +215,7 @@ abstract class AbstractFileStore<T> implements FileStore<T> 
{
     protected ManifestList.Factory manifestListFactory(boolean forWrite) {
         return new ManifestList.Factory(
                 fileIO,
-                options.manifestFormat(),
+                FileFormat.manifestFormat(options),
                 options.manifestCompression(),
                 pathFactory(),
                 forWrite ? writeManifestCache : readManifestCache);
@@ -224,7 +225,7 @@ abstract class AbstractFileStore<T> implements FileStore<T> 
{
     public IndexManifestFile.Factory indexManifestFileFactory() {
         return new IndexManifestFile.Factory(
                 fileIO,
-                options.manifestFormat(),
+                FileFormat.manifestFormat(options),
                 options.manifestCompression(),
                 pathFactory(),
                 readManifestCache);
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 bd3624b66e..8286155fe2 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
@@ -69,9 +69,9 @@ import static 
org.apache.paimon.catalog.CatalogUtils.checkNotSystemTable;
 import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase;
 import static 
org.apache.paimon.catalog.CatalogUtils.listPartitionsFromFileSystem;
 import static org.apache.paimon.catalog.CatalogUtils.validateAutoCreateClose;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 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;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 
 /** Common implementation of {@link Catalog}. */
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 1494c2df19..5183891cef 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
@@ -738,10 +738,8 @@ public interface Catalog extends AutoCloseable {
 
     // ======================= Constants ===============================
 
-    // constants for system table and database
-    String SYSTEM_TABLE_SPLITTER = "$";
+    // constants for sys database
     String SYSTEM_DATABASE_NAME = "sys";
-    String SYSTEM_BRANCH_PREFIX = "branch_";
 
     // constants for table and database
     String COMMENT_PROP = "comment";
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/consumer/ConsumerManager.java 
b/paimon-core/src/main/java/org/apache/paimon/consumer/ConsumerManager.java
index 4762238b6f..8a2ad90ff7 100644
--- a/paimon-core/src/main/java/org/apache/paimon/consumer/ConsumerManager.java
+++ b/paimon-core/src/main/java/org/apache/paimon/consumer/ConsumerManager.java
@@ -35,7 +35,7 @@ import java.util.OptionalLong;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.BranchManager.branchPath;
 import static org.apache.paimon.utils.FileUtils.listOriginalVersionedFiles;
 import static org.apache.paimon.utils.FileUtils.listVersionedFileStatus;
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldListaggAgg.java
 
b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldListaggAgg.java
index a018915011..22843a6315 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldListaggAgg.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldListaggAgg.java
@@ -21,7 +21,7 @@ package org.apache.paimon.mergetree.compact.aggregate;
 import org.apache.paimon.CoreOptions;
 import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.types.VarCharType;
-import org.apache.paimon.utils.StringUtils;
+import org.apache.paimon.utils.BinaryStringUtils;
 
 /** listagg aggregate a field of a row. */
 public class FieldListaggAgg extends FieldAggregator {
@@ -46,6 +46,7 @@ public class FieldListaggAgg extends FieldAggregator {
         BinaryString mergeFieldSD = (BinaryString) accumulator;
         BinaryString inFieldSD = (BinaryString) inputField;
 
-        return StringUtils.concat(mergeFieldSD, 
BinaryString.fromString(delimiter), inFieldSD);
+        return BinaryStringUtils.concat(
+                mergeFieldSD, BinaryString.fromString(delimiter), inFieldSD);
     }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java
 
b/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java
index 5e57d9088a..ed75afdf2b 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/operation/BaseAppendFileStoreWrite.java
@@ -58,6 +58,8 @@ import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.function.Function;
 
+import static org.apache.paimon.format.FileFormat.fileFormat;
+
 /** {@link FileStoreWrite} for {@link AppendOnlyFileStore}. */
 public abstract class BaseAppendFileStoreWrite extends 
MemoryFileStoreWrite<InternalRow>
         implements BundleFileStoreWriter {
@@ -92,7 +94,7 @@ public abstract class BaseAppendFileStoreWrite extends 
MemoryFileStoreWrite<Inte
         this.read = read;
         this.schemaId = schemaId;
         this.rowType = rowType;
-        this.fileFormat = options.fileFormat();
+        this.fileFormat = fileFormat(options);
         this.pathFactory = pathFactory;
 
         this.statsCollectors =
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
 
b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
index 1b0e404a14..9a65e83d06 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
@@ -88,6 +88,7 @@ import java.util.function.Supplier;
 
 import static org.apache.paimon.CoreOptions.ChangelogProducer.FULL_COMPACTION;
 import static org.apache.paimon.CoreOptions.MergeEngine.DEDUPLICATE;
+import static org.apache.paimon.format.FileFormat.fileFormat;
 import static org.apache.paimon.lookup.LookupStoreFactory.bfGenerator;
 import static org.apache.paimon.mergetree.LookupFile.localFilePrefix;
 import static 
org.apache.paimon.utils.FileStorePathFactory.createFormatPathFactories;
@@ -166,7 +167,7 @@ public class KeyValueFileStoreWrite extends 
MemoryFileStoreWrite<KeyValue> {
                         schema.id(),
                         keyType,
                         valueType,
-                        options.fileFormat(),
+                        fileFormat(options),
                         createFormatPathFactories(options, formatPathFactory),
                         options.targetFileSize(true));
         this.keyComparatorSupplier = keyComparatorSupplier;
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java 
b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java
index 5d7e5b6d39..1af56a8f62 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java
@@ -30,7 +30,6 @@ import org.apache.paimon.manifest.ManifestFileMeta;
 import org.apache.paimon.manifest.ManifestList;
 import org.apache.paimon.schema.SchemaManager;
 import org.apache.paimon.table.FileStoreTable;
-import org.apache.paimon.utils.BranchManager;
 import org.apache.paimon.utils.ChangelogManager;
 import org.apache.paimon.utils.DateTimeUtils;
 import org.apache.paimon.utils.FileStorePathFactory;
@@ -61,6 +60,7 @@ import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import static java.util.Collections.emptyList;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.ChangelogManager.CHANGELOG_PREFIX;
 import static org.apache.paimon.utils.FileStorePathFactory.BUCKET_PATH_PREFIX;
 import static org.apache.paimon.utils.HintFileUtils.EARLIEST;
@@ -108,7 +108,7 @@ public abstract class OrphanFilesClean implements 
Serializable {
 
     protected List<String> validBranches() {
         List<String> branches = table.branchManager().branches();
-        branches.add(BranchManager.DEFAULT_MAIN_BRANCH);
+        branches.add(DEFAULT_MAIN_BRANCH);
 
         List<String> abnormalBranches = new ArrayList<>();
         for (String branch : branches) {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java
 
b/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java
index 0db2ae6377..b52bd0b812 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java
@@ -22,6 +22,7 @@ import org.apache.paimon.CoreOptions;
 import org.apache.paimon.KeyValue;
 import org.apache.paimon.data.BinaryRow;
 import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer;
+import org.apache.paimon.format.FileFormat;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.io.DataFileMeta;
 import org.apache.paimon.io.KeyValueFileWriterFactory;
@@ -45,6 +46,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
+import static org.apache.paimon.format.FileFormat.fileFormat;
 import static 
org.apache.paimon.utils.FileStorePathFactory.createFormatPathFactories;
 
 /** {@link FileStoreWrite} for {@code bucket = -2} tables. */
@@ -82,13 +84,14 @@ public class PostponeBucketFileStoreWrite extends 
AbstractFileStoreWrite<KeyValu
                         
ThreadLocalRandom.current().nextInt(Integer.MAX_VALUE)));
         this.options = new CoreOptions(newOptions);
 
+        FileFormat fileFormat = fileFormat(this.options);
         this.writerFactoryBuilder =
                 KeyValueFileWriterFactory.builder(
                         fileIO,
                         schema.id(),
                         keyType,
                         valueType,
-                        this.options.fileFormat(),
+                        fileFormat,
                         createFormatPathFactories(this.options, 
formatPathFactory),
                         this.options.targetFileSize(true));
 
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 a8a1a695ff..5e0b5bf3a5 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
@@ -31,7 +31,6 @@ import org.apache.paimon.catalog.TableMetadata;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.function.FunctionChange;
-import org.apache.paimon.function.FunctionImpl;
 import org.apache.paimon.options.Options;
 import org.apache.paimon.partition.Partition;
 import org.apache.paimon.partition.PartitionStatistics;
@@ -849,7 +848,7 @@ public class RESTCatalog implements Catalog {
                             resourcePaths.function(functionName),
                             GetFunctionResponse.class,
                             restAuthFunction);
-            return new FunctionImpl(response);
+            return response.toFunction();
         } catch (NoSuchResourceException e) {
             throw new FunctionNotExistException(functionName, e);
         }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetFunctionResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetFunctionResponse.java
index 6780c389e4..27fce00ef8 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetFunctionResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetFunctionResponse.java
@@ -18,7 +18,9 @@
 
 package org.apache.paimon.rest.responses;
 
+import org.apache.paimon.function.Function;
 import org.apache.paimon.function.FunctionDefinition;
+import org.apache.paimon.function.FunctionImpl;
 import org.apache.paimon.types.DataField;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
@@ -133,4 +135,16 @@ public class GetFunctionResponse extends AuditRESTResponse 
{
     public Map<String, String> options() {
         return options;
     }
+
+    public Function toFunction() {
+        return new FunctionImpl(
+                uuid,
+                functionName,
+                inputParams,
+                returnParams,
+                deterministic,
+                definitions,
+                comment,
+                options);
+    }
 }
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 351be613ad..f7af7b7c36 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
@@ -76,8 +76,8 @@ import java.util.stream.LongStream;
 
 import static org.apache.paimon.CoreOptions.BUCKET_KEY;
 import static org.apache.paimon.catalog.AbstractCatalog.DB_SUFFIX;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.catalog.Identifier.UNKNOWN_DATABASE;
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.FileUtils.listVersionedFiles;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 import static org.apache.paimon.utils.Preconditions.checkState;
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
index 2997014ad4..2424878951 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
@@ -51,6 +51,7 @@ import 
org.apache.paimon.table.source.snapshot.StaticFromWatermarkStartingScanne
 import org.apache.paimon.tag.Tag;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.ChangelogManager;
+import org.apache.paimon.utils.DateTimeUtils;
 import org.apache.paimon.utils.Filter;
 import org.apache.paimon.utils.Pair;
 import org.apache.paimon.utils.SnapshotManager;
@@ -65,6 +66,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.TimeZone;
 
 import static org.apache.paimon.CoreOptions.FULL_COMPACTION_DELTA_COMMITS;
 import static org.apache.paimon.CoreOptions.IncrementalBetweenScanMode.DIFF;
@@ -225,7 +227,13 @@ abstract class AbstractDataTableScan implements 
DataTableScan {
                     return new CompactedStartingScanner(snapshotManager);
                 }
             case FROM_TIMESTAMP:
+                String timestampStr = options.scanTimestamp();
                 Long startupMillis = options.scanTimestampMills();
+                if (startupMillis == null && timestampStr != null) {
+                    startupMillis =
+                            DateTimeUtils.parseTimestampData(timestampStr, 3, 
TimeZone.getDefault())
+                                    .getMillisecond();
+                }
                 return isStreaming
                         ? new ContinuousFromTimestampStartingScanner(
                                 snapshotManager,
@@ -329,7 +337,32 @@ abstract class AbstractDataTableScan implements 
DataTableScan {
                                 startId, endId, snapshotManager, 
toSnapshotScanMode(scanMode));
             }
         } else if (conf.contains(CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP)) {
-            Pair<Long, Long> incrementalBetween = 
options.incrementalBetweenTimestamp();
+            String incrementalBetweenStr = 
options.incrementalBetweenTimestamp();
+            String[] split = incrementalBetweenStr.split(",");
+            if (split.length != 2) {
+                throw new IllegalArgumentException(
+                        "The incremental-between-timestamp must specific 
start(exclusive) and end timestamp. But is: "
+                                + incrementalBetweenStr);
+            }
+
+            Pair<Long, Long> incrementalBetween;
+            try {
+                incrementalBetween = Pair.of(Long.parseLong(split[0]), 
Long.parseLong(split[1]));
+            } catch (NumberFormatException nfe) {
+                try {
+                    long startTimestamp =
+                            DateTimeUtils.parseTimestampData(split[0], 3, 
TimeZone.getDefault())
+                                    .getMillisecond();
+                    long endTimestamp =
+                            DateTimeUtils.parseTimestampData(split[1], 3, 
TimeZone.getDefault())
+                                    .getMillisecond();
+                    incrementalBetween = Pair.of(startTimestamp, endTimestamp);
+                } catch (Exception e) {
+                    throw new IllegalArgumentException(
+                            "The incremental-between-timestamp must specific 
start(exclusive) and end timestamp. But is: "
+                                    + incrementalBetweenStr);
+                }
+            }
 
             Snapshot earliestSnapshot = snapshotManager.earliestSnapshot();
             Snapshot latestSnapshot = snapshotManager.latestSnapshot();
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/TimeTravelUtil.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/TimeTravelUtil.java
index c2fa1ac5be..20802f961f 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/TimeTravelUtil.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/TimeTravelUtil.java
@@ -36,9 +36,17 @@ import javax.annotation.Nullable;
 
 import java.io.FileNotFoundException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
-
+import java.util.TimeZone;
+
+import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID;
+import static org.apache.paimon.CoreOptions.SCAN_TAG_NAME;
+import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP;
+import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS;
+import static org.apache.paimon.CoreOptions.SCAN_WATERMARK;
+import static org.apache.paimon.utils.DateTimeUtils.parseTimestampData;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 import static 
org.apache.paimon.utils.SnapshotManager.EARLIEST_SNAPSHOT_DEFAULT_RETRY_NUM;
 
@@ -50,10 +58,11 @@ public class TimeTravelUtil {
     private static final String WATERMARK_PREFIX = "watermark-";
 
     private static final String[] SCAN_KEYS = {
-        CoreOptions.SCAN_SNAPSHOT_ID.key(),
-        CoreOptions.SCAN_TAG_NAME.key(),
-        CoreOptions.SCAN_WATERMARK.key(),
-        CoreOptions.SCAN_TIMESTAMP_MILLIS.key()
+        SCAN_SNAPSHOT_ID.key(),
+        SCAN_TAG_NAME.key(),
+        SCAN_WATERMARK.key(),
+        SCAN_TIMESTAMP.key(),
+        SCAN_TIMESTAMP_MILLIS.key()
     };
 
     public static Snapshot tryTravelOrLatest(FileStoreTable table) {
@@ -83,31 +92,38 @@ public class TimeTravelUtil {
         checkArgument(
                 scanHandleKey.size() == 1,
                 String.format(
-                        "Only one of the following parameters may be set : 
[%s, %s, %s, %s]",
-                        CoreOptions.SCAN_SNAPSHOT_ID.key(),
-                        CoreOptions.SCAN_TAG_NAME.key(),
-                        CoreOptions.SCAN_WATERMARK.key(),
-                        CoreOptions.SCAN_TIMESTAMP_MILLIS.key()));
+                        "Only one of the following parameters may be set : %s",
+                        Arrays.toString(SCAN_KEYS)));
 
         String key = scanHandleKey.get(0);
         CoreOptions coreOptions = new CoreOptions(options);
         Snapshot snapshot;
-        if (key.equals(CoreOptions.SCAN_SNAPSHOT_ID.key())) {
+        if (key.equals(SCAN_SNAPSHOT_ID.key())) {
             snapshot =
                     new StaticFromSnapshotStartingScanner(
                                     snapshotManager, 
coreOptions.scanSnapshotId())
                             .getSnapshot();
-        } else if (key.equals(CoreOptions.SCAN_WATERMARK.key())) {
+        } else if (key.equals(SCAN_WATERMARK.key())) {
             snapshot =
                     new StaticFromWatermarkStartingScanner(
                                     snapshotManager, 
coreOptions.scanWatermark())
                             .getSnapshot();
-        } else if (key.equals(CoreOptions.SCAN_TIMESTAMP_MILLIS.key())) {
+        } else if (key.equals(SCAN_TIMESTAMP.key())) {
+            snapshot =
+                    new StaticFromTimestampStartingScanner(
+                                    snapshotManager,
+                                    parseTimestampData(
+                                                    
coreOptions.scanTimestamp(),
+                                                    3,
+                                                    TimeZone.getDefault())
+                                            .getMillisecond())
+                            .getSnapshot();
+        } else if (key.equals(SCAN_TIMESTAMP_MILLIS.key())) {
             snapshot =
                     new StaticFromTimestampStartingScanner(
                                     snapshotManager, 
coreOptions.scanTimestampMills())
                             .getSnapshot();
-        } else if (key.equals(CoreOptions.SCAN_TAG_NAME.key())) {
+        } else if (key.equals(SCAN_TAG_NAME.key())) {
             snapshot =
                     new StaticFromTagStartingScanner(snapshotManager, 
coreOptions.scanTagName())
                             .getSnapshot();
@@ -124,12 +140,12 @@ public class TimeTravelUtil {
         }
 
         if (tagManager.tagExists(version)) {
-            options.set(CoreOptions.SCAN_TAG_NAME, version);
+            options.set(SCAN_TAG_NAME, version);
         } else if (version.startsWith(WATERMARK_PREFIX)) {
             long watermark = 
Long.parseLong(version.substring(WATERMARK_PREFIX.length()));
-            options.set(CoreOptions.SCAN_WATERMARK, watermark);
+            options.set(SCAN_WATERMARK, watermark);
         } else if (version.chars().allMatch(Character::isDigit)) {
-            options.set(CoreOptions.SCAN_SNAPSHOT_ID.key(), version);
+            options.set(SCAN_SNAPSHOT_ID.key(), version);
         } else {
             throw new RuntimeException("Cannot find a time travel version for 
" + version);
         }
@@ -280,4 +296,14 @@ public class TimeTravelUtil {
             return endSnapshotId;
         }
     }
+
+    @Nullable
+    public static Long scanTimestampMills(Options options) {
+        String timestampStr = options.get(SCAN_TIMESTAMP);
+        Long timestampMillis = options.get(SCAN_TIMESTAMP_MILLIS);
+        if (timestampMillis == null && timestampStr != null) {
+            return parseTimestampData(timestampStr, 3, 
TimeZone.getDefault()).getMillisecond();
+        }
+        return timestampMillis;
+    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java
index a9a044cc08..6be92518d7 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java
@@ -55,7 +55,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.function.Function;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing Aggregation of table. */
 public class AggregationFieldsTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java
index 66b55b712d..5df38af3f2 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java
@@ -78,7 +78,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for reading audit log of table. */
 public class AuditLogTable implements DataTable, ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java
index a023028bd1..8488e3ff9b 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java
@@ -39,8 +39,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.IntStream;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /**
  * A {@link Table} for reading binlog of table. The binlog format is as below.
@@ -116,7 +117,13 @@ public class BinlogTable extends AuditLogTable {
         @Override
         public RecordReader<InternalRow> createReader(Split split) throws 
IOException {
             DataSplit dataSplit = (DataSplit) split;
-            InternalRow.FieldGetter[] fieldGetters = 
wrappedReadType.fieldGetters();
+            InternalRow.FieldGetter[] fieldGetters =
+                    IntStream.range(0, wrappedReadType.getFieldCount())
+                            .mapToObj(
+                                    i ->
+                                            InternalRow.createFieldGetter(
+                                                    
wrappedReadType.getTypeAt(i), i))
+                            .toArray(InternalRow.FieldGetter[]::new);
 
             if (dataSplit.isStreaming()) {
                 return new PackChangelogReader(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java
index d5dc8bf850..2330796a33 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java
@@ -58,7 +58,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing branches of table. */
 public class BranchesTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/BucketsTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/BucketsTable.java
index 703be592f0..10ca1775a1 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/BucketsTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BucketsTable.java
@@ -58,7 +58,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing buckets info. */
 public class BucketsTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java
index f6695a700f..6aad82190b 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java
@@ -54,7 +54,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing consumers of table. */
 public class ConsumersTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java
index 3bcba01404..83bfbe831e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java
@@ -82,7 +82,7 @@ import java.util.function.Function;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing files of a snapshot in specific table. */
 public class FilesTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java
index d1e3307b9b..fe5224b139 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java
@@ -26,6 +26,7 @@ import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.data.GenericRow;
 import org.apache.paimon.data.InternalRow;
 import org.apache.paimon.disk.IOManager;
+import org.apache.paimon.format.FileFormat;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.manifest.ManifestFileMeta;
 import org.apache.paimon.manifest.ManifestList;
@@ -60,7 +61,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing committing snapshots of table. */
 public class ManifestsTable implements ReadonlyTable {
@@ -238,7 +239,7 @@ public class ManifestsTable implements ReadonlyTable {
         ManifestList manifestList =
                 new ManifestList.Factory(
                                 dataTable.fileIO(),
-                                options.manifestFormat(),
+                                FileFormat.manifestFormat(options),
                                 options.manifestCompression(),
                                 fileStorePathFactory,
                                 null)
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java
index c18689cde9..d9de8bba19 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java
@@ -49,7 +49,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.apache.paimon.utils.SerializationUtils.newStringType;
 
 /** A {@link Table} for showing options of table. */
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java
index b1f2692dee..07efaf85a5 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java
@@ -59,7 +59,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing partitions info. */
 public class PartitionsTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java
index e4d82b16f8..601444e29a 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java
@@ -50,7 +50,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /**
  * A {@link Table} optimized for reading by avoiding merging files.
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java
index aa54b910c5..659f2355e7 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java
@@ -72,7 +72,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing schemas of table. */
 public class SchemasTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java
index 813ed8f234..df90c5060e 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java
@@ -72,7 +72,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing committing snapshots of table. */
 public class SnapshotsTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java
index d3c499a056..437be76b56 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java
@@ -55,7 +55,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing statistic of table. */
 public class StatisticTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/TableIndexesTable.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/system/TableIndexesTable.java
index b63cd604a7..d49efb41d5 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/TableIndexesTable.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/TableIndexesTable.java
@@ -63,7 +63,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.apache.paimon.utils.SerializationUtils.newStringType;
 
 /** A {@link Table} for showing indexes. */
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java
index 92fcfc4492..236fbe3110 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java
@@ -69,7 +69,7 @@ import java.util.Objects;
 import java.util.Optional;
 import java.util.TreeMap;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 
 /** A {@link Table} for showing tags of table. */
 public class TagsTable implements ReadonlyTable {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java 
b/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java
index b14e551d27..fc3defa8f3 100644
--- a/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java
+++ b/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java
@@ -24,13 +24,13 @@ import javax.annotation.Nullable;
 
 import java.util.List;
 
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 
 /** Manager for {@code Branch}. */
 public interface BranchManager {
 
     String BRANCH_PREFIX = "branch-";
-    String DEFAULT_MAIN_BRANCH = "main";
 
     void createBranch(String branchName);
 
diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java 
b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java
index 4713703bbd..cbebb76ed9 100644
--- a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java
+++ b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java
@@ -48,7 +48,7 @@ import java.util.TreeMap;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.BranchManager.branchPath;
 import static org.apache.paimon.utils.FileUtils.listVersionedFileStatus;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
diff --git a/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java 
b/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java
index daeb299811..536515ba51 100644
--- a/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java
@@ -106,7 +106,7 @@ public class FileFormatTest {
         tableOptions.set(CoreOptions.FILE_FORMAT, identifier);
         tableOptions.set(CoreOptions.READ_BATCH_SIZE, 1024);
         tableOptions.setString(identifier.toLowerCase() + ".hello", "world");
-        FileFormat fileFormat = CoreOptions.createFileFormat(tableOptions, 
CoreOptions.FILE_FORMAT);
+        FileFormat fileFormat = FileFormat.fromIdentifier(identifier, 
tableOptions);
         assertThat(fileFormat instanceof OrcFileFormat).isTrue();
 
         OrcFileFormat orcFileFormat = (OrcFileFormat) fileFormat;
@@ -134,6 +134,6 @@ public class FileFormatTest {
         Options tableOptions = new Options();
         tableOptions.set(CoreOptions.FILE_FORMAT, 
CoreOptions.FILE_FORMAT_AVRO);
         tableOptions.setString("avro.codec", codec);
-        return CoreOptions.createFileFormat(tableOptions, 
CoreOptions.FILE_FORMAT);
+        return FileFormat.fromIdentifier(CoreOptions.FILE_FORMAT_AVRO, 
tableOptions);
     }
 }
diff --git a/paimon-core/src/test/java/org/apache/paimon/SnapshotTest.java 
b/paimon-core/src/test/java/org/apache/paimon/SnapshotTest.java
index 2de9eef913..0cd75f3528 100644
--- a/paimon-core/src/test/java/org/apache/paimon/SnapshotTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/SnapshotTest.java
@@ -25,7 +25,7 @@ import org.apache.paimon.utils.SnapshotManager;
 
 import org.junit.jupiter.api.Test;
 
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Test for snapshots. */
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/manifest/IndexManifestFileHandlerTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/manifest/IndexManifestFileHandlerTest.java
index 78d9ba17c5..c2c5baba0a 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/manifest/IndexManifestFileHandlerTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/manifest/IndexManifestFileHandlerTest.java
@@ -20,6 +20,7 @@ package org.apache.paimon.manifest;
 
 import org.apache.paimon.TestAppendFileStore;
 import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.format.FileFormat;
 import org.apache.paimon.table.BucketMode;
 
 import org.junit.jupiter.api.Test;
@@ -45,7 +46,7 @@ public class IndexManifestFileHandlerTest {
         IndexManifestFile indexManifestFile =
                 new IndexManifestFile.Factory(
                                 fileStore.fileIO(),
-                                fileStore.options().manifestFormat(),
+                                FileFormat.manifestFormat(fileStore.options()),
                                 "zstd",
                                 fileStore.pathFactory(),
                                 null)
@@ -80,7 +81,7 @@ public class IndexManifestFileHandlerTest {
         IndexManifestFile indexManifestFile =
                 new IndexManifestFile.Factory(
                                 fileStore.fileIO(),
-                                fileStore.options().manifestFormat(),
+                                FileFormat.manifestFormat(fileStore.options()),
                                 "zstd",
                                 fileStore.pathFactory(),
                                 null)
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/SimpleTableTestBase.java 
b/paimon-core/src/test/java/org/apache/paimon/table/SimpleTableTestBase.java
index 61166e098c..d8e89c30f6 100644
--- a/paimon-core/src/test/java/org/apache/paimon/table/SimpleTableTestBase.java
+++ b/paimon-core/src/test/java/org/apache/paimon/table/SimpleTableTestBase.java
@@ -110,6 +110,7 @@ import static 
org.apache.paimon.CoreOptions.SNAPSHOT_NUM_RETAINED_MAX;
 import static org.apache.paimon.CoreOptions.SNAPSHOT_NUM_RETAINED_MIN;
 import static org.apache.paimon.CoreOptions.WRITE_ONLY;
 import static org.apache.paimon.SnapshotTest.newSnapshotManager;
+import static org.apache.paimon.format.FileFormat.fileFormat;
 import static 
org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches;
 import static org.apache.paimon.utils.HintFileUtils.EARLIEST;
 import static org.apache.paimon.utils.HintFileUtils.LATEST;
@@ -449,7 +450,7 @@ public abstract class SimpleTableTestBase {
     @Test
     public void testReadFilter() throws Exception {
         FileStoreTable table = createFileStoreTable();
-        if 
(table.coreOptions().fileFormat().getFormatIdentifier().equals("parquet")) {
+        if 
(fileFormat(table.coreOptions()).getFormatIdentifier().equals("parquet")) {
             // TODO support parquet reader filter push down
             return;
         }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/AggregationFieldsTableTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/table/system/AggregationFieldsTableTest.java
index 0cb6ef093a..a633f015e8 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/AggregationFieldsTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/AggregationFieldsTableTest.java
@@ -43,8 +43,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_BRANCH_PREFIX;
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_BRANCH_PREFIX;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static 
org.apache.paimon.table.system.AggregationFieldsTable.extractFieldMultimap;
 import static org.assertj.core.api.Assertions.assertThat;
 
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/AuditLogTableTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/table/system/AuditLogTableTest.java
index ebefa2af65..41facd1d83 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/AuditLogTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/AuditLogTableTest.java
@@ -19,7 +19,6 @@
 package org.apache.paimon.table.system;
 
 import org.apache.paimon.CoreOptions;
-import org.apache.paimon.catalog.Catalog;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.data.GenericRow;
@@ -43,6 +42,7 @@ import org.junit.jupiter.api.Test;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Unit tests for {@link AuditLogTable}. */
@@ -72,7 +72,7 @@ public class AuditLogTableTest extends TableTestBase {
         FileStoreTable table =
                 FileStoreTableFactory.create(LocalFileIO.create(), tablePath, 
tableSchema);
         Identifier filesTableId =
-                identifier(tableName + Catalog.SYSTEM_TABLE_SPLITTER + 
AuditLogTable.AUDIT_LOG);
+                identifier(tableName + SYSTEM_TABLE_SPLITTER + 
AuditLogTable.AUDIT_LOG);
         auditLogTable = (AuditLogTable) catalog.getTable(filesTableId);
 
         write(table, GenericRow.ofKind(RowKind.INSERT, 1, 1, 1));
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/BucketsTableTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/table/system/BucketsTableTest.java
index 1415baefca..3e7c72871d 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/BucketsTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/BucketsTableTest.java
@@ -36,7 +36,7 @@ import org.apache.paimon.types.DataTypes;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.apache.paimon.table.system.BucketsTable.BUCKETS;
 import static org.assertj.core.api.Assertions.assertThat;
 
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java 
b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java
index 7037460a4f..343cf236e3 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java
@@ -19,7 +19,6 @@
 package org.apache.paimon.table.system;
 
 import org.apache.paimon.CoreOptions;
-import org.apache.paimon.catalog.Catalog;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.data.GenericRow;
@@ -54,6 +53,7 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.apache.paimon.SnapshotTest.newSnapshotManager;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.apache.paimon.io.DataFileTestUtils.row;
 import static 
org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -89,8 +89,7 @@ public class FilesTableTest extends TableTestBase {
         table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, 
tableSchema);
         scan = table.store().newScan();
 
-        Identifier filesTableId =
-                identifier(tableName + Catalog.SYSTEM_TABLE_SPLITTER + 
FilesTable.FILES);
+        Identifier filesTableId = identifier(tableName + SYSTEM_TABLE_SPLITTER 
+ FilesTable.FILES);
         filesTable = (FilesTable) catalog.getTable(filesTableId);
         snapshotManager = newSnapshotManager(fileIO, tablePath);
 
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/OptionsTableTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/table/system/OptionsTableTest.java
index ce7f3a9b7e..a2f458fc4f 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/OptionsTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/OptionsTableTest.java
@@ -42,8 +42,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.paimon.catalog.Catalog.SYSTEM_BRANCH_PREFIX;
-import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_BRANCH_PREFIX;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Unit tests for {@link OptionsTableTest}. */
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java
index a286b187c6..e856e97244 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java
@@ -19,7 +19,6 @@
 package org.apache.paimon.table.system;
 
 import org.apache.paimon.CoreOptions;
-import org.apache.paimon.catalog.Catalog;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.data.GenericRow;
@@ -43,6 +42,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Unit tests for {@link PartitionsTable}. */
@@ -73,7 +73,7 @@ public class PartitionsTableTest extends TableTestBase {
         table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, 
tableSchema);
 
         Identifier filesTableId =
-                identifier(tableName + Catalog.SYSTEM_TABLE_SPLITTER + 
PartitionsTable.PARTITIONS);
+                identifier(tableName + SYSTEM_TABLE_SPLITTER + 
PartitionsTable.PARTITIONS);
         partitionsTable = (PartitionsTable) catalog.getTable(filesTableId);
 
         // snapshot 1: append
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/system/SnapshotsTableTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/table/system/SnapshotsTableTest.java
index a7055c5c5e..20efe55ac9 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/system/SnapshotsTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/system/SnapshotsTableTest.java
@@ -20,7 +20,6 @@ package org.apache.paimon.table.system;
 
 import org.apache.paimon.CoreOptions;
 import org.apache.paimon.Snapshot;
-import org.apache.paimon.catalog.Catalog;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.data.GenericRow;
@@ -49,6 +48,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.paimon.SnapshotTest.newSnapshotManager;
+import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for {@link SnapshotsTable}. */
@@ -79,7 +79,7 @@ public class SnapshotsTableTest extends TableTestBase {
                 FileStoreTableFactory.create(LocalFileIO.create(), tablePath, 
tableSchema);
 
         Identifier filesTableId =
-                identifier(tableName + Catalog.SYSTEM_TABLE_SPLITTER + 
SnapshotsTable.SNAPSHOTS);
+                identifier(tableName + SYSTEM_TABLE_SPLITTER + 
SnapshotsTable.SNAPSHOTS);
         snapshotsTable = (SnapshotsTable) catalog.getTable(filesTableId);
 
         // snapshot 1: append
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java 
b/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java
index ed5badb764..cf3d77feee 100644
--- a/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java
@@ -44,7 +44,7 @@ import java.util.stream.Collectors;
 
 import static org.apache.paimon.SnapshotTest.newChangelogManager;
 import static org.apache.paimon.SnapshotTest.newSnapshotManager;
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.fail;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
diff --git 
a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java
 
b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java
index 3935236be4..71792962b8 100644
--- 
a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java
+++ 
b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java
@@ -72,10 +72,9 @@ public class ConfigOptionsDocGenerator {
 
     static final OptionsClassLocation[] LOCATIONS =
             new OptionsClassLocation[] {
-                new OptionsClassLocation("paimon-common", 
"org.apache.paimon.options"),
-                new OptionsClassLocation("paimon-common", "org.apache.paimon"),
+                new OptionsClassLocation("paimon-api", 
"org.apache.paimon.options"),
+                new OptionsClassLocation("paimon-api", "org.apache.paimon"),
                 new OptionsClassLocation("paimon-core", 
"org.apache.paimon.lookup"),
-                new OptionsClassLocation("paimon-core", 
"org.apache.paimon.catalog"),
                 new OptionsClassLocation("paimon-core", 
"org.apache.paimon.jdbc"),
                 new OptionsClassLocation("paimon-core", 
"org.apache.paimon.table"),
                 new OptionsClassLocation("paimon-core", 
"org.apache.paimon.iceberg"),
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
index fd8b629507..dec278fb70 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
@@ -33,6 +33,7 @@ import org.apache.paimon.predicate.Predicate;
 import org.apache.paimon.table.BucketMode;
 import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.source.OutOfRangeException;
+import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.FileIOUtils;
 import org.apache.paimon.utils.Filter;
 import org.apache.paimon.utils.Preconditions;
@@ -117,19 +118,20 @@ public class FileStoreLookupFunction implements 
Serializable, Closeable {
         this.partitionLoader = DynamicPartitionLoader.of(table);
 
         // join keys are based on projection fields
+        RowType rowType = table.rowType();
         this.joinKeys =
                 Arrays.stream(joinKeyIndex)
-                        .mapToObj(i -> 
table.rowType().getFieldNames().get(projection[i]))
+                        .mapToObj(i -> 
rowType.getFieldNames().get(projection[i]))
                         .collect(Collectors.toList());
 
         this.projectFields =
                 Arrays.stream(projection)
-                        .mapToObj(i -> table.rowType().getFieldNames().get(i))
+                        .mapToObj(i -> rowType.getFieldNames().get(i))
                         .collect(Collectors.toList());
 
         this.projectFieldsGetters =
                 Arrays.stream(projection)
-                        .mapToObj(i -> table.rowType().fieldGetters()[i])
+                        .mapToObj(i -> 
InternalRow.createFieldGetter(rowType.getTypeAt(i), i))
                         .collect(Collectors.toList());
 
         // add primary keys
diff --git 
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java
 
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java
index 5245114e80..0d9f512fbe 100644
--- 
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java
+++ 
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java
@@ -33,7 +33,6 @@ import org.apache.paimon.schema.SchemaManager;
 import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.FileStoreTableFactory;
 import org.apache.paimon.utils.BlockingIterator;
-import org.apache.paimon.utils.BranchManager;
 import org.apache.paimon.utils.FailingFileIO;
 
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
@@ -85,6 +84,7 @@ import static org.apache.paimon.CoreOptions.BUCKET;
 import static org.apache.paimon.CoreOptions.BUCKET_KEY;
 import static org.apache.paimon.CoreOptions.FILE_FORMAT;
 import static org.apache.paimon.CoreOptions.PATH;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.flink.LogicalTypeConversion.toDataType;
 import static org.apache.paimon.utils.FailingFileIO.retryArtificialException;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -139,7 +139,7 @@ public class FileStoreITCase extends AbstractTestBase {
 
     @BeforeAll
     public static void before() {
-        branch = BranchManager.DEFAULT_MAIN_BRANCH;
+        branch = DEFAULT_MAIN_BRANCH;
     }
 
     @Parameters(name = "isBatch-{0}")
diff --git 
a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/VectorizedColumnReader.java
 
b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/VectorizedColumnReader.java
index 0f2713f58b..53b11138da 100644
--- 
a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/VectorizedColumnReader.java
+++ 
b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/VectorizedColumnReader.java
@@ -49,7 +49,6 @@ import org.apache.parquet.schema.PrimitiveType;
 
 import java.io.IOException;
 
-import static org.apache.paimon.types.DataTypeRoot.FLOAT;
 import static 
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
 
 /** Decoder to return values from a single column. */
diff --git 
a/paimon-format/src/test/java/org/apache/paimon/format/parquet/reader/FileTypeNotMatchReadTypeTest.java
 
b/paimon-format/src/test/java/org/apache/paimon/format/parquet/reader/FileTypeNotMatchReadTypeTest.java
index bae1c98a8c..7afcbacc9d 100644
--- 
a/paimon-format/src/test/java/org/apache/paimon/format/parquet/reader/FileTypeNotMatchReadTypeTest.java
+++ 
b/paimon-format/src/test/java/org/apache/paimon/format/parquet/reader/FileTypeNotMatchReadTypeTest.java
@@ -62,7 +62,6 @@ import java.util.Random;
 import static 
org.apache.paimon.format.parquet.ParquetSchemaConverter.MAP_KEY_NAME;
 import static 
org.apache.paimon.format.parquet.ParquetSchemaConverter.MAP_VALUE_NAME;
 import static 
org.apache.paimon.format.parquet.ParquetSchemaConverter.PAIMON_SCHEMA;
-import static 
org.apache.paimon.format.parquet.ParquetSchemaConverter.convertToParquetTypes;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Test field type not match correctly with read type. */
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 5321eb378f..728d5f176c 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
@@ -112,6 +112,7 @@ import static 
org.apache.paimon.catalog.CatalogUtils.checkNotSystemDatabase;
 import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemTable;
 import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase;
 import static 
org.apache.paimon.catalog.CatalogUtils.listPartitionsFromFileSystem;
+import static org.apache.paimon.catalog.Identifier.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.hive.HiveCatalogOptions.HADOOP_CONF_DIR;
 import static org.apache.paimon.hive.HiveCatalogOptions.HIVE_CONF_DIR;
 import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER;
@@ -123,7 +124,6 @@ import static 
org.apache.paimon.options.CatalogOptions.SYNC_ALL_PROPERTIES;
 import static org.apache.paimon.options.CatalogOptions.TABLE_TYPE;
 import static 
org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey;
 import static org.apache.paimon.table.FormatTableOptions.FIELD_DELIMITER;
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.HadoopUtils.addHadoopConfIfFound;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 import static org.apache.paimon.utils.StringUtils.isNullOrWhitespaceOnly;
diff --git a/pom.xml b/pom.xml
index 7e47a91170..17c59d64a9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -70,6 +70,7 @@ under the License.
         <module>paimon-arrow</module>
         <module>tools/ci/paimon-ci-tools</module>
         <module>paimon-hudi</module>
+        <module>paimon-api</module>
     </modules>
 
     <properties>


Reply via email to