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 bf58dddfb0 [rest] Fix GetTableResponse should return fields (#4769)
bf58dddfb0 is described below

commit bf58dddfb01b636186d87595841f518e5e65b073
Author: Jingsong Lee <[email protected]>
AuthorDate: Wed Dec 25 10:10:08 2024 +0800

    [rest] Fix GetTableResponse should return fields (#4769)
---
 .../org/apache/paimon/catalog/AbstractCatalog.java |  59 +++--
 .../org/apache/paimon/catalog/CatalogUtils.java    |  46 +---
 .../apache/paimon/catalog/FileSystemCatalog.java   |   5 +-
 .../java/org/apache/paimon/rest/RESTCatalog.java   | 114 +++-------
 .../org/apache/paimon/rest/RESTObjectMapper.java   |   5 +-
 .../paimon/rest/requests/AlterDatabaseRequest.java |   6 +-
 .../rest/requests/CreateDatabaseRequest.java       |   6 +-
 .../paimon/rest/requests/CreateTableRequest.java   |   6 +-
 ...atabaseRequest.java => RenameTableRequest.java} |  37 ++--
 .../apache/paimon/rest/requests/SchemaChanges.java | 238 ---------------------
 .../paimon/rest/requests/UpdateTableRequest.java   |  57 -----
 .../rest/responses/AlterDatabaseResponse.java      |   8 +-
 .../paimon/rest/responses/ConfigResponse.java      |   6 +-
 .../rest/responses/CreateDatabaseResponse.java     |   6 +-
 .../paimon/rest/responses/ErrorResponse.java       |   2 +
 .../paimon/rest/responses/GetDatabaseResponse.java |   2 +
 .../paimon/rest/responses/GetTableResponse.java    |  37 ++--
 .../rest/responses/ListDatabasesResponse.java      |   5 +-
 .../paimon/rest/responses/ListTablesResponse.java  |   5 +-
 .../main/java/org/apache/paimon/schema/Schema.java |   9 +-
 .../org/apache/paimon/schema/SchemaChange.java     |  28 +--
 .../org/apache/paimon/schema/SchemaManager.java    |  16 +-
 .../java/org/apache/paimon/schema/TableSchema.java |  17 ++
 .../paimon/table/system/SystemTableLoader.java     |  20 --
 .../org/apache/paimon/rest/MockRESTMessage.java    |  20 +-
 .../org/apache/paimon/rest/RESTCatalogTest.java    |  36 +---
 .../apache/paimon/rest/RESTObjectMapperTest.java   |  13 +-
 .../java/org/apache/paimon/hive/HiveCatalog.java   |   4 +-
 .../org/apache/paimon/hive/PaimonMetaHook.java     |   4 +-
 .../org/apache/paimon/hive/CreateTableITCase.java  |  20 +-
 .../org/apache/paimon/hive/HiveLocationTest.java   |   4 +-
 .../org/apache/paimon/hive/HiveReadITCaseBase.java |   6 +-
 paimon-open-api/rest-catalog-open-api.yaml         | 156 +-------------
 .../paimon/open/api/RESTCatalogController.java     |  40 ++--
 34 files changed, 251 insertions(+), 792 deletions(-)

diff --git 
a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java 
b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java
index ef6c0e3348..c63d92a144 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
@@ -20,6 +20,7 @@ package org.apache.paimon.catalog;
 
 import org.apache.paimon.CoreOptions;
 import org.apache.paimon.TableType;
+import org.apache.paimon.factories.FactoryUtil;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.FileStatus;
 import org.apache.paimon.fs.Path;
@@ -39,6 +40,8 @@ import org.apache.paimon.table.FormatTable;
 import org.apache.paimon.table.Table;
 import org.apache.paimon.table.object.ObjectTable;
 import org.apache.paimon.table.sink.BatchWriteBuilder;
+import org.apache.paimon.table.system.AllTableOptionsTable;
+import org.apache.paimon.table.system.CatalogOptionsTable;
 import org.apache.paimon.table.system.SystemTableLoader;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.Preconditions;
@@ -62,7 +65,10 @@ import static 
org.apache.paimon.catalog.CatalogUtils.checkNotBranch;
 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.lockFactory;
+import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED;
+import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE;
+import static 
org.apache.paimon.table.system.AllTableOptionsTable.ALL_TABLE_OPTIONS;
+import static 
org.apache.paimon.table.system.CatalogOptionsTable.CATALOG_OPTIONS;
 import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
 import static org.apache.paimon.utils.Preconditions.checkArgument;
 import static org.apache.paimon.utils.Preconditions.checkNotNull;
@@ -96,16 +102,31 @@ public abstract class AbstractCatalog implements Catalog {
         return fileIO;
     }
 
+    public Optional<CatalogLockFactory> lockFactory() {
+        if (!lockEnabled()) {
+            return Optional.empty();
+        }
+
+        String lock = catalogOptions.get(LOCK_TYPE);
+        if (lock == null) {
+            return defaultLockFactory();
+        }
+
+        return Optional.of(
+                FactoryUtil.discoverFactory(
+                        AbstractCatalog.class.getClassLoader(), 
CatalogLockFactory.class, lock));
+    }
+
     public Optional<CatalogLockFactory> defaultLockFactory() {
         return Optional.empty();
     }
 
     public Optional<CatalogLockContext> lockContext() {
-        return CatalogUtils.lockContext(catalogOptions);
+        return Optional.of(CatalogLockContext.fromOptions(catalogOptions));
     }
 
     protected boolean lockEnabled() {
-        return CatalogUtils.lockEnabled(catalogOptions, fileIO);
+        return 
catalogOptions.getOptional(LOCK_ENABLED).orElse(fileIO.isObjectStore());
     }
 
     protected boolean allowCustomTablePath() {
@@ -369,13 +390,14 @@ public abstract class AbstractCatalog implements Catalog {
     public Table getTable(Identifier identifier) throws TableNotExistException 
{
         if (isSystemDatabase(identifier.getDatabaseName())) {
             String tableName = identifier.getTableName();
-            Table table =
-                    SystemTableLoader.loadGlobal(
-                            tableName, fileIO, this::allTablePaths, 
catalogOptions);
-            if (table == null) {
-                throw new TableNotExistException(identifier);
+            switch (tableName.toLowerCase()) {
+                case ALL_TABLE_OPTIONS:
+                    return new AllTableOptionsTable(fileIO, allTablePaths());
+                case CATALOG_OPTIONS:
+                    return new CatalogOptionsTable(catalogOptions);
+                default:
+                    throw new TableNotExistException(identifier);
             }
-            return table;
         } else if (identifier.isSystemTable()) {
             Table originTable =
                     getDataOrFormatTable(
@@ -384,7 +406,7 @@ public abstract class AbstractCatalog implements Catalog {
                                     identifier.getTableName(),
                                     identifier.getBranchName(),
                                     null));
-            return CatalogUtils.getSystemTable(identifier, originTable);
+            return CatalogUtils.createSystemTable(identifier, originTable);
         } else {
             return getDataOrFormatTable(identifier);
         }
@@ -402,8 +424,7 @@ public abstract class AbstractCatalog implements Catalog {
                                 identifier,
                                 tableMeta.uuid,
                                 Lock.factory(
-                                        lockFactory(catalogOptions, fileIO(), 
defaultLockFactory())
-                                                .orElse(null),
+                                        lockFactory().orElse(null),
                                         lockContext().orElse(null),
                                         identifier),
                                 
metastoreClientFactory(identifier).orElse(null)));
@@ -447,7 +468,7 @@ public abstract class AbstractCatalog implements Catalog {
      * @return The warehouse path for the database
      */
     public Path newDatabasePath(String database) {
-        return CatalogUtils.newDatabasePath(warehouse(), database);
+        return newDatabasePath(warehouse(), database);
     }
 
     public Map<String, Map<String, Path>> allTablePaths() {
@@ -490,6 +511,18 @@ public abstract class AbstractCatalog implements Catalog {
         }
     }
 
+    public static Path newTableLocation(String warehouse, Identifier 
identifier) {
+        checkNotBranch(identifier, "newTableLocation");
+        checkNotSystemTable(identifier, "newTableLocation");
+        return new Path(
+                newDatabasePath(warehouse, identifier.getDatabaseName()),
+                identifier.getTableName());
+    }
+
+    public static Path newDatabasePath(String warehouse, String database) {
+        return new Path(warehouse, database + DB_SUFFIX);
+    }
+
     private void copyTableDefaultOptions(Map<String, String> options) {
         tableDefaultOptions.forEach(options::putIfAbsent);
     }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java 
b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java
index 826e2c0847..d454547e31 100644
--- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java
+++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java
@@ -18,10 +18,7 @@
 
 package org.apache.paimon.catalog;
 
-import org.apache.paimon.factories.FactoryUtil;
-import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
-import org.apache.paimon.options.Options;
 import org.apache.paimon.schema.SchemaManager;
 import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.Table;
@@ -29,13 +26,9 @@ import org.apache.paimon.table.system.SystemTableLoader;
 import org.apache.paimon.utils.Preconditions;
 
 import java.util.Map;
-import java.util.Optional;
 
-import static org.apache.paimon.catalog.Catalog.DB_SUFFIX;
 import static org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME;
 import static org.apache.paimon.catalog.Catalog.TABLE_DEFAULT_OPTION_PREFIX;
-import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED;
-import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE;
 import static 
org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey;
 
 /** Utils for {@link Catalog}. */
@@ -97,18 +90,6 @@ public class CatalogUtils {
         }
     }
 
-    public static Path newDatabasePath(String warehouse, String database) {
-        return new Path(warehouse, database + DB_SUFFIX);
-    }
-
-    public static Path newTableLocation(String warehouse, Identifier 
identifier) {
-        checkNotBranch(identifier, "newTableLocation");
-        checkNotSystemTable(identifier, "newTableLocation");
-        return new Path(
-                newDatabasePath(warehouse, identifier.getDatabaseName()),
-                identifier.getTableName());
-    }
-
     public static void checkNotBranch(Identifier identifier, String method) {
         if (identifier.getBranchName() != null) {
             throw new IllegalArgumentException(
@@ -119,32 +100,7 @@ public class CatalogUtils {
         }
     }
 
-    public static Optional<CatalogLockFactory> lockFactory(
-            Options options, FileIO fileIO, Optional<CatalogLockFactory> 
defaultLockFactoryOpt) {
-        boolean lockEnabled = lockEnabled(options, fileIO);
-        if (!lockEnabled) {
-            return Optional.empty();
-        }
-
-        String lock = options.get(LOCK_TYPE);
-        if (lock == null) {
-            return defaultLockFactoryOpt;
-        }
-
-        return Optional.of(
-                FactoryUtil.discoverFactory(
-                        AbstractCatalog.class.getClassLoader(), 
CatalogLockFactory.class, lock));
-    }
-
-    public static Optional<CatalogLockContext> lockContext(Options options) {
-        return Optional.of(CatalogLockContext.fromOptions(options));
-    }
-
-    public static boolean lockEnabled(Options options, FileIO fileIO) {
-        return options.getOptional(LOCK_ENABLED).orElse(fileIO != null && 
fileIO.isObjectStore());
-    }
-
-    public static Table getSystemTable(Identifier identifier, Table 
originTable)
+    public static Table createSystemTable(Identifier identifier, Table 
originTable)
             throws Catalog.TableNotExistException {
         if (!(originTable instanceof FileStoreTable)) {
             throw new UnsupportedOperationException(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java 
b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java
index 577dd9674e..cb0c358259 100644
--- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java
+++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java
@@ -34,7 +34,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
-import static org.apache.paimon.catalog.CatalogUtils.lockFactory;
 import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE;
 
 /** A catalog implementation for {@link FileIO}. */
@@ -124,9 +123,7 @@ public class FileSystemCatalog extends AbstractCatalog {
     private SchemaManager schemaManager(Identifier identifier) {
         Path path = getTableLocation(identifier);
         CatalogLock catalogLock =
-                lockFactory(catalogOptions, fileIO(), defaultLockFactory())
-                        .map(fac -> fac.createLock(assertGetLockContext()))
-                        .orElse(null);
+                lockFactory().map(fac -> 
fac.createLock(assertGetLockContext())).orElse(null);
         return new SchemaManager(fileIO, path, 
identifier.getBranchNameOrDefault())
                 .withLock(catalogLock == null ? null : 
Lock.fromCatalog(catalogLock, identifier));
     }
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 4836710601..b98b0e7078 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
@@ -41,8 +41,7 @@ import 
org.apache.paimon.rest.exceptions.NoSuchResourceException;
 import org.apache.paimon.rest.requests.AlterDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateTableRequest;
-import org.apache.paimon.rest.requests.SchemaChanges;
-import org.apache.paimon.rest.requests.UpdateTableRequest;
+import org.apache.paimon.rest.requests.RenameTableRequest;
 import org.apache.paimon.rest.responses.AlterDatabaseResponse;
 import org.apache.paimon.rest.responses.ConfigResponse;
 import org.apache.paimon.rest.responses.CreateDatabaseResponse;
@@ -58,7 +57,6 @@ import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.FileStoreTableFactory;
 import org.apache.paimon.table.Table;
 import org.apache.paimon.table.object.ObjectTable;
-import org.apache.paimon.table.system.SystemTableLoader;
 import org.apache.paimon.utils.Pair;
 import org.apache.paimon.utils.Preconditions;
 
@@ -69,21 +67,17 @@ import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMap
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.time.Duration;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.function.Supplier;
 
 import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemDatabase;
 import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase;
-import static org.apache.paimon.catalog.CatalogUtils.lockContext;
-import static org.apache.paimon.catalog.CatalogUtils.lockFactory;
-import static org.apache.paimon.catalog.CatalogUtils.newTableLocation;
 import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE;
 import static org.apache.paimon.utils.Preconditions.checkNotNull;
 import static 
org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool;
@@ -96,10 +90,9 @@ public class RESTCatalog implements Catalog {
 
     private final RESTClient client;
     private final ResourcePaths resourcePaths;
-    private final Map<String, String> baseHeader;
     private final AuthSession catalogAuth;
     private final CatalogContext context;
-    private final Optional<FileIO> fileIOOptional;
+    private final FileIO fileIO;
 
     private volatile ScheduledExecutorService refreshExecutor = null;
 
@@ -123,17 +116,17 @@ public class RESTCatalog implements Catalog {
                         threadPoolSize,
                         DefaultErrorHandler.getInstance());
         this.client = new HttpClient(httpClientOptions);
-        this.baseHeader = configHeaders(catalogOptions.toMap());
+        Map<String, String> baseHeader = configHeaders(catalogOptions.toMap());
         CredentialsProvider credentialsProvider =
                 CredentialsProviderFactory.createCredentialsProvider(
                         catalogOptions, RESTCatalog.class.getClassLoader());
         if (credentialsProvider.keepRefreshed()) {
             this.catalogAuth =
                     AuthSession.fromRefreshCredentialsProvider(
-                            tokenRefreshExecutor(), this.baseHeader, 
credentialsProvider);
+                            tokenRefreshExecutor(), baseHeader, 
credentialsProvider);
 
         } else {
-            this.catalogAuth = new AuthSession(this.baseHeader, 
credentialsProvider);
+            this.catalogAuth = new AuthSession(baseHeader, 
credentialsProvider);
         }
         Map<String, String> initHeaders =
                 RESTUtil.merge(
@@ -144,21 +137,21 @@ public class RESTCatalog implements Catalog {
                         options, catalogContext.preferIO(), 
catalogContext.fallbackIO());
         this.resourcePaths =
                 
ResourcePaths.forCatalogProperties(options.get(RESTCatalogInternalOptions.PREFIX));
-        this.fileIOOptional = getFileIOFromOptions(context);
+        this.fileIO = getFileIOFromOptions(context);
     }
 
-    private static Optional<FileIO> getFileIOFromOptions(CatalogContext 
context) {
+    private static FileIO getFileIOFromOptions(CatalogContext context) {
         try {
             Options options = context.options();
             String warehouseStr = options.get(CatalogOptions.WAREHOUSE);
             Path warehousePath = new Path(warehouseStr);
             CatalogContext contextWithNewOptions =
                     CatalogContext.create(options, context.preferIO(), 
context.fallbackIO());
-            return Optional.of(FileIO.get(warehousePath, 
contextWithNewOptions));
-        } catch (Exception ignore) {
+            return FileIO.get(warehousePath, contextWithNewOptions);
+        } catch (IOException e) {
             LOG.warn("Can not get FileIO from options.");
+            throw new RuntimeException(e);
         }
-        return Optional.empty();
     }
 
     @Override
@@ -173,10 +166,7 @@ public class RESTCatalog implements Catalog {
 
     @Override
     public FileIO fileIO() {
-        if (this.fileIOOptional.isPresent()) {
-            return this.fileIOOptional.get();
-        }
-        throw new RuntimeException("FileIO is not configured.");
+        return fileIO;
     }
 
     @Override
@@ -283,7 +273,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public Table getTable(Identifier identifier) throws TableNotExistException 
{
         if (SYSTEM_DATABASE_NAME.equals(identifier.getDatabaseName())) {
-            return getAllInSystemDatabase(identifier);
+            throw new UnsupportedOperationException("TODO support global 
system tables.");
         } else if (identifier.isSystemTable()) {
             return getSystemTable(identifier);
         } else {
@@ -312,7 +302,7 @@ public class RESTCatalog implements Catalog {
     public void renameTable(Identifier fromTable, Identifier toTable, boolean 
ignoreIfNotExists)
             throws TableNotExistException, TableAlreadyExistException {
         try {
-            updateTable(fromTable, toTable, new ArrayList<>());
+            renameTable(fromTable, toTable);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 throw new TableNotExistException(fromTable);
@@ -328,15 +318,7 @@ public class RESTCatalog implements Catalog {
     public void alterTable(
             Identifier identifier, List<SchemaChange> changes, boolean 
ignoreIfNotExists)
             throws TableNotExistException, ColumnAlreadyExistException, 
ColumnNotExistException {
-        try {
-            updateTable(identifier, identifier, changes);
-        } catch (NoSuchResourceException e) {
-            if (!ignoreIfNotExists) {
-                throw new TableNotExistException(identifier);
-            }
-        } catch (ForbiddenException e) {
-            throw new TableNoPermissionException(identifier, e);
-        }
+        throw new UnsupportedOperationException("TODO");
     }
 
     @Override
@@ -395,10 +377,8 @@ public class RESTCatalog implements Catalog {
     }
 
     @VisibleForTesting
-    void updateTable(
-            Identifier fromTable, Identifier newTableIdentifier, 
List<SchemaChange> changes) {
-        UpdateTableRequest request =
-                new UpdateTableRequest(newTableIdentifier, new 
SchemaChanges(changes));
+    void renameTable(Identifier fromTable, Identifier newIdentifier) {
+        RenameTableRequest request = new RenameTableRequest(newIdentifier);
         client.post(
                 resourcePaths.table(fromTable.getDatabaseName(), 
fromTable.getTableName()),
                 request,
@@ -409,19 +389,13 @@ public class RESTCatalog implements Catalog {
     @VisibleForTesting
     Table getDataOrFormatTable(Identifier identifier) throws 
TableNotExistException {
         Preconditions.checkArgument(identifier.getSystemTableName() == null);
-        TableSchema tableSchema = getDataTableSchema(identifier);
-        Lock.Factory lockFactory =
-                Lock.factory(
-                        lockFactory(context.options(), fileIO(), 
Optional.empty()).orElse(null),
-                        lockContext(context.options()).orElse(null),
-                        identifier);
-        // MetastoreClient is not used in RESTCatalog so null is ok.
+        GetTableResponse response = getTableResponse(identifier);
         FileStoreTable table =
                 FileStoreTableFactory.create(
                         fileIO(),
-                        newTableLocation(warehouse(), identifier),
-                        tableSchema,
-                        new CatalogEnvironment(identifier, null, lockFactory, 
null));
+                        new Path(response.getPath()),
+                        TableSchema.create(response.getSchemaId(), 
response.getSchema()),
+                        new CatalogEnvironment(identifier, null, 
Lock.emptyFactory(), null));
         CoreOptions options = table.coreOptions();
         if (options.type() == TableType.OBJECT_TABLE) {
             String objectLocation = options.objectLocation();
@@ -436,15 +410,13 @@ public class RESTCatalog implements Catalog {
         return table;
     }
 
-    protected TableSchema getDataTableSchema(Identifier identifier) throws 
TableNotExistException {
+    protected GetTableResponse getTableResponse(Identifier identifier)
+            throws TableNotExistException {
         try {
-            GetTableResponse response =
-                    client.get(
-                            resourcePaths.table(
-                                    identifier.getDatabaseName(), 
identifier.getTableName()),
-                            GetTableResponse.class,
-                            headers());
-            return response.getSchema();
+            return client.get(
+                    resourcePaths.table(identifier.getDatabaseName(), 
identifier.getTableName()),
+                    GetTableResponse.class,
+                    headers());
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -460,36 +432,6 @@ public class RESTCatalog implements Catalog {
         return catalogAuth.getHeaders();
     }
 
-    private Table getAllInSystemDatabase(Identifier identifier) throws 
TableNotExistException {
-        String tableName = identifier.getTableName();
-        Supplier<Map<String, Map<String, Path>>> getAllTablePathsFunction =
-                () -> {
-                    try {
-                        Map<String, Map<String, Path>> allPaths = new 
HashMap<>();
-                        for (String database : listDatabases()) {
-                            Map<String, Path> tableMap =
-                                    allPaths.computeIfAbsent(database, d -> 
new HashMap<>());
-                            for (String table : listTables(database)) {
-                                Path tableLocation =
-                                        newTableLocation(
-                                                warehouse(), 
Identifier.create(database, table));
-                                tableMap.put(table, tableLocation);
-                            }
-                        }
-                        return allPaths;
-                    } catch (DatabaseNotExistException e) {
-                        throw new RuntimeException("Database is deleted while 
listing", e);
-                    }
-                };
-        Table table =
-                SystemTableLoader.loadGlobal(
-                        tableName, fileIO(), getAllTablePathsFunction, 
context.options());
-        if (table == null) {
-            throw new TableNotExistException(identifier);
-        }
-        return table;
-    }
-
     private Table getSystemTable(Identifier identifier) throws 
TableNotExistException {
         Table originTable =
                 getDataOrFormatTable(
@@ -498,7 +440,7 @@ public class RESTCatalog implements Catalog {
                                 identifier.getTableName(),
                                 identifier.getBranchName(),
                                 null));
-        return CatalogUtils.getSystemTable(identifier, originTable);
+        return CatalogUtils.createSystemTable(identifier, originTable);
     }
 
     private ScheduledExecutorService tokenRefreshExecutor() {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java 
b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java
index ce20158d0b..11314bb153 100644
--- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java
+++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java
@@ -18,8 +18,6 @@
 
 package org.apache.paimon.rest;
 
-import org.apache.paimon.schema.SchemaSerializer;
-import org.apache.paimon.schema.TableSchema;
 import org.apache.paimon.types.DataField;
 import org.apache.paimon.types.DataType;
 import org.apache.paimon.types.DataTypeJsonParser;
@@ -35,6 +33,7 @@ import static 
org.apache.paimon.utils.JsonSerdeUtil.registerJsonObjects;
 
 /** Object mapper for REST request and response. */
 public class RESTObjectMapper {
+
     public static ObjectMapper create() {
         ObjectMapper mapper = new ObjectMapper();
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, 
false);
@@ -46,8 +45,6 @@ public class RESTObjectMapper {
 
     public static Module createPaimonRestJacksonModule() {
         SimpleModule module = new SimpleModule("Paimon_REST");
-        registerJsonObjects(
-                module, TableSchema.class, SchemaSerializer.INSTANCE, 
SchemaSerializer.INSTANCE);
         registerJsonObjects(
                 module,
                 DataField.class,
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java
index c1330142bb..4c1ea03cf2 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java
@@ -22,22 +22,24 @@ import org.apache.paimon.rest.RESTRequest;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.List;
 import java.util.Map;
 
 /** Request for altering database. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class AlterDatabaseRequest implements RESTRequest {
 
     private static final String FIELD_REMOVALS = "removals";
     private static final String FIELD_UPDATES = "updates";
 
     @JsonProperty(FIELD_REMOVALS)
-    private List<String> removals;
+    private final List<String> removals;
 
     @JsonProperty(FIELD_UPDATES)
-    private Map<String, String> updates;
+    private final Map<String, String> updates;
 
     @JsonCreator
     public AlterDatabaseRequest(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
index 07e5cf2462..29f607786e 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
@@ -22,21 +22,23 @@ import org.apache.paimon.rest.RESTRequest;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.Map;
 
 /** Request for creating database. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class CreateDatabaseRequest implements RESTRequest {
 
     private static final String FIELD_NAME = "name";
     private static final String FIELD_OPTIONS = "options";
 
     @JsonProperty(FIELD_NAME)
-    private String name;
+    private final String name;
 
     @JsonProperty(FIELD_OPTIONS)
-    private Map<String, String> options;
+    private final Map<String, String> options;
 
     @JsonCreator
     public CreateDatabaseRequest(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
index 794dd33c46..c55dc8c594 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java
@@ -24,19 +24,21 @@ import org.apache.paimon.schema.Schema;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 /** Request for creating table. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class CreateTableRequest implements RESTRequest {
 
     private static final String FIELD_IDENTIFIER = "identifier";
     private static final String FIELD_SCHEMA = "schema";
 
     @JsonProperty(FIELD_IDENTIFIER)
-    private Identifier identifier;
+    private final Identifier identifier;
 
     @JsonProperty(FIELD_SCHEMA)
-    private Schema schema;
+    private final Schema schema;
 
     @JsonCreator
     public CreateTableRequest(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/RenameTableRequest.java
similarity index 60%
copy from 
paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
copy to 
paimon-core/src/main/java/org/apache/paimon/rest/requests/RenameTableRequest.java
index 07e5cf2462..fd2eb4f951 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/RenameTableRequest.java
@@ -18,41 +18,30 @@
 
 package org.apache.paimon.rest.requests;
 
+import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.rest.RESTRequest;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
-import java.util.Map;
+/** Request for renaming table. */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class RenameTableRequest implements RESTRequest {
 
-/** Request for creating database. */
-public class CreateDatabaseRequest implements RESTRequest {
+    private static final String FIELD_NEW_IDENTIFIER_NAME = "newIdentifier";
 
-    private static final String FIELD_NAME = "name";
-    private static final String FIELD_OPTIONS = "options";
-
-    @JsonProperty(FIELD_NAME)
-    private String name;
-
-    @JsonProperty(FIELD_OPTIONS)
-    private Map<String, String> options;
+    @JsonProperty(FIELD_NEW_IDENTIFIER_NAME)
+    private final Identifier newIdentifier;
 
     @JsonCreator
-    public CreateDatabaseRequest(
-            @JsonProperty(FIELD_NAME) String name,
-            @JsonProperty(FIELD_OPTIONS) Map<String, String> options) {
-        this.name = name;
-        this.options = options;
-    }
-
-    @JsonGetter(FIELD_NAME)
-    public String getName() {
-        return name;
+    public RenameTableRequest(@JsonProperty(FIELD_NEW_IDENTIFIER_NAME) 
Identifier newIdentifier) {
+        this.newIdentifier = newIdentifier;
     }
 
-    @JsonGetter(FIELD_OPTIONS)
-    public Map<String, String> getOptions() {
-        return options;
+    @JsonGetter(FIELD_NEW_IDENTIFIER_NAME)
+    public Identifier getNewIdentifier() {
+        return newIdentifier;
     }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java
deleted file mode 100644
index 1c3e419f13..0000000000
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.paimon.rest.requests;
-
-import org.apache.paimon.schema.SchemaChange;
-
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
-/** Schema changes to serialize List of SchemaChange . */
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class SchemaChanges {
-
-    private static final String FIELD_SET_OPTIONS = "set-options";
-    private static final String FIELD_REMOVE_OPTIONS = "remove-options";
-    private static final String FIELD_COMMENT = "comment";
-    private static final String FIELD_ADD_COLUMNS = "add-columns";
-    private static final String FIELD_RENAME_COLUMNS = "rename-columns";
-    private static final String FIELD_DROP_COLUMNS = "drop-columns";
-    private static final String FIELD_UPDATE_COLUMN_TYPES = 
"update-column-types";
-    private static final String FIELD_UPDATE_COLUMN_NULLABILITIES = 
"update-column-nullabilities";
-    private static final String FIELD_UPDATE_COLUMN_COMMENTS = 
"update-column-comments";
-    private static final String FIELD_UPDATE_COLUMN_POSITIONS = 
"update-column-positions";
-
-    @JsonProperty(FIELD_SET_OPTIONS)
-    private Map<String, String> setOptions;
-
-    @JsonProperty(FIELD_REMOVE_OPTIONS)
-    private List<String> removeOptions;
-
-    @JsonProperty(FIELD_COMMENT)
-    private String comment;
-
-    @JsonProperty(FIELD_ADD_COLUMNS)
-    private List<SchemaChange.AddColumn> addColumns;
-
-    @JsonProperty(FIELD_RENAME_COLUMNS)
-    private List<SchemaChange.RenameColumn> renameColumns;
-
-    @JsonProperty(FIELD_DROP_COLUMNS)
-    private List<String> dropColumns;
-
-    @JsonProperty(FIELD_UPDATE_COLUMN_TYPES)
-    private List<SchemaChange.UpdateColumnType> updateColumnTypes;
-
-    @JsonProperty(FIELD_UPDATE_COLUMN_NULLABILITIES)
-    private List<SchemaChange.UpdateColumnNullability> 
updateColumnNullabilities;
-
-    @JsonProperty(FIELD_UPDATE_COLUMN_COMMENTS)
-    private List<SchemaChange.UpdateColumnComment> updateColumnComments;
-
-    @JsonProperty(FIELD_UPDATE_COLUMN_POSITIONS)
-    private List<SchemaChange.Move> updateColumnPositions;
-
-    @JsonCreator
-    public SchemaChanges(
-            @JsonProperty(FIELD_SET_OPTIONS) Map<String, String> setOptions,
-            @JsonProperty(FIELD_REMOVE_OPTIONS) List<String> removeOptions,
-            @JsonProperty(FIELD_COMMENT) String comment,
-            @JsonProperty(FIELD_ADD_COLUMNS) List<SchemaChange.AddColumn> 
addColumns,
-            @JsonProperty(FIELD_RENAME_COLUMNS) 
List<SchemaChange.RenameColumn> renameColumns,
-            @JsonProperty(FIELD_DROP_COLUMNS) List<String> dropColumns,
-            @JsonProperty(FIELD_UPDATE_COLUMN_TYPES)
-                    List<SchemaChange.UpdateColumnType> updateColumnTypes,
-            @JsonProperty(FIELD_UPDATE_COLUMN_NULLABILITIES)
-                    List<SchemaChange.UpdateColumnNullability> 
updateColumnNullabilities,
-            @JsonProperty(FIELD_UPDATE_COLUMN_COMMENTS)
-                    List<SchemaChange.UpdateColumnComment> 
updateColumnComments,
-            @JsonProperty(FIELD_UPDATE_COLUMN_POSITIONS)
-                    List<SchemaChange.Move> updateColumnPositions) {
-        this.setOptions = setOptions;
-        this.removeOptions = removeOptions;
-        this.comment = comment;
-        this.addColumns = addColumns;
-        this.renameColumns = renameColumns;
-        this.dropColumns = dropColumns;
-        this.updateColumnTypes = updateColumnTypes;
-        this.updateColumnNullabilities = updateColumnNullabilities;
-        this.updateColumnComments = updateColumnComments;
-        this.updateColumnPositions = updateColumnPositions;
-    }
-
-    public SchemaChanges(List<SchemaChange> changes) {
-        Map<String, String> setOptions = new HashMap<>();
-        List<String> removeOptions = new ArrayList<>();
-        String comment = null;
-        List<SchemaChange.AddColumn> addColumns = new ArrayList<>();
-        List<SchemaChange.RenameColumn> renameColumns = new ArrayList<>();
-        List<String> dropColumns = new ArrayList<>();
-        List<SchemaChange.UpdateColumnType> updateColumnTypes = new 
ArrayList<>();
-        List<SchemaChange.UpdateColumnNullability> updateColumnNullabilities = 
new ArrayList<>();
-        List<SchemaChange.UpdateColumnComment> updateColumnComments = new 
ArrayList<>();
-        List<SchemaChange.Move> updateColumnPositions = new ArrayList<>();
-        for (SchemaChange change : changes) {
-            if (change instanceof SchemaChange.SetOption) {
-                setOptions.put(
-                        ((SchemaChange.SetOption) change).key(),
-                        ((SchemaChange.SetOption) change).value());
-            } else if (change instanceof SchemaChange.RemoveOption) {
-                removeOptions.add(((SchemaChange.RemoveOption) change).key());
-            } else if (change instanceof SchemaChange.UpdateComment) {
-                comment = ((SchemaChange.UpdateComment) change).comment();
-            } else if (change instanceof SchemaChange.AddColumn) {
-                addColumns.add((SchemaChange.AddColumn) change);
-            } else if (change instanceof SchemaChange.RenameColumn) {
-                renameColumns.add((SchemaChange.RenameColumn) change);
-            } else if (change instanceof SchemaChange.DropColumn) {
-                dropColumns.addAll(Arrays.asList(((SchemaChange.DropColumn) 
change).fieldNames()));
-            } else if (change instanceof SchemaChange.UpdateColumnType) {
-                updateColumnTypes.add((SchemaChange.UpdateColumnType) change);
-            } else if (change instanceof SchemaChange.UpdateColumnNullability) 
{
-                
updateColumnNullabilities.add((SchemaChange.UpdateColumnNullability) change);
-            } else if (change instanceof SchemaChange.UpdateColumnComment) {
-                updateColumnComments.add((SchemaChange.UpdateColumnComment) 
change);
-            } else if (change instanceof SchemaChange.UpdateColumnPosition) {
-                updateColumnPositions.add(((SchemaChange.UpdateColumnPosition) 
change).move());
-            }
-        }
-        this.setOptions = setOptions;
-        this.removeOptions = removeOptions;
-        this.comment = comment;
-        this.addColumns = addColumns;
-        this.renameColumns = renameColumns;
-        this.dropColumns = dropColumns;
-        this.updateColumnTypes = updateColumnTypes;
-        this.updateColumnNullabilities = updateColumnNullabilities;
-        this.updateColumnComments = updateColumnComments;
-        this.updateColumnPositions = updateColumnPositions;
-    }
-
-    @JsonGetter(FIELD_SET_OPTIONS)
-    public Map<String, String> getSetOptions() {
-        return setOptions;
-    }
-
-    @JsonGetter(FIELD_REMOVE_OPTIONS)
-    public List<String> getRemoveOptions() {
-        return removeOptions;
-    }
-
-    @JsonGetter(FIELD_COMMENT)
-    public String getComment() {
-        return comment;
-    }
-
-    @JsonGetter(FIELD_ADD_COLUMNS)
-    public List<SchemaChange.AddColumn> getAddColumns() {
-        return addColumns;
-    }
-
-    @JsonGetter(FIELD_RENAME_COLUMNS)
-    public List<SchemaChange.RenameColumn> getRenameColumns() {
-        return renameColumns;
-    }
-
-    @JsonGetter(FIELD_DROP_COLUMNS)
-    public List<String> getDropColumns() {
-        return dropColumns;
-    }
-
-    @JsonGetter(FIELD_UPDATE_COLUMN_TYPES)
-    public List<SchemaChange.UpdateColumnType> getUpdateColumnTypes() {
-        return updateColumnTypes;
-    }
-
-    @JsonGetter(FIELD_UPDATE_COLUMN_NULLABILITIES)
-    public List<SchemaChange.UpdateColumnNullability> 
getUpdateColumnNullabilities() {
-        return updateColumnNullabilities;
-    }
-
-    @JsonGetter(FIELD_UPDATE_COLUMN_COMMENTS)
-    public List<SchemaChange.UpdateColumnComment> getUpdateColumnComments() {
-        return updateColumnComments;
-    }
-
-    @JsonGetter(FIELD_UPDATE_COLUMN_POSITIONS)
-    public List<SchemaChange.Move> getUpdateColumnPositions() {
-        return updateColumnPositions;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        SchemaChanges that = (SchemaChanges) o;
-        return Objects.equals(setOptions, that.setOptions)
-                && Objects.equals(removeOptions, that.removeOptions)
-                && Objects.equals(comment, that.comment)
-                && Objects.equals(addColumns, that.addColumns)
-                && Objects.equals(renameColumns, that.renameColumns)
-                && Objects.equals(dropColumns, that.dropColumns)
-                && Objects.equals(updateColumnTypes, that.updateColumnTypes)
-                && Objects.equals(updateColumnNullabilities, 
that.updateColumnNullabilities)
-                && Objects.equals(updateColumnComments, 
that.updateColumnComments)
-                && Objects.equals(updateColumnPositions, 
that.updateColumnPositions);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(
-                setOptions,
-                removeOptions,
-                comment,
-                addColumns,
-                renameColumns,
-                dropColumns,
-                updateColumnTypes,
-                updateColumnNullabilities,
-                updateColumnComments,
-                updateColumnPositions);
-    }
-}
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java
deleted file mode 100644
index b522dc8ea1..0000000000
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.paimon.rest.requests;
-
-import org.apache.paimon.catalog.Identifier;
-import org.apache.paimon.rest.RESTRequest;
-
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
-import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
-
-/** Request for updating table. */
-public class UpdateTableRequest implements RESTRequest {
-
-    private static final String FIELD_IDENTIFIER_NAME = "identifier-change";
-    private static final String FIELD_SCHEMA_CHANGES_NAME = "schema-changes";
-
-    @JsonProperty(FIELD_IDENTIFIER_NAME)
-    private Identifier identifierChange;
-
-    @JsonProperty(FIELD_SCHEMA_CHANGES_NAME)
-    private SchemaChanges changes;
-
-    @JsonCreator
-    public UpdateTableRequest(
-            @JsonProperty(FIELD_IDENTIFIER_NAME) Identifier identifierChange,
-            @JsonProperty(FIELD_SCHEMA_CHANGES_NAME) SchemaChanges changes) {
-        this.identifierChange = identifierChange;
-        this.changes = changes;
-    }
-
-    @JsonGetter(FIELD_IDENTIFIER_NAME)
-    public Identifier getIdentifierChange() {
-        return identifierChange;
-    }
-
-    @JsonGetter(FIELD_SCHEMA_CHANGES_NAME)
-    public SchemaChanges getChanges() {
-        return changes;
-    }
-}
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java
index 08d751dc59..b963103e3e 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java
@@ -22,11 +22,13 @@ import org.apache.paimon.rest.RESTResponse;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.List;
 
 /** Response for altering database. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class AlterDatabaseResponse implements RESTResponse {
 
     private static final String FIELD_REMOVED = "removed";
@@ -34,13 +36,13 @@ public class AlterDatabaseResponse implements RESTResponse {
     private static final String FIELD_MISSING = "missing";
 
     @JsonProperty(FIELD_REMOVED)
-    private List<String> removed;
+    private final List<String> removed;
 
     @JsonProperty(FIELD_UPDATED)
-    private List<String> updated;
+    private final List<String> updated;
 
     @JsonProperty(FIELD_MISSING)
-    private List<String> missing;
+    private final List<String> missing;
 
     @JsonCreator
     public AlterDatabaseResponse(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java
index e8fff88b09..54e89931b6 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java
@@ -25,22 +25,24 @@ import 
org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap;
 import org.apache.paimon.shade.guava30.com.google.common.collect.Maps;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.Map;
 import java.util.Objects;
 
 /** Response for getting config. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class ConfigResponse implements RESTResponse {
 
     private static final String FIELD_DEFAULTS = "defaults";
     private static final String FIELD_OVERRIDES = "overrides";
 
     @JsonProperty(FIELD_DEFAULTS)
-    private Map<String, String> defaults;
+    private final Map<String, String> defaults;
 
     @JsonProperty(FIELD_OVERRIDES)
-    private Map<String, String> overrides;
+    private final Map<String, String> overrides;
 
     @JsonCreator
     public ConfigResponse(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java
index 43c99254f3..df71ff0e28 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java
@@ -22,21 +22,23 @@ import org.apache.paimon.rest.RESTResponse;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.Map;
 
 /** Response for creating database. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class CreateDatabaseResponse implements RESTResponse {
 
     private static final String FIELD_NAME = "name";
     private static final String FIELD_OPTIONS = "options";
 
     @JsonProperty(FIELD_NAME)
-    private String name;
+    private final String name;
 
     @JsonProperty(FIELD_OPTIONS)
-    private Map<String, String> options;
+    private final Map<String, String> options;
 
     @JsonCreator
     public CreateDatabaseResponse(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java
index d24c8f0f99..eb95ff448a 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java
@@ -22,6 +22,7 @@ import org.apache.paimon.rest.RESTResponse;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.io.PrintWriter;
@@ -31,6 +32,7 @@ import java.util.Arrays;
 import java.util.List;
 
 /** Response for error. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class ErrorResponse implements RESTResponse {
 
     private static final String FIELD_MESSAGE = "message";
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java
index f8f7c8794b..029e5c83cc 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java
@@ -23,6 +23,7 @@ import org.apache.paimon.rest.RESTResponse;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.Map;
@@ -31,6 +32,7 @@ import java.util.Optional;
 import static 
org.apache.paimon.rest.RESTCatalogInternalOptions.DATABASE_COMMENT;
 
 /** Response for getting database. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class GetDatabaseResponse implements RESTResponse, Database {
 
     private static final String FIELD_NAME = "name";
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java
index 671c50cac5..11f7f3a50d 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java
@@ -19,39 +19,52 @@
 package org.apache.paimon.rest.responses;
 
 import org.apache.paimon.rest.RESTResponse;
-import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.schema.Schema;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 /** Response for getting table. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class GetTableResponse implements RESTResponse {
 
-    private static final String FIELD_LOCATION = "location";
+    private static final String FIELD_PATH = "path";
+    private static final String FIELD_SCHEMA_ID = "schemaId";
     private static final String FIELD_SCHEMA = "schema";
 
-    @JsonProperty(FIELD_LOCATION)
-    private final String location;
+    @JsonProperty(FIELD_PATH)
+    private final String path;
+
+    @JsonProperty(FIELD_SCHEMA_ID)
+    private final long schemaId;
 
     @JsonProperty(FIELD_SCHEMA)
-    private final TableSchema schema;
+    private final Schema schema;
 
     @JsonCreator
     public GetTableResponse(
-            @JsonProperty(FIELD_LOCATION) String location,
-            @JsonProperty(FIELD_SCHEMA) TableSchema schema) {
-        this.location = location;
+            @JsonProperty(FIELD_PATH) String path,
+            @JsonProperty(FIELD_SCHEMA_ID) long schemaId,
+            @JsonProperty(FIELD_SCHEMA) Schema schema) {
+        this.path = path;
+        this.schemaId = schemaId;
         this.schema = schema;
     }
 
-    @JsonGetter(FIELD_LOCATION)
-    public String getLocation() {
-        return this.location;
+    @JsonGetter(FIELD_PATH)
+    public String getPath() {
+        return this.path;
+    }
+
+    @JsonGetter(FIELD_SCHEMA_ID)
+    public long getSchemaId() {
+        return this.schemaId;
     }
 
     @JsonGetter(FIELD_SCHEMA)
-    public TableSchema getSchema() {
+    public Schema getSchema() {
         return this.schema;
     }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java
index 64a17a6be7..8c27a1bacc 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java
@@ -22,16 +22,19 @@ import org.apache.paimon.rest.RESTResponse;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.List;
 
 /** Response for listing databases. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class ListDatabasesResponse implements RESTResponse {
+
     private static final String FIELD_DATABASES = "databases";
 
     @JsonProperty(FIELD_DATABASES)
-    private List<String> databases;
+    private final List<String> databases;
 
     @JsonCreator
     public ListDatabasesResponse(@JsonProperty(FIELD_DATABASES) List<String> 
databases) {
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListTablesResponse.java
 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListTablesResponse.java
index bccaa48438..9e1ce5ae87 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListTablesResponse.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListTablesResponse.java
@@ -22,16 +22,19 @@ import org.apache.paimon.rest.RESTResponse;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.List;
 
 /** Response for listing tables. */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class ListTablesResponse implements RESTResponse {
+
     private static final String FIELD_TABLES = "tables";
 
     @JsonProperty(FIELD_TABLES)
-    private List<String> tables;
+    private final List<String> tables;
 
     @JsonCreator
     public ListTablesResponse(@JsonProperty(FIELD_TABLES) List<String> tables) 
{
diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java 
b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java
index ee09841588..8a679665fb 100644
--- a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java
+++ b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java
@@ -29,6 +29,7 @@ import org.apache.paimon.utils.Preconditions;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonInclude;
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 
 import javax.annotation.Nullable;
@@ -55,8 +56,8 @@ import java.util.stream.Collectors;
 public class Schema {
 
     private static final String FIELD_FIELDS = "fields";
-    private static final String FIELD_PARTITION_KEYS = "partition-keys";
-    private static final String FIELD_PRIMARY_KEYS = "primary-keys";
+    private static final String FIELD_PARTITION_KEYS = "partitionKeys";
+    private static final String FIELD_PRIMARY_KEYS = "primaryKeys";
     private static final String FIELD_OPTIONS = "options";
     private static final String FIELD_COMMENT = "comment";
 
@@ -72,7 +73,9 @@ public class Schema {
     @JsonProperty(FIELD_OPTIONS)
     private final Map<String, String> options;
 
+    @Nullable
     @JsonProperty(FIELD_COMMENT)
+    @JsonInclude(JsonInclude.Include.NON_NULL)
     private final String comment;
 
     @JsonCreator
@@ -81,7 +84,7 @@ public class Schema {
             @JsonProperty(FIELD_PARTITION_KEYS) List<String> partitionKeys,
             @JsonProperty(FIELD_PRIMARY_KEYS) List<String> primaryKeys,
             @JsonProperty(FIELD_OPTIONS) Map<String, String> options,
-            @JsonProperty(FIELD_COMMENT) String comment) {
+            @Nullable @JsonProperty(FIELD_COMMENT) String comment) {
         this.options = new HashMap<>(options);
         this.partitionKeys = normalizePartitionKeys(partitionKeys);
         this.primaryKeys = normalizePrimaryKeys(primaryKeys);
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java 
b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java
index a600b089c5..5f6b104575 100644
--- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java
+++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java
@@ -231,8 +231,8 @@ public interface SchemaChange extends Serializable {
 
         private static final long serialVersionUID = 1L;
 
-        private static final String FIELD_FILED_NAMES = "field-names";
-        private static final String FIELD_DATA_TYPE = "data-type";
+        private static final String FIELD_FILED_NAMES = "fieldNames";
+        private static final String FIELD_DATA_TYPE = "dataType";
         private static final String FIELD_COMMENT = "comment";
         private static final String FIELD_MOVE = "move";
 
@@ -312,8 +312,8 @@ public interface SchemaChange extends Serializable {
 
         private static final long serialVersionUID = 1L;
 
-        private static final String FIELD_FILED_NAMES = "field-names";
-        private static final String FIELD_NEW_NAME = "new-name";
+        private static final String FIELD_FILED_NAMES = "fieldNames";
+        private static final String FIELD_NEW_NAME = "newName";
 
         @JsonProperty(FIELD_FILED_NAMES)
         private final String[] fieldNames;
@@ -366,7 +366,7 @@ public interface SchemaChange extends Serializable {
 
         private static final long serialVersionUID = 1L;
 
-        private static final String FIELD_FILED_NAMES = "field-names";
+        private static final String FIELD_FILED_NAMES = "fieldNames";
 
         @JsonProperty(FIELD_FILED_NAMES)
         private final String[] fieldNames;
@@ -404,9 +404,9 @@ public interface SchemaChange extends Serializable {
     final class UpdateColumnType implements SchemaChange {
 
         private static final long serialVersionUID = 1L;
-        private static final String FIELD_FILED_NAMES = "field-names";
-        private static final String FIELD_NEW_DATA_TYPE = "new-data-type";
-        private static final String FIELD_KEEP_NULLABILITY = 
"keep-nullability";
+        private static final String FIELD_FILED_NAMES = "fieldNames";
+        private static final String FIELD_NEW_DATA_TYPE = "newDataType";
+        private static final String FIELD_KEEP_NULLABILITY = "keepNullability";
 
         @JsonProperty(FIELD_FILED_NAMES)
         private final String[] fieldNames;
@@ -525,8 +525,8 @@ public interface SchemaChange extends Serializable {
 
         private static final long serialVersionUID = 1L;
 
-        private static final String FIELD_FILED_NAMES = "field-name";
-        private static final String FIELD_REFERENCE_FIELD_NAME = 
"reference-field-name";
+        private static final String FIELD_FILED_NAMES = "fieldName";
+        private static final String FIELD_REFERENCE_FIELD_NAME = 
"referenceFieldName";
         private static final String FIELD_TYPE = "type";
 
         @JsonProperty(FIELD_FILED_NAMES)
@@ -589,8 +589,8 @@ public interface SchemaChange extends Serializable {
 
         private static final long serialVersionUID = 1L;
 
-        private static final String FIELD_FILED_NAMES = "field-names";
-        private static final String FIELD_NEW_NULLABILITY = "new-nullability";
+        private static final String FIELD_FILED_NAMES = "fieldNames";
+        private static final String FIELD_NEW_NULLABILITY = "newNullability";
 
         @JsonProperty(FIELD_FILED_NAMES)
         private final String[] fieldNames;
@@ -643,8 +643,8 @@ public interface SchemaChange extends Serializable {
 
         private static final long serialVersionUID = 1L;
 
-        private static final String FIELD_FILED_NAMES = "field-names";
-        private static final String FIELD_NEW_COMMENT = "new-comment";
+        private static final String FIELD_FILED_NAMES = "fieldNames";
+        private static final String FIELD_NEW_COMMENT = "newComment";
 
         @JsonProperty(FIELD_FILED_NAMES)
         private final String[] fieldNames;
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 9edbe901b6..bf75abb7c4 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
@@ -222,21 +222,7 @@ public class SchemaManager implements Serializable {
                 }
             }
 
-            List<DataField> fields = schema.fields();
-            List<String> partitionKeys = schema.partitionKeys();
-            List<String> primaryKeys = schema.primaryKeys();
-            Map<String, String> options = schema.options();
-            int highestFieldId = RowType.currentHighestFieldId(fields);
-
-            TableSchema newSchema =
-                    new TableSchema(
-                            0,
-                            fields,
-                            highestFieldId,
-                            partitionKeys,
-                            primaryKeys,
-                            options,
-                            schema.comment());
+            TableSchema newSchema = TableSchema.create(0, schema);
 
             // validate table from creating table
             FileStoreTableFactory.create(fileIO, tableRoot, newSchema).store();
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java 
b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
index 791269dc73..a38340a6e5 100644
--- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
+++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
@@ -358,4 +358,21 @@ public class TableSchema implements Serializable {
             throw new UncheckedIOException(e);
         }
     }
+
+    public static TableSchema create(long schemaId, Schema schema) {
+        List<DataField> fields = schema.fields();
+        List<String> partitionKeys = schema.partitionKeys();
+        List<String> primaryKeys = schema.primaryKeys();
+        Map<String, String> options = schema.options();
+        int highestFieldId = RowType.currentHighestFieldId(fields);
+
+        return new TableSchema(
+                schemaId,
+                fields,
+                highestFieldId,
+                partitionKeys,
+                primaryKeys,
+                options,
+                schema.comment());
+    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
index 763e4d1216..b77b72e412 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
@@ -18,9 +18,6 @@
 
 package org.apache.paimon.table.system;
 
-import org.apache.paimon.fs.FileIO;
-import org.apache.paimon.fs.Path;
-import org.apache.paimon.options.Options;
 import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.Table;
 
@@ -34,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.function.Function;
-import java.util.function.Supplier;
 
 import static 
org.apache.paimon.table.system.AggregationFieldsTable.AGGREGATION_FIELDS;
 import static 
org.apache.paimon.table.system.AllTableOptionsTable.ALL_TABLE_OPTIONS;
@@ -85,22 +81,6 @@ public class SystemTableLoader {
                 .orElse(null);
     }
 
-    @Nullable
-    public static Table loadGlobal(
-            String tableName,
-            FileIO fileIO,
-            Supplier<Map<String, Map<String, Path>>> allTablePaths,
-            Options catalogOptions) {
-        switch (tableName.toLowerCase()) {
-            case ALL_TABLE_OPTIONS:
-                return new AllTableOptionsTable(fileIO, allTablePaths.get());
-            case CATALOG_OPTIONS:
-                return new CatalogOptionsTable(catalogOptions);
-            default:
-                return null;
-        }
-    }
-
     public static List<String> loadGlobalTableNames() {
         return Arrays.asList(ALL_TABLE_OPTIONS, CATALOG_OPTIONS);
     }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java
index 3e9f32ba08..6a9f2df751 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java
@@ -18,13 +18,11 @@
 
 package org.apache.paimon.rest;
 
-import org.apache.paimon.CoreOptions;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.rest.requests.AlterDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateTableRequest;
-import org.apache.paimon.rest.requests.SchemaChanges;
-import org.apache.paimon.rest.requests.UpdateTableRequest;
+import org.apache.paimon.rest.requests.RenameTableRequest;
 import org.apache.paimon.rest.responses.AlterDatabaseResponse;
 import org.apache.paimon.rest.responses.CreateDatabaseResponse;
 import org.apache.paimon.rest.responses.ErrorResponse;
@@ -34,7 +32,6 @@ import org.apache.paimon.rest.responses.ListDatabasesResponse;
 import org.apache.paimon.rest.responses.ListTablesResponse;
 import org.apache.paimon.schema.Schema;
 import org.apache.paimon.schema.SchemaChange;
-import org.apache.paimon.schema.TableSchema;
 import org.apache.paimon.types.DataField;
 import org.apache.paimon.types.DataType;
 import org.apache.paimon.types.DataTypes;
@@ -124,10 +121,9 @@ public class MockRESTMessage {
         return new CreateTableRequest(identifier, schema);
     }
 
-    public static UpdateTableRequest updateTableRequest(String toTableName) {
-        Identifier identifierChange = Identifier.create(databaseName(), 
toTableName);
-        SchemaChanges changes = new SchemaChanges(getChanges());
-        return new UpdateTableRequest(identifierChange, changes);
+    public static RenameTableRequest renameRequest(String toTableName) {
+        Identifier newIdentifier = Identifier.create(databaseName(), 
toTableName);
+        return new RenameTableRequest(newIdentifier);
     }
 
     public static List<SchemaChange> getChanges() {
@@ -199,10 +195,10 @@ public class MockRESTMessage {
     }
 
     public static GetTableResponse getTableResponse() {
-        return new GetTableResponse("location", tableSchema());
+        return new GetTableResponse("/tmp/1", 1, schema());
     }
 
-    private static TableSchema tableSchema() {
+    private static Schema schema() {
         List<DataField> fields =
                 Arrays.asList(
                         new DataField(0, "f0", new IntType()),
@@ -212,8 +208,6 @@ public class MockRESTMessage {
         Map<String, String> options = new HashMap<>();
         options.put("option-1", "value-1");
         options.put("option-2", "value-2");
-        // set path for test as if not set system will add one
-        options.put(CoreOptions.PATH.key(), "/a/b/c");
-        return new TableSchema(1, fields, 1, partitionKeys, primaryKeys, 
options, "comment");
+        return new Schema(fields, partitionKeys, primaryKeys, options, 
"comment");
     }
 }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java
index a3c4ab3401..95a991a74a 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java
@@ -32,7 +32,6 @@ import org.apache.paimon.rest.responses.GetDatabaseResponse;
 import org.apache.paimon.rest.responses.GetTableResponse;
 import org.apache.paimon.rest.responses.ListDatabasesResponse;
 import org.apache.paimon.rest.responses.ListTablesResponse;
-import org.apache.paimon.schema.SchemaChange;
 import org.apache.paimon.table.Table;
 
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
@@ -56,7 +55,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyList;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -69,7 +67,6 @@ public class RESTCatalogTest {
     private MockWebServer mockWebServer;
     private RESTCatalog restCatalog;
     private RESTCatalog mockRestCatalog;
-    private CatalogContext context;
     private String warehouseStr;
     @Rule public TemporaryFolder folder = new TemporaryFolder();
 
@@ -92,8 +89,7 @@ public class RESTCatalogTest {
                         CatalogOptions.WAREHOUSE.key(),
                         warehouseStr);
         mockResponse(mockResponse, 200);
-        context = CatalogContext.create(options);
-        restCatalog = new RESTCatalog(context);
+        restCatalog = new RESTCatalog(CatalogContext.create(options));
         mockRestCatalog = spy(restCatalog);
     }
 
@@ -247,7 +243,8 @@ public class RESTCatalogTest {
         GetTableResponse response = MockRESTMessage.getTableResponse();
         mockResponse(mapper.writeValueAsString(response), 200);
         Table result = 
mockRestCatalog.getTable(Identifier.create(databaseName, "table"));
-        assertEquals(response.getSchema().options().size(), 
result.options().size());
+        // catalog will add path option
+        assertEquals(response.getSchema().options().size() + 1, 
result.options().size());
         verify(mockRestCatalog, times(1)).getDataOrFormatTable(any());
     }
 
@@ -282,7 +279,7 @@ public class RESTCatalogTest {
                                 Identifier.create(databaseName, fromTableName),
                                 Identifier.create(databaseName, toTableName),
                                 true));
-        verify(mockRestCatalog, times(1)).updateTable(any(), any(), anyList());
+        verify(mockRestCatalog, times(1)).renameTable(any(), any());
     }
 
     @Test
@@ -315,31 +312,6 @@ public class RESTCatalogTest {
                                 false));
     }
 
-    @Test
-    public void testAlterTable() throws Exception {
-        String databaseName = MockRESTMessage.databaseName();
-        List<SchemaChange> changes = MockRESTMessage.getChanges();
-        GetTableResponse response = MockRESTMessage.getTableResponse();
-        mockResponse(mapper.writeValueAsString(response), 200);
-        assertDoesNotThrow(
-                () ->
-                        mockRestCatalog.alterTable(
-                                Identifier.create(databaseName, "t1"), 
changes, true));
-        verify(mockRestCatalog, times(1)).updateTable(any(), any(), anyList());
-    }
-
-    @Test
-    public void testAlterTableWhenTableNotExistAndIgnoreIfNotExistsIsFalse() 
throws Exception {
-        String databaseName = MockRESTMessage.databaseName();
-        List<SchemaChange> changes = MockRESTMessage.getChanges();
-        mockResponse("", 404);
-        assertThrows(
-                Catalog.TableNotExistException.class,
-                () ->
-                        mockRestCatalog.alterTable(
-                                Identifier.create(databaseName, "t1"), 
changes, false));
-    }
-
     @Test
     public void testDropTable() throws Exception {
         String databaseName = MockRESTMessage.databaseName();
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java
index 9cc362881f..c2d67fd46e 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java
@@ -21,7 +21,7 @@ package org.apache.paimon.rest;
 import org.apache.paimon.rest.requests.AlterDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateTableRequest;
-import org.apache.paimon.rest.requests.UpdateTableRequest;
+import org.apache.paimon.rest.requests.RenameTableRequest;
 import org.apache.paimon.rest.responses.AlterDatabaseResponse;
 import org.apache.paimon.rest.responses.ConfigResponse;
 import org.apache.paimon.rest.responses.CreateDatabaseResponse;
@@ -161,12 +161,11 @@ public class RESTObjectMapperTest {
     }
 
     @Test
-    public void updateTableRequestParseTest() throws Exception {
-        UpdateTableRequest request = MockRESTMessage.updateTableRequest("t2");
+    public void renameTableRequestParseTest() throws Exception {
+        RenameTableRequest request = MockRESTMessage.renameRequest("t2");
         String requestStr = mapper.writeValueAsString(request);
-        UpdateTableRequest parseData = mapper.readValue(requestStr, 
UpdateTableRequest.class);
-        assertEquals(request.getIdentifierChange(), 
parseData.getIdentifierChange());
-        assertEquals(request.getChanges(), parseData.getChanges());
+        RenameTableRequest parseData = mapper.readValue(requestStr, 
RenameTableRequest.class);
+        assertEquals(request.getNewIdentifier(), parseData.getNewIdentifier());
     }
 
     @Test
@@ -174,7 +173,7 @@ public class RESTObjectMapperTest {
         GetTableResponse response = MockRESTMessage.getTableResponse();
         String responseStr = mapper.writeValueAsString(response);
         GetTableResponse parseData = mapper.readValue(responseStr, 
GetTableResponse.class);
-        assertEquals(response.getLocation(), parseData.getLocation());
+        assertEquals(response.getSchemaId(), parseData.getSchemaId());
         assertEquals(response.getSchema(), parseData.getSchema());
     }
 
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 fd22ca2032..e29d83db45 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
@@ -104,7 +104,6 @@ import static 
org.apache.paimon.catalog.CatalogUtils.checkNotBranch;
 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.lockFactory;
 import static org.apache.paimon.hive.HiveCatalogLock.acquireTimeout;
 import static org.apache.paimon.hive.HiveCatalogLock.checkMaxSleep;
 import static org.apache.paimon.hive.HiveCatalogOptions.HADOOP_CONF_DIR;
@@ -637,8 +636,7 @@ public class HiveCatalog extends AbstractCatalog {
                             identifier,
                             tableMeta.uuid(),
                             Lock.factory(
-                                    lockFactory(catalogOptions, fileIO(), 
defaultLockFactory())
-                                            .orElse(null),
+                                    lockFactory().orElse(null),
                                     lockContext().orElse(null),
                                     identifier),
                             metastoreClientFactory(identifier).orElse(null)));
diff --git 
a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java
 
b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java
index 38fa4dfe4d..5cc826b554 100644
--- 
a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java
+++ 
b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java
@@ -19,8 +19,8 @@
 package org.apache.paimon.hive;
 
 import org.apache.paimon.CoreOptions;
+import org.apache.paimon.catalog.AbstractCatalog;
 import org.apache.paimon.catalog.CatalogContext;
-import org.apache.paimon.catalog.CatalogUtils;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
@@ -87,7 +87,7 @@ public class PaimonMetaHook implements HiveMetaHook {
             org.apache.hadoop.fs.Path hadoopPath =
                     getDnsPath(new org.apache.hadoop.fs.Path(warehouse), conf);
             warehouse = hadoopPath.toUri().toString();
-            location = CatalogUtils.newTableLocation(warehouse, 
identifier).toUri().toString();
+            location = AbstractCatalog.newTableLocation(warehouse, 
identifier).toUri().toString();
             table.getSd().setLocation(location);
         }
 
diff --git 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java
 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java
index 992272b0f6..7819f7b44c 100644
--- 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java
+++ 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java
@@ -21,7 +21,6 @@ package org.apache.paimon.hive;
 import org.apache.paimon.catalog.Catalog;
 import org.apache.paimon.catalog.CatalogContext;
 import org.apache.paimon.catalog.CatalogFactory;
-import org.apache.paimon.catalog.CatalogUtils;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.fs.local.LocalFileIO;
@@ -51,6 +50,7 @@ import java.util.List;
 import java.util.Optional;
 
 import static org.apache.paimon.CoreOptions.METASTORE_PARTITIONED_TABLE;
+import static org.apache.paimon.catalog.AbstractCatalog.newTableLocation;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatCode;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
@@ -94,7 +94,7 @@ public class CreateTableITCase extends HiveTestBase {
                         Maps.newHashMap(),
                         "");
         Identifier identifier = Identifier.create(DATABASE_TEST, tableName);
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema);
 
         // Create hive external table
@@ -189,7 +189,7 @@ public class CreateTableITCase extends HiveTestBase {
 
         // check the paimon table schema
         Identifier identifier = Identifier.create(DATABASE_TEST, tableName);
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         Optional<TableSchema> tableSchema =
                 new SchemaManager(LocalFileIO.create(), tablePath).latest();
         assertThat(tableSchema).isPresent();
@@ -245,7 +245,7 @@ public class CreateTableITCase extends HiveTestBase {
         }
         // check the paimon table name and schema
         Identifier identifier = Identifier.create(DATABASE_TEST, 
tableName.toLowerCase());
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         Options conf = new Options();
         conf.set(CatalogOptions.WAREHOUSE, path);
         CatalogContext catalogContext = CatalogContext.create(conf);
@@ -310,7 +310,7 @@ public class CreateTableITCase extends HiveTestBase {
 
         // check the paimon db name态table name and schema
         Identifier identifier = Identifier.create(upperDB.toLowerCase(), 
tableName.toLowerCase());
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         Options conf = new Options();
         conf.set(CatalogOptions.WAREHOUSE, path);
         CatalogContext catalogContext = CatalogContext.create(conf);
@@ -355,7 +355,7 @@ public class CreateTableITCase extends HiveTestBase {
 
         // check the paimon table schema
         Identifier identifier = Identifier.create(DATABASE_TEST, tableName);
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         Optional<TableSchema> tableSchema =
                 new SchemaManager(LocalFileIO.create(), tablePath).latest();
         assertThat(tableSchema).isPresent();
@@ -397,7 +397,7 @@ public class CreateTableITCase extends HiveTestBase {
 
         // check the paimon table schema
         Identifier identifier = Identifier.create(DATABASE_TEST, tableName);
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         Optional<TableSchema> tableSchema =
                 new SchemaManager(LocalFileIO.create(), tablePath).latest();
         assertThat(tableSchema).isPresent();
@@ -441,7 +441,7 @@ public class CreateTableITCase extends HiveTestBase {
 
         // check the paimon table schema
         Identifier identifier = Identifier.create(DATABASE_TEST, tableName);
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = newTableLocation(path, identifier);
         Optional<TableSchema> tableSchema =
                 new SchemaManager(LocalFileIO.create(), tablePath).latest();
         assertThat(tableSchema).isPresent();
@@ -489,7 +489,7 @@ public class CreateTableITCase extends HiveTestBase {
                             Maps.newHashMap(),
                             "");
             Identifier identifier = Identifier.create(DATABASE_TEST, 
tableName);
-            Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+            Path tablePath = newTableLocation(path, identifier);
             new SchemaManager(LocalFileIO.create(), 
tablePath).createTable(schema);
 
             String hiveSql =
@@ -533,7 +533,7 @@ public class CreateTableITCase extends HiveTestBase {
             } catch (Exception ignore) {
             } finally {
                 Identifier identifier = Identifier.create(DATABASE_TEST, 
tableName);
-                Path tablePath = CatalogUtils.newTableLocation(path, 
identifier);
+                Path tablePath = newTableLocation(path, identifier);
                 boolean isPresent =
                         new SchemaManager(LocalFileIO.create(), 
tablePath).latest().isPresent();
                 Assertions.assertThat(isPresent).isFalse();
diff --git 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java
 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java
index 7e52b89279..f3fe03fbba 100644
--- 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java
+++ 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.paimon.hive;
 
+import org.apache.paimon.catalog.AbstractCatalog;
 import org.apache.paimon.catalog.CatalogContext;
-import org.apache.paimon.catalog.CatalogUtils;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
@@ -257,7 +257,7 @@ public class HiveLocationTest {
 
             Identifier identifier = Identifier.create(dbName, tableName);
             String location =
-                    CatalogUtils.newTableLocation(warehouse, 
identifier).toUri().toString();
+                    AbstractCatalog.newTableLocation(warehouse, 
identifier).toUri().toString();
 
             String createTableSqlStr =
                     getCreateTableSqlStr(tableName, location, 
locationInProperties);
diff --git 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java
 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java
index 882215f7c0..4b16788ee7 100644
--- 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java
+++ 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java
@@ -19,7 +19,7 @@
 package org.apache.paimon.hive;
 
 import org.apache.paimon.CoreOptions;
-import org.apache.paimon.catalog.CatalogUtils;
+import org.apache.paimon.catalog.AbstractCatalog;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.data.BinaryString;
 import org.apache.paimon.data.Decimal;
@@ -975,7 +975,7 @@ public abstract class HiveReadITCaseBase extends 
HiveTestBase {
                         Maps.newHashMap(),
                         "");
         Identifier identifier = Identifier.create(DATABASE_TEST, tableName);
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = AbstractCatalog.newTableLocation(path, identifier);
         new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema);
 
         // Create hive external table
@@ -1057,7 +1057,7 @@ public abstract class HiveReadITCaseBase extends 
HiveTestBase {
         commit.close();
 
         // add column, do some ddl which will generate a new version schema-n 
file.
-        Path tablePath = CatalogUtils.newTableLocation(path, identifier);
+        Path tablePath = AbstractCatalog.newTableLocation(path, identifier);
         SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), 
tablePath);
         schemaManager.commitChanges(SchemaChange.addColumn("N1", 
DataTypes.STRING()));
 
diff --git a/paimon-open-api/rest-catalog-open-api.yaml 
b/paimon-open-api/rest-catalog-open-api.yaml
index f3f3f33349..de0faeeb4a 100644
--- a/paimon-open-api/rest-catalog-open-api.yaml
+++ b/paimon-open-api/rest-catalog-open-api.yaml
@@ -176,8 +176,8 @@ paths:
     post:
       tags:
         - table
-      summary: Update table
-      operationId: updateTable
+      summary: Rename table
+      operationId: renameTable
       parameters:
         - name: prefix
           in: path
@@ -198,7 +198,7 @@ paths:
         content:
           application/json:
             schema:
-              $ref: '#/components/schemas/UpdateTableRequest'
+              $ref: '#/components/schemas/RenameTableRequest'
       responses:
         "200":
           description: OK
@@ -413,11 +413,11 @@ components:
           type: array
           items:
             $ref: '#/components/schemas/DataField'
-        partition-keys:
+        partitionKeys:
           type: array
           items:
             type: string
-        primary-keys:
+        primaryKeys:
           type: array
           items:
             type: string
@@ -430,152 +430,18 @@ components:
     GetTableResponse:
       type: object
       properties:
-        location:
+        path:
           type: string
-        schema:
-          $ref: '#/components/schemas/TableSchema'
-    TableSchema:
-      type: object
-      properties:
-        version:
-          type: integer
-          format: int32
-        id:
+        schemaId:
           type: integer
           format: int64
-        highestFieldId:
-          type: integer
-          format: int32
-        partitionKeys:
-          type: array
-          items:
-            type: string
-        primaryKeys:
-          type: array
-          items:
-            type: string
-        options:
-          type: object
-          additionalProperties:
-            type: string
-        comment:
-          type: string
-        timeMillis:
-          type: integer
-          format: int64
-    AddColumn:
-      type: object
-      properties:
-        field-names:
-          type: array
-          items:
-            type: string
-        data-types:
-          $ref: '#/components/schemas/DataType'
-        comment:
-          type: string
-        move:
-          $ref: '#/components/schemas/Move'
-    Move:
-      type: object
-      properties:
-        field-name:
-          type: string
-        reference-field-name:
-          type: string
-        type:
-          type: string
-          enum:
-            - FIRST
-            - AFTER
-            - BEFORE
-            - LAST
-    RenameColumn:
-      type: object
-      properties:
-        field-names:
-          type: array
-          items:
-            type: string
-        new-name:
-          type: string
-    SchemaChanges:
-      type: object
-      properties:
-        set-options:
-          type: object
-          additionalProperties:
-            type: string
-        remove-options:
-          type: array
-          items:
-            type: string
-        comment:
-          type: string
-        add-columns:
-          type: array
-          items:
-            $ref: '#/components/schemas/AddColumn'
-        rename-columns:
-          type: array
-          items:
-            $ref: '#/components/schemas/RenameColumn'
-        drop-columns:
-          type: array
-          items:
-            type: string
-        update-column-types:
-          type: array
-          items:
-            $ref: '#/components/schemas/UpdateColumnType'
-        update-column-nullabilities:
-          type: array
-          items:
-            $ref: '#/components/schemas/UpdateColumnNullability'
-        update-column-comments:
-          type: array
-          items:
-            $ref: '#/components/schemas/UpdateColumnComment'
-        update-column-positions:
-          type: array
-          items:
-            $ref: '#/components/schemas/Move'
-    UpdateColumnComment:
-      type: object
-      properties:
-        field-names:
-          type: array
-          items:
-            type: string
-        new-comment:
-          type: string
-    UpdateColumnNullability:
-      type: object
-      properties:
-        field-names:
-          type: array
-          items:
-            type: string
-        new-nullability:
-          type: boolean
-    UpdateColumnType:
-      type: object
-      properties:
-        field-names:
-          type: array
-          items:
-            type: string
-        new-data-types:
-          $ref: '#/components/schemas/DataType'
-        keep-nullability:
-          type: boolean
-    UpdateTableRequest:
+        schema:
+          $ref: '#/components/schemas/Schema'
+    RenameTableRequest:
       type: object
       properties:
-        identifier-change:
+        newIdentifier:
           $ref: '#/components/schemas/Identifier'
-        schema-changes:
-          $ref: '#/components/schemas/SchemaChanges'
     AlterDatabaseRequest:
       type: object
       properties:
diff --git 
a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java
 
b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java
index 0341b2556e..3ea8faee8c 100644
--- 
a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java
+++ 
b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java
@@ -22,7 +22,7 @@ import org.apache.paimon.rest.ResourcePaths;
 import org.apache.paimon.rest.requests.AlterDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateDatabaseRequest;
 import org.apache.paimon.rest.requests.CreateTableRequest;
-import org.apache.paimon.rest.requests.UpdateTableRequest;
+import org.apache.paimon.rest.requests.RenameTableRequest;
 import org.apache.paimon.rest.responses.AlterDatabaseResponse;
 import org.apache.paimon.rest.responses.ConfigResponse;
 import org.apache.paimon.rest.responses.CreateDatabaseResponse;
@@ -31,7 +31,6 @@ import org.apache.paimon.rest.responses.GetDatabaseResponse;
 import org.apache.paimon.rest.responses.GetTableResponse;
 import org.apache.paimon.rest.responses.ListDatabasesResponse;
 import org.apache.paimon.rest.responses.ListTablesResponse;
-import org.apache.paimon.schema.TableSchema;
 
 import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList;
 import org.apache.paimon.shade.guava30.com.google.common.collect.Lists;
@@ -224,17 +223,14 @@ public class RESTCatalogController {
             @PathVariable String database,
             @PathVariable String table) {
         return new GetTableResponse(
-                "location",
-                new TableSchema(
-                        1,
-                        1,
+                "",
+                1,
+                new org.apache.paimon.schema.Schema(
                         ImmutableList.of(),
-                        1,
                         ImmutableList.of(),
                         ImmutableList.of(),
                         new HashMap<>(),
-                        "comment",
-                        1L));
+                        "comment"));
     }
 
     @Operation(
@@ -254,17 +250,14 @@ public class RESTCatalogController {
             @PathVariable String database,
             @RequestBody CreateTableRequest request) {
         return new GetTableResponse(
-                "location",
-                new TableSchema(
-                        1,
-                        1,
+                "",
+                1,
+                new org.apache.paimon.schema.Schema(
                         ImmutableList.of(),
-                        1,
                         ImmutableList.of(),
                         ImmutableList.of(),
                         new HashMap<>(),
-                        "comment",
-                        1L));
+                        "comment"));
     }
 
     @Operation(
@@ -279,23 +272,20 @@ public class RESTCatalogController {
                 content = {@Content(schema = @Schema())})
     })
     @PostMapping("/v1/{prefix}/databases/{database}/tables/{table}")
-    public GetTableResponse updateTable(
+    public GetTableResponse renameTable(
             @PathVariable String prefix,
             @PathVariable String database,
             @PathVariable String table,
-            @RequestBody UpdateTableRequest request) {
+            @RequestBody RenameTableRequest request) {
         return new GetTableResponse(
-                "location",
-                new TableSchema(
-                        1,
-                        1,
+                "",
+                1,
+                new org.apache.paimon.schema.Schema(
                         ImmutableList.of(),
-                        1,
                         ImmutableList.of(),
                         ImmutableList.of(),
                         new HashMap<>(),
-                        "comment",
-                        1L));
+                        "comment"));
     }
 
     @Operation(

Reply via email to