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 8feb1be82b [core] Introduce paimon-api module for light rest api, part 
3 (#5610)
8feb1be82b is described below

commit 8feb1be82b56aedff5bc93e10aab0fe73c5ac013
Author: Jingsong Lee <[email protected]>
AuthorDate: Fri May 16 10:26:08 2025 +0800

    [core] Introduce paimon-api module for light rest api, part 3 (#5610)
---
 docs/content/concepts/rest/overview.md             |   8 +
 docs/content/program-api/catalog-api.md            |   2 +-
 docs/content/program-api/flink-api.md              |   2 +-
 docs/content/program-api/java-api.md               |   2 +-
 docs/content/program-api/python-api.md             |   2 +-
 docs/content/program-api/rest-api.md               |  88 ++++
 paimon-api/README.md                               |   6 +-
 .../src/main/java/org/apache/paimon/PagedList.java |   1 +
 .../java/org/apache/paimon/rest/HttpClient.java    |   9 +-
 .../main/java/org/apache/paimon/rest/RESTApi.java  | 572 +++++++++++++++++++++
 .../org/apache/paimon/rest/RESTObjectMapper.java   |  11 +-
 .../apache/paimon/rest/auth/DLFAuthProvider.java   |   3 +-
 .../apache/paimon/rest/auth/DLFECSTokenLoader.java |   4 +-
 .../paimon/rest/auth/DLFLocalFileTokenLoader.java  |   5 +-
 .../java/org/apache/paimon/utils/StringUtils.java  |   4 +
 .../java/org/apache/paimon/rest/RESTCatalog.java   | 459 +++--------------
 .../org/apache/paimon/rest/RESTTokenFileIO.java    |   2 +-
 .../apache/paimon/rest/MockRESTCatalogTest.java    |   2 +-
 ...TObjectMapperTest.java => RESTApiJsonTest.java} | 112 ++--
 .../org/apache/paimon/rest/RESTCatalogServer.java  |  56 +-
 .../org/apache/paimon/rest/RESTCatalogTest.java    |  31 +-
 .../org/apache/paimon/rest/TestHttpWebServer.java  |   6 +-
 .../apache/paimon/rest/auth/AuthProviderTest.java  |   2 +-
 .../paimon/rest/auth/DLFAuthSignatureTest.java     |   6 +-
 24 files changed, 863 insertions(+), 532 deletions(-)

diff --git a/docs/content/concepts/rest/overview.md 
b/docs/content/concepts/rest/overview.md
index df2349bd18..a38744dbed 100644
--- a/docs/content/concepts/rest/overview.md
+++ b/docs/content/concepts/rest/overview.md
@@ -62,3 +62,11 @@ RESTCatalog supports multiple access authentication methods, 
including the follo
 
 1. [Bear Token]({{< ref "concepts/rest/bear" >}}).
 2. [DLF Token]({{< ref "concepts/rest/dlf" >}}).
+
+## REST Open API
+
+See [REST API]({{< ref "concepts/rest/rest-api" >}}).
+
+## REST Java API
+
+See [REST Java API]({{< ref "program-api/rest-api" >}}).
diff --git a/docs/content/program-api/catalog-api.md 
b/docs/content/program-api/catalog-api.md
index c7fcf46aba..cab46f77dc 100644
--- a/docs/content/program-api/catalog-api.md
+++ b/docs/content/program-api/catalog-api.md
@@ -1,6 +1,6 @@
 ---
 title: "Catalog API"
-weight: 3
+weight: 4
 type: docs
 aliases:
 - /api/catalog-api.html
diff --git a/docs/content/program-api/flink-api.md 
b/docs/content/program-api/flink-api.md
index cf3170fedb..f32e63d3fc 100644
--- a/docs/content/program-api/flink-api.md
+++ b/docs/content/program-api/flink-api.md
@@ -1,6 +1,6 @@
 ---
 title: "Flink API"
-weight: 1
+weight: 2
 type: docs
 aliases:
 - /api/flink-api.html
diff --git a/docs/content/program-api/java-api.md 
b/docs/content/program-api/java-api.md
index 09e9b7b9cd..402da51951 100644
--- a/docs/content/program-api/java-api.md
+++ b/docs/content/program-api/java-api.md
@@ -1,6 +1,6 @@
 ---
 title: "Java API"
-weight: 2
+weight: 3
 type: docs
 aliases:
 - /api/java-api.html
diff --git a/docs/content/program-api/python-api.md 
b/docs/content/program-api/python-api.md
index 0c2813d529..80be8d4efc 100644
--- a/docs/content/program-api/python-api.md
+++ b/docs/content/program-api/python-api.md
@@ -1,6 +1,6 @@
 ---
 title: "Python API"
-weight: 4
+weight: 5
 type: docs
 aliases:
 - /api/python-api.html
diff --git a/docs/content/program-api/rest-api.md 
b/docs/content/program-api/rest-api.md
new file mode 100644
index 0000000000..c4f0725061
--- /dev/null
+++ b/docs/content/program-api/rest-api.md
@@ -0,0 +1,88 @@
+---
+title: "REST API"
+weight: 1
+type: docs
+aliases:
+- /api/rest-api.html
+---
+<!--
+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.
+-->
+
+# REST API
+
+This is Java API for [REST]({{< ref "concepts/rest/overview" >}}).
+
+## Dependency
+
+Maven dependency:
+
+```xml
+<dependency>
+  <groupId>org.apache.paimon</groupId>
+  <artifactId>paimon-api</artifactId>
+  <version>{{< version >}}</version>
+</dependency>
+```
+
+Or download the jar file:
+{{< stable >}}[Paimon 
API](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-api/{{< 
version >}}/paimon-api-{{< version >}}.jar).{{< /stable >}}
+{{< unstable >}}[Paimon 
API](https://repository.apache.org/snapshots/org/apache/paimon/paimon-api/{{< 
version >}}/).{{< /unstable >}}
+
+## RESTApi
+
+```java
+import org.apache.paimon.options.Options;
+import org.apache.paimon.rest.RESTApi;
+
+import java.util.List;
+
+import static org.apache.paimon.options.CatalogOptions.WAREHOUSE;
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_ID;
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_SECRET;
+import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN;
+import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN_PROVIDER;
+import static org.apache.paimon.rest.RESTCatalogOptions.URI;
+
+public class RESTApiExample {
+
+    public static void main(String[] args) {
+        Options options = new Options();
+        options.set(URI, "<catalog server url>");
+        options.set(WAREHOUSE, "my_instance_name");
+        setBearToken(options); // or setDlfToken
+
+        RESTApi api = new RESTApi(options);
+        List<String> tables = api.listTables("my_database");
+        System.out.println(tables);
+    }
+
+    private static void setBearToken(Options options) {
+        options.set(TOKEN_PROVIDER, "bear");
+        options.set(TOKEN, "<token>");
+    }
+
+    private static void setDlfToken(Options options) {
+        options.set(TOKEN_PROVIDER, "dlf");
+        options.set(DLF_ACCESS_KEY_ID, "<access-key-id>");
+        options.set(DLF_ACCESS_KEY_SECRET, "<access-key-secret>");
+    }
+}
+```
+
+See more methods in `'RESTApi'`.
diff --git a/paimon-api/README.md b/paimon-api/README.md
index e347cbb122..7aec8acd44 100644
--- a/paimon-api/README.md
+++ b/paimon-api/README.md
@@ -1,7 +1,7 @@
 ## Paimon API
 
-This module is for light API, aims to simplify dependencies as much as 
possible and avoid
+This module is for light SDK, aims to simplify dependencies as much as 
possible and avoid
 introducing dependencies such as Hadoop, including:
 
-1. Including types, table, view, function and etc.
-2. Including http client REST API.
+1. Including types, table, view, function etc.
+2. Including http client REST API, See `'RESTApi'`.
diff --git a/paimon-common/src/main/java/org/apache/paimon/PagedList.java 
b/paimon-api/src/main/java/org/apache/paimon/PagedList.java
similarity index 99%
rename from paimon-common/src/main/java/org/apache/paimon/PagedList.java
rename to paimon-api/src/main/java/org/apache/paimon/PagedList.java
index 77f78118ab..ca39c92aaa 100644
--- a/paimon-common/src/main/java/org/apache/paimon/PagedList.java
+++ b/paimon-api/src/main/java/org/apache/paimon/PagedList.java
@@ -28,6 +28,7 @@ import java.util.List;
  * @since 1.1.0
  */
 public class PagedList<T> {
+
     private final List<T> elements;
 
     @Nullable private final String nextPageToken;
diff --git a/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java
index 12f7f2938a..f3b6c6a685 100644
--- a/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java
+++ b/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java
@@ -47,7 +47,6 @@ import static okhttp3.ConnectionSpec.COMPATIBLE_TLS;
 import static okhttp3.ConnectionSpec.MODERN_TLS;
 import static org.apache.paimon.rest.LoggingInterceptor.DEFAULT_REQUEST_ID;
 import static org.apache.paimon.rest.LoggingInterceptor.REQUEST_ID_KEY;
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
 
 /** HTTP client for REST catalog. */
 public class HttpClient implements RESTClient {
@@ -125,7 +124,7 @@ public class HttpClient implements RESTClient {
             Class<T> responseType,
             RESTAuthFunction restAuthFunction) {
         try {
-            String bodyStr = OBJECT_MAPPER.writeValueAsString(body);
+            String bodyStr = RESTApi.toJson(body);
             Map<String, String> authHeaders = getHeaders(path, "POST", 
bodyStr, restAuthFunction);
             RequestBody requestBody = buildRequestBody(bodyStr);
             Request request =
@@ -156,7 +155,7 @@ public class HttpClient implements RESTClient {
     public <T extends RESTResponse> T delete(
             String path, RESTRequest body, RESTAuthFunction restAuthFunction) {
         try {
-            String bodyStr = OBJECT_MAPPER.writeValueAsString(body);
+            String bodyStr = RESTApi.toJson(body);
             Map<String, String> authHeaders = getHeaders(path, "DELETE", 
bodyStr, restAuthFunction);
             RequestBody requestBody = buildRequestBody(bodyStr);
             Request request =
@@ -190,7 +189,7 @@ public class HttpClient implements RESTClient {
             if (!response.isSuccessful()) {
                 ErrorResponse error;
                 try {
-                    error = OBJECT_MAPPER.readValue(responseBodyStr, 
ErrorResponse.class);
+                    error = RESTApi.fromJson(responseBodyStr, 
ErrorResponse.class);
                 } catch (JsonProcessingException e) {
                     error =
                             new ErrorResponse(
@@ -205,7 +204,7 @@ public class HttpClient implements RESTClient {
                 errorHandler.accept(error, requestId);
             }
             if (responseType != null && responseBodyStr != null) {
-                return OBJECT_MAPPER.readValue(responseBodyStr, responseType);
+                return RESTApi.fromJson(responseBodyStr, responseType);
             } else if (responseType == null) {
                 return null;
             } else {
diff --git a/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java
new file mode 100644
index 0000000000..50b734fbe2
--- /dev/null
+++ b/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java
@@ -0,0 +1,572 @@
+/*
+ * 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;
+
+import org.apache.paimon.PagedList;
+import org.apache.paimon.Snapshot;
+import org.apache.paimon.annotation.Public;
+import org.apache.paimon.annotation.VisibleForTesting;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.function.FunctionChange;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.partition.Partition;
+import org.apache.paimon.partition.PartitionStatistics;
+import org.apache.paimon.rest.auth.AuthProvider;
+import org.apache.paimon.rest.auth.RESTAuthFunction;
+import org.apache.paimon.rest.requests.AlterDatabaseRequest;
+import org.apache.paimon.rest.requests.AlterFunctionRequest;
+import org.apache.paimon.rest.requests.AlterTableRequest;
+import org.apache.paimon.rest.requests.AlterViewRequest;
+import org.apache.paimon.rest.requests.AuthTableQueryRequest;
+import org.apache.paimon.rest.requests.CommitTableRequest;
+import org.apache.paimon.rest.requests.CreateBranchRequest;
+import org.apache.paimon.rest.requests.CreateDatabaseRequest;
+import org.apache.paimon.rest.requests.CreateFunctionRequest;
+import org.apache.paimon.rest.requests.CreateTableRequest;
+import org.apache.paimon.rest.requests.CreateViewRequest;
+import org.apache.paimon.rest.requests.ForwardBranchRequest;
+import org.apache.paimon.rest.requests.MarkDonePartitionsRequest;
+import org.apache.paimon.rest.requests.RenameTableRequest;
+import org.apache.paimon.rest.requests.RollbackTableRequest;
+import org.apache.paimon.rest.responses.AlterDatabaseResponse;
+import org.apache.paimon.rest.responses.CommitTableResponse;
+import org.apache.paimon.rest.responses.ConfigResponse;
+import org.apache.paimon.rest.responses.GetDatabaseResponse;
+import org.apache.paimon.rest.responses.GetFunctionResponse;
+import org.apache.paimon.rest.responses.GetTableResponse;
+import org.apache.paimon.rest.responses.GetTableSnapshotResponse;
+import org.apache.paimon.rest.responses.GetTableTokenResponse;
+import org.apache.paimon.rest.responses.GetViewResponse;
+import org.apache.paimon.rest.responses.ListBranchesResponse;
+import org.apache.paimon.rest.responses.ListDatabasesResponse;
+import org.apache.paimon.rest.responses.ListFunctionsResponse;
+import org.apache.paimon.rest.responses.ListPartitionsResponse;
+import org.apache.paimon.rest.responses.ListTableDetailsResponse;
+import org.apache.paimon.rest.responses.ListTablesResponse;
+import org.apache.paimon.rest.responses.ListViewDetailsResponse;
+import org.apache.paimon.rest.responses.ListViewsResponse;
+import org.apache.paimon.rest.responses.PagedResponse;
+import org.apache.paimon.schema.Schema;
+import org.apache.paimon.schema.SchemaChange;
+import org.apache.paimon.table.Instant;
+import org.apache.paimon.table.TableSnapshot;
+import org.apache.paimon.utils.JsonSerdeUtil;
+import org.apache.paimon.utils.StringUtils;
+import org.apache.paimon.view.ViewChange;
+import org.apache.paimon.view.ViewSchema;
+
+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.core.JsonProcessingException;
+import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+
+import static java.util.Collections.emptyList;
+import static org.apache.paimon.options.CatalogOptions.WAREHOUSE;
+import static org.apache.paimon.rest.RESTUtil.extractPrefixMap;
+import static 
org.apache.paimon.rest.auth.AuthProviderFactory.createAuthProvider;
+
+/**
+ * REST API for REST Catalog.
+ *
+ * @since 1.2.0
+ */
+@Public
+public class RESTApi {
+
+    public static final String HEADER_PREFIX = "header.";
+    public static final String MAX_RESULTS = "maxResults";
+    public static final String PAGE_TOKEN = "pageToken";
+
+    public static final String TABLE_NAME_PATTERN = "tableNamePattern";
+    public static final String VIEW_NAME_PATTERN = "viewNamePattern";
+    public static final String PARTITION_NAME_PATTERN = "partitionNamePattern";
+
+    public static final long TOKEN_EXPIRATION_SAFE_TIME_MILLIS = 3_600_000L;
+
+    public static final ObjectMapper OBJECT_MAPPER = 
JsonSerdeUtil.OBJECT_MAPPER_INSTANCE;
+
+    private final HttpClient client;
+    private final RESTAuthFunction restAuthFunction;
+    private final Options options;
+    private final ResourcePaths resourcePaths;
+
+    public RESTApi(Options options) {
+        this(options, true);
+    }
+
+    public RESTApi(Options options, boolean configRequired) {
+        this.client = new HttpClient(options.get(RESTCatalogOptions.URI));
+        AuthProvider authProvider = createAuthProvider(options);
+        Map<String, String> baseHeaders = Collections.emptyMap();
+        if (configRequired) {
+            String warehouse = options.get(WAREHOUSE);
+            Map<String, String> queryParams =
+                    StringUtils.isNotEmpty(warehouse)
+                            ? ImmutableMap.of(WAREHOUSE.key(), 
RESTUtil.encodeString(warehouse))
+                            : ImmutableMap.of();
+            baseHeaders = extractPrefixMap(options, HEADER_PREFIX);
+            options =
+                    new Options(
+                            client.get(
+                                            ResourcePaths.config(),
+                                            queryParams,
+                                            ConfigResponse.class,
+                                            new RESTAuthFunction(
+                                                    Collections.emptyMap(), 
authProvider))
+                                    .merge(options.toMap()));
+            baseHeaders.putAll(extractPrefixMap(options, HEADER_PREFIX));
+        }
+        this.restAuthFunction = new RESTAuthFunction(baseHeaders, 
authProvider);
+        this.options = options;
+        this.resourcePaths = ResourcePaths.forCatalogProperties(options);
+    }
+
+    public Options options() {
+        return options;
+    }
+
+    public List<String> listDatabases() {
+        return listDataFromPageApi(
+                queryParams ->
+                        client.get(
+                                resourcePaths.databases(),
+                                queryParams,
+                                ListDatabasesResponse.class,
+                                restAuthFunction));
+    }
+
+    public PagedList<String> listDatabasesPaged(
+            @Nullable Integer maxResults, @Nullable String pageToken) {
+        ListDatabasesResponse response =
+                client.get(
+                        resourcePaths.databases(),
+                        buildPagedQueryParams(maxResults, pageToken),
+                        ListDatabasesResponse.class,
+                        restAuthFunction);
+        List<String> databases = response.getDatabases();
+        if (databases == null) {
+            return new PagedList<>(emptyList(), null);
+        }
+        return new PagedList<>(databases, response.getNextPageToken());
+    }
+
+    public void createDatabase(String name, Map<String, String> properties) {
+        CreateDatabaseRequest request = new CreateDatabaseRequest(name, 
properties);
+        client.post(resourcePaths.databases(), request, restAuthFunction);
+    }
+
+    public GetDatabaseResponse getDatabase(String name) {
+        return client.get(
+                resourcePaths.database(name), GetDatabaseResponse.class, 
restAuthFunction);
+    }
+
+    public void dropDatabase(String name) {
+        client.delete(resourcePaths.database(name), restAuthFunction);
+    }
+
+    public void alterDatabase(String name, List<String> removals, Map<String, 
String> updates) {
+        client.post(
+                resourcePaths.database(name),
+                new AlterDatabaseRequest(removals, updates),
+                AlterDatabaseResponse.class,
+                restAuthFunction);
+    }
+
+    public List<String> listTables(String databaseName) {
+        return listDataFromPageApi(
+                queryParams ->
+                        client.get(
+                                resourcePaths.tables(databaseName),
+                                queryParams,
+                                ListTablesResponse.class,
+                                restAuthFunction));
+    }
+
+    public PagedList<String> listTablesPaged(
+            String databaseName,
+            @Nullable Integer maxResults,
+            @Nullable String pageToken,
+            @Nullable String tableNamePattern) {
+        ListTablesResponse response =
+                client.get(
+                        resourcePaths.tables(databaseName),
+                        buildPagedQueryParams(
+                                maxResults, pageToken, TABLE_NAME_PATTERN, 
tableNamePattern),
+                        ListTablesResponse.class,
+                        restAuthFunction);
+        List<String> tables = response.getTables();
+        if (tables == null) {
+            return new PagedList<>(emptyList(), null);
+        }
+        return new PagedList<>(tables, response.getNextPageToken());
+    }
+
+    public PagedList<GetTableResponse> listTableDetailsPaged(
+            String db,
+            @Nullable Integer maxResults,
+            @Nullable String pageToken,
+            @Nullable String tableNamePattern) {
+        ListTableDetailsResponse response =
+                client.get(
+                        resourcePaths.tableDetails(db),
+                        buildPagedQueryParams(
+                                maxResults, pageToken, TABLE_NAME_PATTERN, 
tableNamePattern),
+                        ListTableDetailsResponse.class,
+                        restAuthFunction);
+        List<GetTableResponse> tables = response.getTableDetails();
+        if (tables == null) {
+            return new PagedList<>(emptyList(), null);
+        }
+        return new PagedList<>(tables, response.getNextPageToken());
+    }
+
+    public GetTableResponse getTable(Identifier identifier) {
+        return client.get(
+                resourcePaths.table(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                GetTableResponse.class,
+                restAuthFunction);
+    }
+
+    public TableSnapshot loadSnapshot(Identifier identifier) {
+        GetTableSnapshotResponse response =
+                client.get(
+                        resourcePaths.tableSnapshot(
+                                identifier.getDatabaseName(), 
identifier.getObjectName()),
+                        GetTableSnapshotResponse.class,
+                        restAuthFunction);
+        return response.getSnapshot();
+    }
+
+    public boolean commitSnapshot(
+            Identifier identifier, Snapshot snapshot, 
List<PartitionStatistics> statistics) {
+        CommitTableRequest request = new CommitTableRequest(snapshot, 
statistics);
+        CommitTableResponse response =
+                client.post(
+                        resourcePaths.commitTable(
+                                identifier.getDatabaseName(), 
identifier.getObjectName()),
+                        request,
+                        CommitTableResponse.class,
+                        restAuthFunction);
+        return response.isSuccess();
+    }
+
+    public void rollbackTo(Identifier identifier, Instant instant) {
+        RollbackTableRequest request = new RollbackTableRequest(instant);
+        client.post(
+                resourcePaths.rollbackTable(
+                        identifier.getDatabaseName(), 
identifier.getObjectName()),
+                request,
+                restAuthFunction);
+    }
+
+    public void createTable(Identifier identifier, Schema schema) {
+        CreateTableRequest request = new CreateTableRequest(identifier, 
schema);
+        client.post(resourcePaths.tables(identifier.getDatabaseName()), 
request, restAuthFunction);
+    }
+
+    public void renameTable(Identifier fromTable, Identifier toTable) {
+        RenameTableRequest request = new RenameTableRequest(fromTable, 
toTable);
+        client.post(resourcePaths.renameTable(), request, restAuthFunction);
+    }
+
+    public void alterTable(Identifier identifier, List<SchemaChange> changes) {
+        AlterTableRequest request = new AlterTableRequest(changes);
+        client.post(
+                resourcePaths.table(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                request,
+                restAuthFunction);
+    }
+
+    public void authTableQuery(Identifier identifier, List<String> select, 
List<String> filter) {
+        AuthTableQueryRequest request = new AuthTableQueryRequest(select, 
filter);
+        client.post(
+                resourcePaths.authTable(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                request,
+                restAuthFunction);
+    }
+
+    public void dropTable(Identifier identifier) {
+        client.delete(
+                resourcePaths.table(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                restAuthFunction);
+    }
+
+    public void markDonePartitions(Identifier identifier, List<Map<String, 
String>> partitions) {
+        MarkDonePartitionsRequest request = new 
MarkDonePartitionsRequest(partitions);
+        client.post(
+                resourcePaths.markDonePartitions(
+                        identifier.getDatabaseName(), 
identifier.getObjectName()),
+                request,
+                restAuthFunction);
+    }
+
+    public List<Partition> listPartitions(Identifier identifier) {
+        return listDataFromPageApi(
+                queryParams ->
+                        client.get(
+                                resourcePaths.partitions(
+                                        identifier.getDatabaseName(), 
identifier.getObjectName()),
+                                queryParams,
+                                ListPartitionsResponse.class,
+                                restAuthFunction));
+    }
+
+    public PagedList<Partition> listPartitionsPaged(
+            Identifier identifier,
+            @Nullable Integer maxResults,
+            @Nullable String pageToken,
+            @Nullable String partitionNamePattern) {
+        ListPartitionsResponse response =
+                client.get(
+                        resourcePaths.partitions(
+                                identifier.getDatabaseName(), 
identifier.getObjectName()),
+                        buildPagedQueryParams(
+                                maxResults,
+                                pageToken,
+                                PARTITION_NAME_PATTERN,
+                                partitionNamePattern),
+                        ListPartitionsResponse.class,
+                        restAuthFunction);
+        List<Partition> partitions = response.getPartitions();
+        if (partitions == null) {
+            return new PagedList<>(emptyList(), null);
+        }
+        return new PagedList<>(partitions, response.getNextPageToken());
+    }
+
+    public void createBranch(Identifier identifier, String branch, @Nullable 
String fromTag) {
+        CreateBranchRequest request = new CreateBranchRequest(branch, fromTag);
+        client.post(
+                resourcePaths.branches(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                request,
+                restAuthFunction);
+    }
+
+    public void dropBranch(Identifier identifier, String branch) {
+        client.delete(
+                resourcePaths.branch(
+                        identifier.getDatabaseName(), 
identifier.getObjectName(), branch),
+                restAuthFunction);
+    }
+
+    public void fastForward(Identifier identifier, String branch) {
+        ForwardBranchRequest request = new ForwardBranchRequest();
+        client.post(
+                resourcePaths.forwardBranch(
+                        identifier.getDatabaseName(), 
identifier.getObjectName(), branch),
+                request,
+                restAuthFunction);
+    }
+
+    public List<String> listBranches(Identifier identifier) {
+        ListBranchesResponse response =
+                client.get(
+                        resourcePaths.branches(
+                                identifier.getDatabaseName(), 
identifier.getObjectName()),
+                        ListBranchesResponse.class,
+                        restAuthFunction);
+        if (response.branches() == null) {
+            return emptyList();
+        }
+        return response.branches();
+    }
+
+    public List<String> listFunctions() {
+        return listDataFromPageApi(
+                queryParams ->
+                        client.get(
+                                resourcePaths.functions(),
+                                queryParams,
+                                ListFunctionsResponse.class,
+                                restAuthFunction));
+    }
+
+    public GetFunctionResponse getFunction(String functionName) {
+        return client.get(
+                resourcePaths.function(functionName), 
GetFunctionResponse.class, restAuthFunction);
+    }
+
+    public void createFunction(String functionName, 
org.apache.paimon.function.Function function) {
+        client.post(
+                resourcePaths.functions(), new 
CreateFunctionRequest(function), restAuthFunction);
+    }
+
+    public void dropFunction(String functionName) {
+        client.delete(resourcePaths.function(functionName), restAuthFunction);
+    }
+
+    public void alterFunction(String functionName, List<FunctionChange> 
changes) {
+        client.post(
+                resourcePaths.function(functionName),
+                new AlterFunctionRequest(changes),
+                restAuthFunction);
+    }
+
+    public GetViewResponse getView(Identifier identifier) {
+        return client.get(
+                resourcePaths.view(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                GetViewResponse.class,
+                restAuthFunction);
+    }
+
+    public void dropView(Identifier identifier) {
+        client.delete(
+                resourcePaths.view(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                restAuthFunction);
+    }
+
+    public void createView(Identifier identifier, ViewSchema schema) {
+        CreateViewRequest request = new CreateViewRequest(identifier, schema);
+        client.post(resourcePaths.views(identifier.getDatabaseName()), 
request, restAuthFunction);
+    }
+
+    public List<String> listViews(String databaseName) {
+        return listDataFromPageApi(
+                queryParams ->
+                        client.get(
+                                resourcePaths.views(databaseName),
+                                queryParams,
+                                ListViewsResponse.class,
+                                restAuthFunction));
+    }
+
+    public PagedList<String> listViewsPaged(
+            String databaseName,
+            @Nullable Integer maxResults,
+            @Nullable String pageToken,
+            @Nullable String viewNamePattern) {
+        ListViewsResponse response =
+                client.get(
+                        resourcePaths.views(databaseName),
+                        buildPagedQueryParams(
+                                maxResults, pageToken, VIEW_NAME_PATTERN, 
viewNamePattern),
+                        ListViewsResponse.class,
+                        restAuthFunction);
+        List<String> views = response.getViews();
+        if (views == null) {
+            return new PagedList<>(emptyList(), null);
+        }
+        return new PagedList<>(views, response.getNextPageToken());
+    }
+
+    public PagedList<GetViewResponse> listViewDetailsPaged(
+            String db,
+            @Nullable Integer maxResults,
+            @Nullable String pageToken,
+            @Nullable String viewNamePattern) {
+        ListViewDetailsResponse response =
+                client.get(
+                        resourcePaths.viewDetails(db),
+                        buildPagedQueryParams(
+                                maxResults, pageToken, VIEW_NAME_PATTERN, 
viewNamePattern),
+                        ListViewDetailsResponse.class,
+                        restAuthFunction);
+        List<GetViewResponse> views = response.getViewDetails();
+        if (views == null) {
+            return new PagedList<>(emptyList(), null);
+        }
+        return new PagedList<>(views, response.getNextPageToken());
+    }
+
+    public void renameView(Identifier fromView, Identifier toView) {
+        RenameTableRequest request = new RenameTableRequest(fromView, toView);
+        client.post(resourcePaths.renameView(), request, restAuthFunction);
+    }
+
+    public void alterView(Identifier identifier, List<ViewChange> viewChanges) 
{
+        AlterViewRequest request = new AlterViewRequest(viewChanges);
+        client.post(
+                resourcePaths.view(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                request,
+                restAuthFunction);
+    }
+
+    public GetTableTokenResponse loadTableToken(Identifier identifier) {
+        return client.get(
+                resourcePaths.tableToken(identifier.getDatabaseName(), 
identifier.getObjectName()),
+                GetTableTokenResponse.class,
+                restAuthFunction);
+    }
+
+    @VisibleForTesting
+    <T> List<T> listDataFromPageApi(Function<Map<String, String>, 
PagedResponse<T>> pageApi) {
+        List<T> results = new ArrayList<>();
+        Map<String, String> queryParams = Maps.newHashMap();
+        String pageToken = null;
+        do {
+            if (pageToken != null) {
+                queryParams.put(PAGE_TOKEN, pageToken);
+            }
+            PagedResponse<T> response = pageApi.apply(queryParams);
+            pageToken = response.getNextPageToken();
+            if (response.data() != null) {
+                results.addAll(response.data());
+            }
+            if (pageToken == null || response.data() == null || 
response.data().isEmpty()) {
+                break;
+            }
+        } while (StringUtils.isNotEmpty(pageToken));
+        return results;
+    }
+
+    private Map<String, String> buildPagedQueryParams(
+            @Nullable Integer maxResults, @Nullable String pageToken) {
+        return buildPagedQueryParams(maxResults, pageToken, null, null);
+    }
+
+    private Map<String, String> buildPagedQueryParams(
+            @Nullable Integer maxResults,
+            @Nullable String pageToken,
+            @Nullable String namePatternKey,
+            @Nullable String namePatternValue) {
+        Map<String, String> queryParams = Maps.newHashMap();
+        if (Objects.nonNull(maxResults) && maxResults > 0) {
+            queryParams.put(MAX_RESULTS, maxResults.toString());
+        }
+        if (Objects.nonNull(pageToken)) {
+            queryParams.put(PAGE_TOKEN, pageToken);
+        }
+        if (Objects.nonNull(namePatternValue)) {
+            queryParams.put(namePatternKey, namePatternValue);
+        }
+        return queryParams;
+    }
+
+    @VisibleForTesting
+    RESTAuthFunction authFunction() {
+        return restAuthFunction;
+    }
+
+    public static <T> T fromJson(String json, Class<T> clazz) throws 
JsonProcessingException {
+        return OBJECT_MAPPER.readValue(json, clazz);
+    }
+
+    public static <T> String toJson(T t) throws JsonProcessingException {
+        return OBJECT_MAPPER.writeValueAsString(t);
+    }
+}
diff --git 
a/paimon-api/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java
index 8a1f7052ff..c143b8549a 100644
--- a/paimon-api/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java
+++ b/paimon-api/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java
@@ -18,12 +18,15 @@
 
 package org.apache.paimon.rest;
 
-import org.apache.paimon.utils.JsonSerdeUtil;
-
 import 
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
 
-/** Object mapper for REST request and response. */
+/**
+ * Object mapper for REST request and response.
+ *
+ * @deprecated Use {@link RESTApi}
+ */
+@Deprecated
 public class RESTObjectMapper {
 
-    public static final ObjectMapper OBJECT_MAPPER = 
JsonSerdeUtil.OBJECT_MAPPER_INSTANCE;
+    public static final ObjectMapper OBJECT_MAPPER = RESTApi.OBJECT_MAPPER;
 }
diff --git 
a/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
index 9b50bb70ee..94814d6d85 100644
--- a/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
+++ b/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
@@ -32,6 +32,7 @@ import java.time.format.DateTimeFormatter;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.paimon.rest.RESTApi.TOKEN_EXPIRATION_SAFE_TIME_MILLIS;
 import static org.apache.paimon.utils.Preconditions.checkNotNull;
 
 /** Auth provider for <b>Ali CLoud</b> DLF. */
@@ -39,8 +40,6 @@ public class DLFAuthProvider implements AuthProvider {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(DLFAuthProvider.class);
 
-    public static final long TOKEN_EXPIRATION_SAFE_TIME_MILLIS = 3_600_000L;
-
     public static final String DLF_AUTHORIZATION_HEADER_KEY = "Authorization";
     public static final String DLF_CONTENT_MD5_HEADER_KEY = "Content-MD5";
     public static final String DLF_CONTENT_TYPE_KEY = "Content-Type";
diff --git 
a/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFECSTokenLoader.java 
b/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFECSTokenLoader.java
index 3806b045d3..b473cd89e1 100644
--- 
a/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFECSTokenLoader.java
+++ 
b/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFECSTokenLoader.java
@@ -20,6 +20,7 @@ package org.apache.paimon.rest.auth;
 
 import org.apache.paimon.annotation.VisibleForTesting;
 import org.apache.paimon.rest.ExponentialHttpRetryInterceptor;
+import org.apache.paimon.rest.RESTApi;
 
 import okhttp3.OkHttpClient;
 import okhttp3.Request;
@@ -37,7 +38,6 @@ import java.util.Arrays;
 import static okhttp3.ConnectionSpec.CLEARTEXT;
 import static okhttp3.ConnectionSpec.COMPATIBLE_TLS;
 import static okhttp3.ConnectionSpec.MODERN_TLS;
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
 
 /** DLF Token Loader for ECS Metadata Service. */
 public class DLFECSTokenLoader implements DLFTokenLoader {
@@ -86,7 +86,7 @@ public class DLFECSTokenLoader implements DLFTokenLoader {
     private static DLFToken getToken(String url) {
         try {
             String token = getResponseBody(url);
-            return OBJECT_MAPPER.readValue(token, DLFToken.class);
+            return RESTApi.fromJson(token, DLFToken.class);
         } catch (IOException e) {
             throw new UncheckedIOException(e);
         } catch (Exception e) {
diff --git 
a/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
 
b/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
index 9f772a668c..1991524744 100644
--- 
a/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
+++ 
b/paimon-api/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
@@ -18,12 +18,11 @@
 
 package org.apache.paimon.rest.auth;
 
+import org.apache.paimon.rest.RESTApi;
 import org.apache.paimon.utils.FileReadUtils;
 
 import java.io.File;
 
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
-
 /** DLF Token Loader for local file. */
 public class DLFLocalFileTokenLoader implements DLFTokenLoader {
 
@@ -49,7 +48,7 @@ public class DLFLocalFileTokenLoader implements 
DLFTokenLoader {
         while (retry <= 5) {
             try {
                 String tokenStr = FileReadUtils.readFileUtf8(new 
File(tokenFilePath));
-                return OBJECT_MAPPER.readValue(tokenStr, DLFToken.class);
+                return RESTApi.fromJson(tokenStr, DLFToken.class);
             } catch (Exception e) {
                 lastException = e;
             }
diff --git a/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java 
b/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java
index 2581c9ac82..57a14fa4f2 100644
--- a/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java
+++ b/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java
@@ -299,6 +299,10 @@ public class StringUtils {
         return cs == null || cs.length() == 0;
     }
 
+    public static boolean isNotEmpty(CharSequence cs) {
+        return !isEmpty(cs);
+    }
+
     public static String randomNumericString(int len) {
         StringBuilder builder = new StringBuilder();
         ThreadLocalRandom rnd = ThreadLocalRandom.current();
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 5e0b5bf3a5..3c450a15ca 100644
--- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java
+++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java
@@ -31,52 +31,21 @@ import org.apache.paimon.catalog.TableMetadata;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.function.FunctionChange;
-import org.apache.paimon.options.Options;
 import org.apache.paimon.partition.Partition;
 import org.apache.paimon.partition.PartitionStatistics;
-import org.apache.paimon.rest.auth.AuthProvider;
-import org.apache.paimon.rest.auth.RESTAuthFunction;
-import org.apache.paimon.rest.auth.RESTAuthParameter;
 import org.apache.paimon.rest.exceptions.AlreadyExistsException;
 import org.apache.paimon.rest.exceptions.BadRequestException;
 import org.apache.paimon.rest.exceptions.ForbiddenException;
 import org.apache.paimon.rest.exceptions.NoSuchResourceException;
 import org.apache.paimon.rest.exceptions.NotImplementedException;
 import org.apache.paimon.rest.exceptions.ServiceFailureException;
-import org.apache.paimon.rest.requests.AlterDatabaseRequest;
-import org.apache.paimon.rest.requests.AlterFunctionRequest;
-import org.apache.paimon.rest.requests.AlterTableRequest;
-import org.apache.paimon.rest.requests.AlterViewRequest;
-import org.apache.paimon.rest.requests.AuthTableQueryRequest;
-import org.apache.paimon.rest.requests.CommitTableRequest;
-import org.apache.paimon.rest.requests.CreateBranchRequest;
-import org.apache.paimon.rest.requests.CreateDatabaseRequest;
-import org.apache.paimon.rest.requests.CreateFunctionRequest;
-import org.apache.paimon.rest.requests.CreateTableRequest;
-import org.apache.paimon.rest.requests.CreateViewRequest;
-import org.apache.paimon.rest.requests.ForwardBranchRequest;
-import org.apache.paimon.rest.requests.MarkDonePartitionsRequest;
-import org.apache.paimon.rest.requests.RenameTableRequest;
 import org.apache.paimon.rest.requests.RollbackTableRequest;
-import org.apache.paimon.rest.responses.AlterDatabaseResponse;
-import org.apache.paimon.rest.responses.CommitTableResponse;
-import org.apache.paimon.rest.responses.ConfigResponse;
 import org.apache.paimon.rest.responses.ErrorResponse;
 import org.apache.paimon.rest.responses.GetDatabaseResponse;
 import org.apache.paimon.rest.responses.GetFunctionResponse;
 import org.apache.paimon.rest.responses.GetTableResponse;
-import org.apache.paimon.rest.responses.GetTableSnapshotResponse;
 import org.apache.paimon.rest.responses.GetTableTokenResponse;
 import org.apache.paimon.rest.responses.GetViewResponse;
-import org.apache.paimon.rest.responses.ListBranchesResponse;
-import org.apache.paimon.rest.responses.ListDatabasesResponse;
-import org.apache.paimon.rest.responses.ListFunctionsResponse;
-import org.apache.paimon.rest.responses.ListPartitionsResponse;
-import org.apache.paimon.rest.responses.ListTableDetailsResponse;
-import org.apache.paimon.rest.responses.ListTablesResponse;
-import org.apache.paimon.rest.responses.ListViewDetailsResponse;
-import org.apache.paimon.rest.responses.ListViewsResponse;
-import org.apache.paimon.rest.responses.PagedResponse;
 import org.apache.paimon.schema.Schema;
 import org.apache.paimon.schema.SchemaChange;
 import org.apache.paimon.schema.TableSchema;
@@ -91,9 +60,6 @@ import org.apache.paimon.view.ViewChange;
 import org.apache.paimon.view.ViewImpl;
 import org.apache.paimon.view.ViewSchema;
 
-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.commons.lang3.StringUtils;
 
 import javax.annotation.Nullable;
@@ -101,17 +67,13 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
-import java.util.function.Function;
 import java.util.stream.Collectors;
 
-import static java.util.Collections.emptyList;
 import static org.apache.paimon.CoreOptions.BRANCH;
 import static org.apache.paimon.CoreOptions.PATH;
 import static org.apache.paimon.catalog.CatalogUtils.checkNotBranch;
@@ -121,62 +83,23 @@ import static 
org.apache.paimon.catalog.CatalogUtils.isSystemDatabase;
 import static 
org.apache.paimon.catalog.CatalogUtils.listPartitionsFromFileSystem;
 import static org.apache.paimon.catalog.CatalogUtils.validateAutoCreateClose;
 import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE;
-import static org.apache.paimon.options.CatalogOptions.WAREHOUSE;
-import static org.apache.paimon.rest.RESTUtil.extractPrefixMap;
-import static 
org.apache.paimon.rest.auth.AuthProviderFactory.createAuthProvider;
 
 /** A catalog implementation for REST. */
 public class RESTCatalog implements Catalog {
 
-    public static final String HEADER_PREFIX = "header.";
-    public static final String MAX_RESULTS = "maxResults";
-    public static final String PAGE_TOKEN = "pageToken";
-    public static final String QUERY_PARAMETER_WAREHOUSE_KEY = "warehouse";
-
-    public static final String TABLE_NAME_PATTERN = "tableNamePattern";
-    public static final String VIEW_NAME_PATTERN = "viewNamePattern";
-    public static final String PARTITION_NAME_PATTERN = "partitionNamePattern";
-    public static final long TOKEN_EXPIRATION_SAFE_TIME_MILLIS = 3_600_000L;
-
-    private final RESTClient client;
-    private final ResourcePaths resourcePaths;
+    private final RESTApi api;
     private final CatalogContext context;
     private final boolean dataTokenEnabled;
-    private final RESTAuthFunction restAuthFunction;
 
     public RESTCatalog(CatalogContext context) {
         this(context, true);
     }
 
     public RESTCatalog(CatalogContext context, boolean configRequired) {
-        this.client = new 
HttpClient(context.options().get(RESTCatalogOptions.URI));
-        AuthProvider authProvider = createAuthProvider(context.options());
-        Options options = context.options();
-        Map<String, String> baseHeaders = Collections.emptyMap();
-        if (configRequired) {
-            String warehouse = options.get(WAREHOUSE);
-            Map<String, String> queryParams =
-                    StringUtils.isNotEmpty(warehouse)
-                            ? ImmutableMap.of(
-                                    QUERY_PARAMETER_WAREHOUSE_KEY, 
RESTUtil.encodeString(warehouse))
-                            : ImmutableMap.of();
-            baseHeaders = extractPrefixMap(context.options(), HEADER_PREFIX);
-            options =
-                    new Options(
-                            client.get(
-                                            ResourcePaths.config(),
-                                            queryParams,
-                                            ConfigResponse.class,
-                                            new RESTAuthFunction(
-                                                    Collections.emptyMap(), 
authProvider))
-                                    .merge(context.options().toMap()));
-            baseHeaders.putAll(extractPrefixMap(options, HEADER_PREFIX));
-        }
-        this.restAuthFunction = new RESTAuthFunction(baseHeaders, 
authProvider);
-        context = CatalogContext.create(options, context.preferIO(), 
context.fallbackIO());
-        this.context = context;
-        this.resourcePaths = ResourcePaths.forCatalogProperties(options);
-        this.dataTokenEnabled = 
options.get(RESTTokenFileIO.DATA_TOKEN_ENABLED);
+        this.api = new RESTApi(context.options(), configRequired);
+        this.context =
+                CatalogContext.create(api.options(), context.preferIO(), 
context.fallbackIO());
+        this.dataTokenEnabled = 
api.options().get(RESTTokenFileIO.DATA_TOKEN_ENABLED);
     }
 
     @Override
@@ -191,38 +114,21 @@ public class RESTCatalog implements Catalog {
 
     @Override
     public List<String> listDatabases() {
-        return listDataFromPageApi(
-                queryParams ->
-                        client.get(
-                                resourcePaths.databases(),
-                                queryParams,
-                                ListDatabasesResponse.class,
-                                restAuthFunction));
+        return api.listDatabases();
     }
 
     @Override
     public PagedList<String> listDatabasesPaged(
             @Nullable Integer maxResults, @Nullable String pageToken) {
-        ListDatabasesResponse response =
-                client.get(
-                        resourcePaths.databases(),
-                        buildPagedQueryParams(maxResults, pageToken),
-                        ListDatabasesResponse.class,
-                        restAuthFunction);
-        List<String> databases = response.getDatabases();
-        if (databases == null) {
-            return new PagedList<>(emptyList(), null);
-        }
-        return new PagedList<>(databases, response.getNextPageToken());
+        return api.listDatabasesPaged(maxResults, pageToken);
     }
 
     @Override
     public void createDatabase(String name, boolean ignoreIfExists, 
Map<String, String> properties)
             throws DatabaseAlreadyExistException {
         checkNotSystemDatabase(name);
-        CreateDatabaseRequest request = new CreateDatabaseRequest(name, 
properties);
         try {
-            client.post(resourcePaths.databases(), request, restAuthFunction);
+            api.createDatabase(name, properties);
         } catch (AlreadyExistsException e) {
             if (!ignoreIfExists) {
                 throw new DatabaseAlreadyExistException(name);
@@ -240,11 +146,7 @@ public class RESTCatalog implements Catalog {
             return Database.of(name);
         }
         try {
-            GetDatabaseResponse response =
-                    client.get(
-                            resourcePaths.database(name),
-                            GetDatabaseResponse.class,
-                            restAuthFunction);
+            GetDatabaseResponse response = api.getDatabase(name);
             Map<String, String> options = new HashMap<>(response.getOptions());
             options.put(DB_LOCATION_PROP, response.getLocation());
             response.putAuditOptionsTo(options);
@@ -264,7 +166,7 @@ public class RESTCatalog implements Catalog {
             if (!cascade && !this.listTables(name).isEmpty()) {
                 throw new DatabaseNotEmptyException(name);
             }
-            client.delete(resourcePaths.database(name), restAuthFunction);
+            api.dropDatabase(name);
         } catch (NoSuchResourceException | DatabaseNotExistException e) {
             if (!ignoreIfNotExists) {
                 throw new DatabaseNotExistException(name);
@@ -283,13 +185,7 @@ public class RESTCatalog implements Catalog {
                     PropertyChange.getSetPropertiesToRemoveKeys(changes);
             Map<String, String> updateProperties = 
setPropertiesToRemoveKeys.getLeft();
             Set<String> removeKeys = setPropertiesToRemoveKeys.getRight();
-            AlterDatabaseRequest request =
-                    new AlterDatabaseRequest(new ArrayList<>(removeKeys), 
updateProperties);
-            client.post(
-                    resourcePaths.database(name),
-                    request,
-                    AlterDatabaseResponse.class,
-                    restAuthFunction);
+            api.alterDatabase(name, new ArrayList<>(removeKeys), 
updateProperties);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 throw new DatabaseNotExistException(name);
@@ -307,13 +203,7 @@ public class RESTCatalog implements Catalog {
             if (isSystemDatabase(databaseName)) {
                 return SystemTableLoader.loadGlobalTableNames();
             }
-            return listDataFromPageApi(
-                    queryParams ->
-                            client.get(
-                                    resourcePaths.tables(databaseName),
-                                    queryParams,
-                                    ListTablesResponse.class,
-                                    restAuthFunction));
+            return api.listTables(databaseName);
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(databaseName);
         }
@@ -327,18 +217,7 @@ public class RESTCatalog implements Catalog {
             @Nullable String tableNamePattern)
             throws DatabaseNotExistException {
         try {
-            ListTablesResponse response =
-                    client.get(
-                            resourcePaths.tables(databaseName),
-                            buildPagedQueryParams(
-                                    maxResults, pageToken, TABLE_NAME_PATTERN, 
tableNamePattern),
-                            ListTablesResponse.class,
-                            restAuthFunction);
-            List<String> tables = response.getTables();
-            if (tables == null) {
-                return new PagedList<>(emptyList(), null);
-            }
-            return new PagedList<>(tables, response.getNextPageToken());
+            return api.listTablesPaged(databaseName, maxResults, pageToken, 
tableNamePattern);
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(databaseName);
         }
@@ -352,20 +231,13 @@ public class RESTCatalog implements Catalog {
             @Nullable String tableNamePattern)
             throws DatabaseNotExistException {
         try {
-            ListTableDetailsResponse response =
-                    client.get(
-                            resourcePaths.tableDetails(db),
-                            buildPagedQueryParams(
-                                    maxResults, pageToken, TABLE_NAME_PATTERN, 
tableNamePattern),
-                            ListTableDetailsResponse.class,
-                            restAuthFunction);
-            List<GetTableResponse> tables = response.getTableDetails();
-            if (tables == null) {
-                return new PagedList<>(emptyList(), null);
-            }
+            PagedList<GetTableResponse> tables =
+                    api.listTableDetailsPaged(db, maxResults, pageToken, 
tableNamePattern);
             return new PagedList<>(
-                    tables.stream().map(t -> toTable(db, 
t)).collect(Collectors.toList()),
-                    response.getNextPageToken());
+                    tables.getElements().stream()
+                            .map(t -> toTable(db, t))
+                            .collect(Collectors.toList()),
+                    tables.getNextPageToken());
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(db);
         }
@@ -386,14 +258,8 @@ public class RESTCatalog implements Catalog {
     @Override
     public Optional<TableSnapshot> loadSnapshot(Identifier identifier)
             throws TableNotExistException {
-        GetTableSnapshotResponse response;
         try {
-            response =
-                    client.get(
-                            resourcePaths.tableSnapshot(
-                                    identifier.getDatabaseName(), 
identifier.getObjectName()),
-                            GetTableSnapshotResponse.class,
-                            restAuthFunction);
+            return Optional.ofNullable(api.loadSnapshot(identifier));
         } catch (NoSuchResourceException e) {
             if (StringUtils.equals(e.resourceType(), 
ErrorResponse.RESOURCE_TYPE_SNAPSHOT)) {
                 return Optional.empty();
@@ -402,8 +268,6 @@ public class RESTCatalog implements Catalog {
         } catch (ForbiddenException e) {
             throw new TableNoPermissionException(identifier, e);
         }
-
-        return Optional.of(response.getSnapshot());
     }
 
     @Override
@@ -425,17 +289,8 @@ public class RESTCatalog implements Catalog {
     public boolean commitSnapshot(
             Identifier identifier, Snapshot snapshot, 
List<PartitionStatistics> statistics)
             throws TableNotExistException {
-        CommitTableRequest request = new CommitTableRequest(snapshot, 
statistics);
-        CommitTableResponse response;
-
         try {
-            response =
-                    client.post(
-                            resourcePaths.commitTable(
-                                    identifier.getDatabaseName(), 
identifier.getObjectName()),
-                            request,
-                            CommitTableResponse.class,
-                            restAuthFunction);
+            return api.commitSnapshot(identifier, snapshot, statistics);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -443,8 +298,6 @@ public class RESTCatalog implements Catalog {
         } catch (BadRequestException e) {
             throw new IllegalArgumentException(e.getMessage());
         }
-
-        return response.isSuccess();
     }
 
     @Override
@@ -452,11 +305,7 @@ public class RESTCatalog implements Catalog {
             throws Catalog.TableNotExistException {
         RollbackTableRequest request = new RollbackTableRequest(instant);
         try {
-            client.post(
-                    resourcePaths.rollbackTable(
-                            identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    request,
-                    restAuthFunction);
+            api.rollbackTo(identifier, instant);
         } catch (NoSuchResourceException e) {
             if (StringUtils.equals(e.resourceType(), 
ErrorResponse.RESOURCE_TYPE_SNAPSHOT)) {
                 throw new IllegalArgumentException(
@@ -472,24 +321,19 @@ public class RESTCatalog implements Catalog {
     }
 
     private TableMetadata loadTableMetadata(Identifier identifier) throws 
TableNotExistException {
+        // if the table is system table, we need to load table metadata from 
the system table's data
+        // table
+        Identifier loadTableIdentifier =
+                identifier.isSystemTable()
+                        ? new Identifier(
+                                identifier.getDatabaseName(),
+                                identifier.getTableName(),
+                                identifier.getBranchName())
+                        : identifier;
+
         GetTableResponse response;
         try {
-            // if the table is system table, we need to load table metadata 
from the system table's
-            // data table
-            Identifier loadTableIdentifier =
-                    identifier.isSystemTable()
-                            ? new Identifier(
-                                    identifier.getDatabaseName(),
-                                    identifier.getTableName(),
-                                    identifier.getBranchName())
-                            : identifier;
-            response =
-                    client.get(
-                            resourcePaths.table(
-                                    loadTableIdentifier.getDatabaseName(),
-                                    loadTableIdentifier.getObjectName()),
-                            GetTableResponse.class,
-                            restAuthFunction);
+            response = api.getTable(loadTableIdentifier);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -534,10 +378,7 @@ public class RESTCatalog implements Catalog {
             checkNotBranch(identifier, "createTable");
             checkNotSystemTable(identifier, "createTable");
             validateAutoCreateClose(schema.options());
-            CreateTableRequest request = new CreateTableRequest(identifier, 
schema);
-            client.post(
-                    resourcePaths.tables(identifier.getDatabaseName()), 
request, restAuthFunction);
-
+            api.createTable(identifier, schema);
         } catch (AlreadyExistsException e) {
             if (!ignoreIfExists) {
                 throw new TableAlreadyExistException(identifier);
@@ -563,8 +404,7 @@ public class RESTCatalog implements Catalog {
         checkNotSystemTable(fromTable, "renameTable");
         checkNotSystemTable(toTable, "renameTable");
         try {
-            RenameTableRequest request = new RenameTableRequest(fromTable, 
toTable);
-            client.post(resourcePaths.renameTable(), request, 
restAuthFunction);
+            api.renameTable(fromTable, toTable);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 throw new TableNotExistException(fromTable);
@@ -584,11 +424,7 @@ public class RESTCatalog implements Catalog {
             throws TableNotExistException, ColumnAlreadyExistException, 
ColumnNotExistException {
         checkNotSystemTable(identifier, "alterTable");
         try {
-            AlterTableRequest request = new AlterTableRequest(changes);
-            client.post(
-                    resourcePaths.table(identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    request,
-                    restAuthFunction);
+            api.alterTable(identifier, changes);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 if (StringUtils.equals(e.resourceType(), 
ErrorResponse.RESOURCE_TYPE_TABLE)) {
@@ -616,12 +452,7 @@ public class RESTCatalog implements Catalog {
             throws TableNotExistException {
         checkNotSystemTable(identifier, "authTable");
         try {
-            AuthTableQueryRequest request = new AuthTableQueryRequest(select, 
filter);
-            client.post(
-                    resourcePaths.authTable(
-                            identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    request,
-                    restAuthFunction);
+            api.authTableQuery(identifier, select, filter);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -641,9 +472,7 @@ public class RESTCatalog implements Catalog {
         checkNotBranch(identifier, "dropTable");
         checkNotSystemTable(identifier, "dropTable");
         try {
-            client.delete(
-                    resourcePaths.table(identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    restAuthFunction);
+            api.dropTable(identifier);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 throw new TableNotExistException(identifier);
@@ -657,12 +486,7 @@ public class RESTCatalog implements Catalog {
     public void markDonePartitions(Identifier identifier, List<Map<String, 
String>> partitions)
             throws TableNotExistException {
         try {
-            MarkDonePartitionsRequest request = new 
MarkDonePartitionsRequest(partitions);
-            client.post(
-                    resourcePaths.markDonePartitions(
-                            identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    request,
-                    restAuthFunction);
+            api.markDonePartitions(identifier, partitions);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (NotImplementedException ignored) {
@@ -673,15 +497,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public List<Partition> listPartitions(Identifier identifier) throws 
TableNotExistException {
         try {
-            return listDataFromPageApi(
-                    queryParams ->
-                            client.get(
-                                    resourcePaths.partitions(
-                                            identifier.getDatabaseName(),
-                                            identifier.getObjectName()),
-                                    queryParams,
-                                    ListPartitionsResponse.class,
-                                    restAuthFunction));
+            return api.listPartitions(identifier);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -700,22 +516,7 @@ public class RESTCatalog implements Catalog {
             @Nullable String partitionNamePattern)
             throws TableNotExistException {
         try {
-            ListPartitionsResponse response =
-                    client.get(
-                            resourcePaths.partitions(
-                                    identifier.getDatabaseName(), 
identifier.getObjectName()),
-                            buildPagedQueryParams(
-                                    maxResults,
-                                    pageToken,
-                                    PARTITION_NAME_PATTERN,
-                                    partitionNamePattern),
-                            ListPartitionsResponse.class,
-                            restAuthFunction);
-            List<Partition> partitions = response.getPartitions();
-            if (partitions == null) {
-                return new PagedList<>(emptyList(), null);
-            }
-            return new PagedList<>(partitions, response.getNextPageToken());
+            return api.listPartitionsPaged(identifier, maxResults, pageToken, 
partitionNamePattern);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -730,12 +531,7 @@ public class RESTCatalog implements Catalog {
     public void createBranch(Identifier identifier, String branch, @Nullable 
String fromTag)
             throws TableNotExistException, BranchAlreadyExistException, 
TagNotExistException {
         try {
-            CreateBranchRequest request = new CreateBranchRequest(branch, 
fromTag);
-            client.post(
-                    resourcePaths.branches(
-                            identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    request,
-                    restAuthFunction);
+            api.createBranch(identifier, branch, fromTag);
         } catch (NoSuchResourceException e) {
             if (StringUtils.equals(e.resourceType(), 
ErrorResponse.RESOURCE_TYPE_TABLE)) {
                 throw new TableNotExistException(identifier, e);
@@ -756,10 +552,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public void dropBranch(Identifier identifier, String branch) throws 
BranchNotExistException {
         try {
-            client.delete(
-                    resourcePaths.branch(
-                            identifier.getDatabaseName(), 
identifier.getObjectName(), branch),
-                    restAuthFunction);
+            api.dropBranch(identifier, branch);
         } catch (NoSuchResourceException e) {
             throw new BranchNotExistException(identifier, branch, e);
         } catch (ForbiddenException e) {
@@ -770,12 +563,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public void fastForward(Identifier identifier, String branch) throws 
BranchNotExistException {
         try {
-            ForwardBranchRequest request = new ForwardBranchRequest();
-            client.post(
-                    resourcePaths.forwardBranch(
-                            identifier.getDatabaseName(), 
identifier.getObjectName(), branch),
-                    request,
-                    restAuthFunction);
+            api.fastForward(identifier, branch);
         } catch (NoSuchResourceException e) {
             throw new BranchNotExistException(identifier, branch, e);
         } catch (ForbiddenException e) {
@@ -786,16 +574,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public List<String> listBranches(Identifier identifier) throws 
TableNotExistException {
         try {
-            ListBranchesResponse response =
-                    client.get(
-                            resourcePaths.branches(
-                                    identifier.getDatabaseName(), 
identifier.getObjectName()),
-                            ListBranchesResponse.class,
-                            restAuthFunction);
-            if (response.branches() == null) {
-                return emptyList();
-            }
-            return response.branches();
+            return api.listBranches(identifier);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
@@ -830,24 +609,14 @@ public class RESTCatalog implements Catalog {
 
     @Override
     public List<String> listFunctions() {
-        return listDataFromPageApi(
-                queryParams ->
-                        client.get(
-                                resourcePaths.functions(),
-                                queryParams,
-                                ListFunctionsResponse.class,
-                                restAuthFunction));
+        return api.listFunctions();
     }
 
     @Override
     public org.apache.paimon.function.Function getFunction(String functionName)
             throws FunctionNotExistException {
         try {
-            GetFunctionResponse response =
-                    client.get(
-                            resourcePaths.function(functionName),
-                            GetFunctionResponse.class,
-                            restAuthFunction);
+            GetFunctionResponse response = api.getFunction(functionName);
             return response.toFunction();
         } catch (NoSuchResourceException e) {
             throw new FunctionNotExistException(functionName, e);
@@ -861,10 +630,7 @@ public class RESTCatalog implements Catalog {
             boolean ignoreIfExists)
             throws FunctionAlreadyExistException {
         try {
-            client.post(
-                    resourcePaths.functions(),
-                    new CreateFunctionRequest(function),
-                    restAuthFunction);
+            api.createFunction(functionName, function);
         } catch (AlreadyExistsException e) {
             if (ignoreIfExists) {
                 return;
@@ -877,7 +643,7 @@ public class RESTCatalog implements Catalog {
     public void dropFunction(String functionName, boolean ignoreIfNotExists)
             throws FunctionNotExistException {
         try {
-            client.delete(resourcePaths.function(functionName), 
restAuthFunction);
+            api.dropFunction(functionName);
         } catch (NoSuchResourceException e) {
             if (ignoreIfNotExists) {
                 return;
@@ -892,10 +658,7 @@ public class RESTCatalog implements Catalog {
             throws FunctionNotExistException, DefinitionAlreadyExistException,
                     DefinitionNotExistException {
         try {
-            client.post(
-                    resourcePaths.function(functionName),
-                    new AlterFunctionRequest(changes),
-                    restAuthFunction);
+            api.alterFunction(functionName, changes);
         } catch (AlreadyExistsException e) {
             throw new DefinitionAlreadyExistException(functionName, 
e.resourceName());
         } catch (NoSuchResourceException e) {
@@ -913,12 +676,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public View getView(Identifier identifier) throws ViewNotExistException {
         try {
-            GetViewResponse response =
-                    client.get(
-                            resourcePaths.view(
-                                    identifier.getDatabaseName(), 
identifier.getObjectName()),
-                            GetViewResponse.class,
-                            restAuthFunction);
+            GetViewResponse response = api.getView(identifier);
             return toView(identifier.getDatabaseName(), response);
         } catch (NoSuchResourceException e) {
             throw new ViewNotExistException(identifier);
@@ -929,9 +687,7 @@ public class RESTCatalog implements Catalog {
     public void dropView(Identifier identifier, boolean ignoreIfNotExists)
             throws ViewNotExistException {
         try {
-            client.delete(
-                    resourcePaths.view(identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    restAuthFunction);
+            api.dropView(identifier);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 throw new ViewNotExistException(identifier);
@@ -950,9 +706,7 @@ public class RESTCatalog implements Catalog {
                             view.dialects(),
                             view.comment().orElse(null),
                             view.options());
-            CreateViewRequest request = new CreateViewRequest(identifier, 
schema);
-            client.post(
-                    resourcePaths.views(identifier.getDatabaseName()), 
request, restAuthFunction);
+            api.createView(identifier, schema);
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(identifier.getDatabaseName());
         } catch (AlreadyExistsException e) {
@@ -967,13 +721,7 @@ public class RESTCatalog implements Catalog {
     @Override
     public List<String> listViews(String databaseName) throws 
DatabaseNotExistException {
         try {
-            return listDataFromPageApi(
-                    queryParams ->
-                            client.get(
-                                    resourcePaths.views(databaseName),
-                                    queryParams,
-                                    ListViewsResponse.class,
-                                    restAuthFunction));
+            return api.listViews(databaseName);
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(databaseName);
         }
@@ -987,18 +735,7 @@ public class RESTCatalog implements Catalog {
             @Nullable String viewNamePattern)
             throws DatabaseNotExistException {
         try {
-            ListViewsResponse response =
-                    client.get(
-                            resourcePaths.views(databaseName),
-                            buildPagedQueryParams(
-                                    maxResults, pageToken, VIEW_NAME_PATTERN, 
viewNamePattern),
-                            ListViewsResponse.class,
-                            restAuthFunction);
-            List<String> views = response.getViews();
-            if (views == null) {
-                return new PagedList<>(emptyList(), null);
-            }
-            return new PagedList<>(views, response.getNextPageToken());
+            return api.listViewsPaged(databaseName, maxResults, pageToken, 
viewNamePattern);
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(databaseName);
         }
@@ -1012,20 +749,13 @@ public class RESTCatalog implements Catalog {
             @Nullable String viewNamePattern)
             throws DatabaseNotExistException {
         try {
-            ListViewDetailsResponse response =
-                    client.get(
-                            resourcePaths.viewDetails(db),
-                            buildPagedQueryParams(
-                                    maxResults, pageToken, VIEW_NAME_PATTERN, 
viewNamePattern),
-                            ListViewDetailsResponse.class,
-                            restAuthFunction);
-            List<GetViewResponse> views = response.getViewDetails();
-            if (views == null) {
-                return new PagedList<>(emptyList(), null);
-            }
+            PagedList<GetViewResponse> views =
+                    api.listViewDetailsPaged(db, maxResults, pageToken, 
viewNamePattern);
             return new PagedList<>(
-                    views.stream().map(v -> toView(db, 
v)).collect(Collectors.toList()),
-                    response.getNextPageToken());
+                    views.getElements().stream()
+                            .map(v -> toView(db, v))
+                            .collect(Collectors.toList()),
+                    views.getNextPageToken());
         } catch (NoSuchResourceException e) {
             throw new DatabaseNotExistException(db);
         }
@@ -1048,9 +778,7 @@ public class RESTCatalog implements Catalog {
     public void renameView(Identifier fromView, Identifier toView, boolean 
ignoreIfNotExists)
             throws ViewNotExistException, ViewAlreadyExistException {
         try {
-            RenameTableRequest request = new RenameTableRequest(fromView, 
toView);
-            client.post(resourcePaths.renameView(), request, restAuthFunction);
-
+            api.renameView(fromView, toView);
         } catch (NoSuchResourceException e) {
             if (!ignoreIfNotExists) {
                 throw new ViewNotExistException(fromView);
@@ -1067,11 +795,7 @@ public class RESTCatalog implements Catalog {
             Identifier identifier, List<ViewChange> viewChanges, boolean 
ignoreIfNotExists)
             throws ViewNotExistException, DialectAlreadyExistException, 
DialectNotExistException {
         try {
-            AlterViewRequest request = new AlterViewRequest(viewChanges);
-            client.post(
-                    resourcePaths.view(identifier.getDatabaseName(), 
identifier.getObjectName()),
-                    request,
-                    restAuthFunction);
+            api.alterView(identifier, viewChanges);
         } catch (AlreadyExistsException e) {
             throw new DialectAlreadyExistException(identifier, 
e.resourceName());
         } catch (NoSuchResourceException e) {
@@ -1095,47 +819,19 @@ public class RESTCatalog implements Catalog {
     public void close() throws Exception {}
 
     @VisibleForTesting
-    Map<String, String> headers(RESTAuthParameter restAuthParameter) {
-        return restAuthFunction.apply(restAuthParameter);
+    RESTApi api() {
+        return api;
     }
 
     protected GetTableTokenResponse loadTableToken(Identifier identifier)
             throws TableNotExistException {
-        GetTableTokenResponse response;
         try {
-            response =
-                    client.get(
-                            resourcePaths.tableToken(
-                                    identifier.getDatabaseName(), 
identifier.getObjectName()),
-                            GetTableTokenResponse.class,
-                            restAuthFunction);
+            return api.loadTableToken(identifier);
         } catch (NoSuchResourceException e) {
             throw new TableNotExistException(identifier);
         } catch (ForbiddenException e) {
             throw new TableNoPermissionException(identifier, e);
         }
-        return response;
-    }
-
-    protected <T> List<T> listDataFromPageApi(
-            Function<Map<String, String>, PagedResponse<T>> pageApi) {
-        List<T> results = new ArrayList<>();
-        Map<String, String> queryParams = Maps.newHashMap();
-        String pageToken = null;
-        do {
-            if (pageToken != null) {
-                queryParams.put(PAGE_TOKEN, pageToken);
-            }
-            PagedResponse<T> response = pageApi.apply(queryParams);
-            pageToken = response.getNextPageToken();
-            if (response.data() != null) {
-                results.addAll(response.data());
-            }
-            if (pageToken == null || response.data() == null || 
response.data().isEmpty()) {
-                break;
-            }
-        } while (StringUtils.isNotEmpty(pageToken));
-        return results;
     }
 
     private FileIO fileIOForData(Path path, Identifier identifier) {
@@ -1151,27 +847,4 @@ public class RESTCatalog implements Catalog {
             throw new UncheckedIOException(e);
         }
     }
-
-    private Map<String, String> buildPagedQueryParams(
-            @Nullable Integer maxResults, @Nullable String pageToken) {
-        return buildPagedQueryParams(maxResults, pageToken, null, null);
-    }
-
-    private Map<String, String> buildPagedQueryParams(
-            @Nullable Integer maxResults,
-            @Nullable String pageToken,
-            @Nullable String namePatternKey,
-            @Nullable String namePatternValue) {
-        Map<String, String> queryParams = Maps.newHashMap();
-        if (Objects.nonNull(maxResults) && maxResults > 0) {
-            queryParams.put(MAX_RESULTS, maxResults.toString());
-        }
-        if (Objects.nonNull(pageToken)) {
-            queryParams.put(PAGE_TOKEN, pageToken);
-        }
-        if (Objects.nonNull(namePatternValue)) {
-            queryParams.put(namePatternKey, namePatternValue);
-        }
-        return queryParams;
-    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java 
b/paimon-core/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java
index 13e31e8634..2151bd7a8a 100644
--- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java
+++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java
@@ -46,7 +46,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.paimon.options.CatalogOptions.FILE_IO_ALLOW_CACHE;
-import static 
org.apache.paimon.rest.RESTCatalog.TOKEN_EXPIRATION_SAFE_TIME_MILLIS;
+import static org.apache.paimon.rest.RESTApi.TOKEN_EXPIRATION_SAFE_TIME_MILLIS;
 
 /** A {@link FileIO} to support getting token from REST Server. */
 public class RESTTokenFileIO implements FileIO {
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
index 4416008372..01161fd0f6 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
@@ -142,7 +142,7 @@ class MockRESTCatalogTest extends RESTCatalogTest {
         parameters.put("k2", "v2");
         RESTAuthParameter restAuthParameter =
                 new RESTAuthParameter("/path", parameters, "method", "data");
-        Map<String, String> headers = restCatalog.headers(restAuthParameter);
+        Map<String, String> headers = 
restCatalog.api().authFunction().apply(restAuthParameter);
         assertEquals(
                 headers.get(BearTokenAuthProvider.AUTHORIZATION_HEADER_KEY), 
"Bearer init_token");
         assertEquals(headers.get(serverDefineHeaderName), 
serverDefineHeaderValue);
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/RESTApiJsonTest.java
similarity index 71%
rename from 
paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java
rename to paimon-core/src/test/java/org/apache/paimon/rest/RESTApiJsonTest.java
index 9a36191d09..10aea7f3b4 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTApiJsonTest.java
@@ -52,12 +52,11 @@ import org.junit.Test;
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-/** Test for {@link RESTObjectMapper}. */
-public class RESTObjectMapperTest {
+/** Test for {@link RESTApi} json. */
+public class RESTApiJsonTest {
 
     @Test
     public void configResponseParseTest() throws Exception {
@@ -65,8 +64,8 @@ public class RESTObjectMapperTest {
         Map<String, String> conf = new HashMap<>();
         conf.put(confKey, "b");
         ConfigResponse response = new ConfigResponse(conf, conf);
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        ConfigResponse parseData = OBJECT_MAPPER.readValue(responseStr, 
ConfigResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        ConfigResponse parseData = RESTApi.fromJson(responseStr, 
ConfigResponse.class);
         assertEquals(conf.get(confKey), parseData.getDefaults().get(confKey));
     }
 
@@ -75,8 +74,8 @@ public class RESTObjectMapperTest {
         String message = "message";
         Integer code = 400;
         ErrorResponse response = new ErrorResponse(null, null, message, code);
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        ErrorResponse parseData = OBJECT_MAPPER.readValue(responseStr, 
ErrorResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        ErrorResponse parseData = RESTApi.fromJson(responseStr, 
ErrorResponse.class);
         assertEquals(message, parseData.getMessage());
         assertEquals(code, parseData.getCode());
     }
@@ -85,9 +84,8 @@ public class RESTObjectMapperTest {
     public void createDatabaseRequestParseTest() throws Exception {
         String name = MockRESTMessage.databaseName();
         CreateDatabaseRequest request = 
MockRESTMessage.createDatabaseRequest(name);
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        CreateDatabaseRequest parseData =
-                OBJECT_MAPPER.readValue(requestStr, 
CreateDatabaseRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        CreateDatabaseRequest parseData = RESTApi.fromJson(requestStr, 
CreateDatabaseRequest.class);
         assertEquals(request.getName(), parseData.getName());
         assertEquals(request.getOptions().size(), 
parseData.getOptions().size());
     }
@@ -96,9 +94,8 @@ public class RESTObjectMapperTest {
     public void getDatabaseResponseParseTest() throws Exception {
         String name = MockRESTMessage.databaseName();
         GetDatabaseResponse response = 
MockRESTMessage.getDatabaseResponse(name);
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        GetDatabaseResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, 
GetDatabaseResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        GetDatabaseResponse parseData = RESTApi.fromJson(responseStr, 
GetDatabaseResponse.class);
         assertEquals(name, parseData.getName());
         assertEquals(response.getOptions().size(), 
parseData.getOptions().size());
     }
@@ -107,9 +104,9 @@ public class RESTObjectMapperTest {
     public void listDatabaseResponseParseTest() throws Exception {
         String name = MockRESTMessage.databaseName();
         ListDatabasesResponse response = 
MockRESTMessage.listDatabasesResponse(name);
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
+        String responseStr = RESTApi.toJson(response);
         ListDatabasesResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, 
ListDatabasesResponse.class);
+                RESTApi.fromJson(responseStr, ListDatabasesResponse.class);
         assertEquals(response.getDatabases().size(), 
parseData.getDatabases().size());
         assertEquals(name, parseData.getDatabases().get(0));
     }
@@ -117,9 +114,8 @@ public class RESTObjectMapperTest {
     @Test
     public void alterDatabaseRequestParseTest() throws Exception {
         AlterDatabaseRequest request = MockRESTMessage.alterDatabaseRequest();
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        AlterDatabaseRequest parseData =
-                OBJECT_MAPPER.readValue(requestStr, 
AlterDatabaseRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        AlterDatabaseRequest parseData = RESTApi.fromJson(requestStr, 
AlterDatabaseRequest.class);
         assertEquals(request.getRemovals().size(), 
parseData.getRemovals().size());
         assertEquals(request.getUpdates().size(), 
parseData.getUpdates().size());
     }
@@ -127,9 +123,9 @@ public class RESTObjectMapperTest {
     @Test
     public void alterDatabaseResponseParseTest() throws Exception {
         AlterDatabaseResponse response = 
MockRESTMessage.alterDatabaseResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
+        String responseStr = RESTApi.toJson(response);
         AlterDatabaseResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, 
AlterDatabaseResponse.class);
+                RESTApi.fromJson(responseStr, AlterDatabaseResponse.class);
         assertEquals(response.getRemoved().size(), 
parseData.getRemoved().size());
         assertEquals(response.getUpdated().size(), 
parseData.getUpdated().size());
         assertEquals(response.getMissing().size(), 
parseData.getMissing().size());
@@ -138,9 +134,8 @@ public class RESTObjectMapperTest {
     @Test
     public void createTableRequestParseTest() throws Exception {
         CreateTableRequest request = MockRESTMessage.createTableRequest("t1");
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        CreateTableRequest parseData =
-                OBJECT_MAPPER.readValue(requestStr, CreateTableRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        CreateTableRequest parseData = RESTApi.fromJson(requestStr, 
CreateTableRequest.class);
         assertEquals(request.getIdentifier(), parseData.getIdentifier());
         assertEquals(request.getSchema(), parseData.getSchema());
     }
@@ -156,7 +151,7 @@ public class RESTObjectMapperTest {
                 String.format(
                         "{\"id\": %d,\"name\":\"%s\",\"type\":\"%s\", 
\"description\":\"%s\"}",
                         id, name, type, descStr);
-        DataField parseData = OBJECT_MAPPER.readValue(dataFieldStr, 
DataField.class);
+        DataField parseData = RESTApi.fromJson(dataFieldStr, DataField.class);
         assertEquals(id, parseData.id());
         assertEquals(name, parseData.name());
         assertEquals(type, parseData.type());
@@ -166,9 +161,8 @@ public class RESTObjectMapperTest {
     @Test
     public void renameTableRequestParseTest() throws Exception {
         RenameTableRequest request = MockRESTMessage.renameRequest("t1", "t2");
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        RenameTableRequest parseData =
-                OBJECT_MAPPER.readValue(requestStr, RenameTableRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        RenameTableRequest parseData = RESTApi.fromJson(requestStr, 
RenameTableRequest.class);
         assertEquals(request.getSource(), parseData.getSource());
         assertEquals(request.getDestination(), parseData.getDestination());
     }
@@ -176,8 +170,8 @@ public class RESTObjectMapperTest {
     @Test
     public void getTableResponseParseTest() throws Exception {
         GetTableResponse response = MockRESTMessage.getTableResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        GetTableResponse parseData = OBJECT_MAPPER.readValue(responseStr, 
GetTableResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        GetTableResponse parseData = RESTApi.fromJson(responseStr, 
GetTableResponse.class);
         assertEquals(response.getSchemaId(), parseData.getSchemaId());
         assertEquals(response.getSchema(), parseData.getSchema());
     }
@@ -185,26 +179,25 @@ public class RESTObjectMapperTest {
     @Test
     public void listTablesResponseParseTest() throws Exception {
         ListTablesResponse response = MockRESTMessage.listTablesResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        ListTablesResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, ListTablesResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        ListTablesResponse parseData = RESTApi.fromJson(responseStr, 
ListTablesResponse.class);
         assertEquals(response.getTables(), parseData.getTables());
     }
 
     @Test
     public void alterTableRequestParseTest() throws Exception {
         AlterTableRequest request = MockRESTMessage.alterTableRequest();
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        AlterTableRequest parseData = OBJECT_MAPPER.readValue(requestStr, 
AlterTableRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        AlterTableRequest parseData = RESTApi.fromJson(requestStr, 
AlterTableRequest.class);
         assertEquals(parseData.getChanges().size(), 
parseData.getChanges().size());
     }
 
     @Test
     public void listPartitionsResponseParseTest() throws Exception {
         ListPartitionsResponse response = 
MockRESTMessage.listPartitionsResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
+        String responseStr = RESTApi.toJson(response);
         ListPartitionsResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, 
ListPartitionsResponse.class);
+                RESTApi.fromJson(responseStr, ListPartitionsResponse.class);
         assertEquals(
                 response.getPartitions().get(0).fileCount(),
                 parseData.getPartitions().get(0).fileCount());
@@ -213,8 +206,8 @@ public class RESTObjectMapperTest {
     @Test
     public void createViewRequestParseTest() throws Exception {
         CreateViewRequest request = MockRESTMessage.createViewRequest("t1");
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        CreateViewRequest parseData = OBJECT_MAPPER.readValue(requestStr, 
CreateViewRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        CreateViewRequest parseData = RESTApi.fromJson(requestStr, 
CreateViewRequest.class);
         assertEquals(request.getIdentifier(), parseData.getIdentifier());
         assertEquals(request.getSchema(), parseData.getSchema());
     }
@@ -222,8 +215,8 @@ public class RESTObjectMapperTest {
     @Test
     public void getViewResponseParseTest() throws Exception {
         GetViewResponse response = MockRESTMessage.getViewResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        GetViewResponse parseData = OBJECT_MAPPER.readValue(responseStr, 
GetViewResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        GetViewResponse parseData = RESTApi.fromJson(responseStr, 
GetViewResponse.class);
         assertEquals(response.getId(), parseData.getId());
         assertEquals(response.getName(), parseData.getName());
         assertEquals(response.getSchema(), parseData.getSchema());
@@ -232,17 +225,17 @@ public class RESTObjectMapperTest {
     @Test
     public void listViewsResponseParseTest() throws Exception {
         ListViewsResponse response = MockRESTMessage.listViewsResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        ListViewsResponse parseData = OBJECT_MAPPER.readValue(responseStr, 
ListViewsResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        ListViewsResponse parseData = RESTApi.fromJson(responseStr, 
ListViewsResponse.class);
         assertEquals(response.getViews(), parseData.getViews());
     }
 
     @Test
     public void getTableTokenResponseParseTest() throws Exception {
         GetTableTokenResponse response = 
MockRESTMessage.getTableCredentialsResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
+        String responseStr = RESTApi.toJson(response);
         GetTableTokenResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, 
GetTableTokenResponse.class);
+                RESTApi.fromJson(responseStr, GetTableTokenResponse.class);
         assertEquals(response.getToken(), parseData.getToken());
         assertEquals(response.getExpiresAtMillis(), 
parseData.getExpiresAtMillis());
     }
@@ -253,24 +246,20 @@ public class RESTObjectMapperTest {
         String tagName = "tagName";
         RollbackTableRequest rollbackTableRequestBySnapshot =
                 MockRESTMessage.rollbackTableRequestBySnapshot(snapshotId);
-        String rollbackTableRequestBySnapshotStr =
-                
OBJECT_MAPPER.writeValueAsString(rollbackTableRequestBySnapshot);
+        String rollbackTableRequestBySnapshotStr = 
RESTApi.toJson(rollbackTableRequestBySnapshot);
         Instant.SnapshotInstant rollbackTableRequestParseData =
                 (Instant.SnapshotInstant)
-                        OBJECT_MAPPER
-                                .readValue(
+                        RESTApi.fromJson(
                                         rollbackTableRequestBySnapshotStr,
                                         RollbackTableRequest.class)
                                 .getInstant();
         assertTrue(rollbackTableRequestParseData.getSnapshotId() == 
snapshotId);
         RollbackTableRequest rollbackTableRequestByTag =
                 MockRESTMessage.rollbackTableRequestByTag(tagName);
-        String rollbackTableRequestByTagStr =
-                OBJECT_MAPPER.writeValueAsString(rollbackTableRequestByTag);
+        String rollbackTableRequestByTagStr = 
RESTApi.toJson(rollbackTableRequestByTag);
         Instant.TagInstant rollbackTableRequestByTagParseData =
                 (Instant.TagInstant)
-                        OBJECT_MAPPER
-                                .readValue(rollbackTableRequestByTagStr, 
RollbackTableRequest.class)
+                        RESTApi.fromJson(rollbackTableRequestByTagStr, 
RollbackTableRequest.class)
                                 .getInstant();
         assertEquals(rollbackTableRequestByTagParseData.getTagName(), tagName);
     }
@@ -278,8 +267,8 @@ public class RESTObjectMapperTest {
     @Test
     public void alterViewRequestParseTest() throws Exception {
         AlterViewRequest request = MockRESTMessage.alterViewRequest();
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        AlterViewRequest parseData = OBJECT_MAPPER.readValue(requestStr, 
AlterViewRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        AlterViewRequest parseData = RESTApi.fromJson(requestStr, 
AlterViewRequest.class);
         assertEquals(parseData.viewChanges().size(), 
request.viewChanges().size());
         for (int i = 0; i < request.viewChanges().size(); i++) {
             assertEquals(parseData.viewChanges().get(i), 
request.viewChanges().get(i));
@@ -289,27 +278,24 @@ public class RESTObjectMapperTest {
     @Test
     public void getFunctionResponseParseTest() throws Exception {
         GetFunctionResponse response = MockRESTMessage.getFunctionResponse();
-        String responseStr = OBJECT_MAPPER.writeValueAsString(response);
-        GetFunctionResponse parseData =
-                OBJECT_MAPPER.readValue(responseStr, 
GetFunctionResponse.class);
+        String responseStr = RESTApi.toJson(response);
+        GetFunctionResponse parseData = RESTApi.fromJson(responseStr, 
GetFunctionResponse.class);
         assertEquals(response.uuid(), parseData.uuid());
     }
 
     @Test
     public void createFunctionRequestParseTest() throws 
JsonProcessingException {
         CreateFunctionRequest request = 
MockRESTMessage.createFunctionRequest();
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        CreateFunctionRequest parseData =
-                OBJECT_MAPPER.readValue(requestStr, 
CreateFunctionRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        CreateFunctionRequest parseData = RESTApi.fromJson(requestStr, 
CreateFunctionRequest.class);
         assertEquals(parseData.name(), request.name());
     }
 
     @Test
     public void alterFunctionRequestParseTest() throws JsonProcessingException 
{
         AlterFunctionRequest request = MockRESTMessage.alterFunctionRequest();
-        String requestStr = OBJECT_MAPPER.writeValueAsString(request);
-        AlterFunctionRequest parseData =
-                OBJECT_MAPPER.readValue(requestStr, 
AlterFunctionRequest.class);
+        String requestStr = RESTApi.toJson(request);
+        AlterFunctionRequest parseData = RESTApi.fromJson(requestStr, 
AlterFunctionRequest.class);
         assertEquals(parseData.changes().size(), request.changes().size());
     }
 }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java
index ff5d9e4116..6ac647f4ff 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java
@@ -38,7 +38,6 @@ import org.apache.paimon.function.FunctionChange;
 import org.apache.paimon.function.FunctionDefinition;
 import org.apache.paimon.function.FunctionImpl;
 import org.apache.paimon.operation.Lock;
-import org.apache.paimon.options.CatalogOptions;
 import org.apache.paimon.options.Options;
 import org.apache.paimon.partition.Partition;
 import org.apache.paimon.partition.PartitionStatistics;
@@ -120,12 +119,12 @@ import static org.apache.paimon.CoreOptions.PATH;
 import static org.apache.paimon.CoreOptions.SNAPSHOT_CLEAN_EMPTY_DIRECTORIES;
 import static org.apache.paimon.CoreOptions.TYPE;
 import static org.apache.paimon.TableType.FORMAT_TABLE;
-import static org.apache.paimon.rest.RESTCatalog.MAX_RESULTS;
-import static org.apache.paimon.rest.RESTCatalog.PAGE_TOKEN;
-import static org.apache.paimon.rest.RESTCatalog.PARTITION_NAME_PATTERN;
-import static org.apache.paimon.rest.RESTCatalog.TABLE_NAME_PATTERN;
-import static org.apache.paimon.rest.RESTCatalog.VIEW_NAME_PATTERN;
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
+import static org.apache.paimon.options.CatalogOptions.WAREHOUSE;
+import static org.apache.paimon.rest.RESTApi.MAX_RESULTS;
+import static org.apache.paimon.rest.RESTApi.PAGE_TOKEN;
+import static org.apache.paimon.rest.RESTApi.PARTITION_NAME_PATTERN;
+import static org.apache.paimon.rest.RESTApi.TABLE_NAME_PATTERN;
+import static org.apache.paimon.rest.RESTApi.VIEW_NAME_PATTERN;
 
 /** Mock REST server for testing. */
 public class RESTCatalogServer {
@@ -167,7 +166,7 @@ public class RESTCatalogServer {
         this.functionUri = resourcePaths.functions();
         Options conf = new Options();
         this.configResponse.getDefaults().forEach(conf::setString);
-        conf.setString(CatalogOptions.WAREHOUSE.key(), dataPath);
+        conf.setString(WAREHOUSE.key(), dataPath);
         CatalogContext context = CatalogContext.create(conf);
         Path warehousePath = new Path(dataPath);
         FileIO fileIO;
@@ -269,7 +268,7 @@ public class RESTCatalogServer {
                     }
                     if (request.getPath().startsWith(resourcePaths.config())
                             && request.getRequestUrl()
-                                    
.queryParameter(RESTCatalog.QUERY_PARAMETER_WAREHOUSE_KEY)
+                                    .queryParameter(WAREHOUSE.key())
                                     .equals(warehouse)) {
                         return mockResponse(configResponse, 200);
                     } else if (databaseUri.equals(request.getPath())
@@ -376,7 +375,7 @@ public class RESTCatalogServer {
                         }
                         if (isMarkDonePartitions) {
                             MarkDonePartitionsRequest 
markDonePartitionsRequest =
-                                    OBJECT_MAPPER.readValue(data, 
MarkDonePartitionsRequest.class);
+                                    RESTApi.fromJson(data, 
MarkDonePartitionsRequest.class);
                             catalog.markDonePartitions(
                                     identifier, 
markDonePartitionsRequest.getPartitionSpecs());
                             return new MockResponse().setResponseCode(200);
@@ -399,7 +398,7 @@ public class RESTCatalogServer {
                             return commitTableHandle(identifier, 
restAuthParameter.data());
                         } else if (isRollbackTable) {
                             RollbackTableRequest requestBody =
-                                    OBJECT_MAPPER.readValue(data, 
RollbackTableRequest.class);
+                                    RESTApi.fromJson(data, 
RollbackTableRequest.class);
                             if 
(noPermissionTables.contains(identifier.getFullName())) {
                                 throw new 
Catalog.TableNoPermissionException(identifier);
                             }
@@ -625,7 +624,7 @@ public class RESTCatalogServer {
                 new GetTableTokenResponse(dataToken.token(), 
dataToken.expireAtMillis());
         return new MockResponse()
                 .setResponseCode(200)
-                
.setBody(OBJECT_MAPPER.writeValueAsString(getTableTokenResponse));
+                .setBody(RESTApi.toJson(getTableTokenResponse));
     }
 
     private MockResponse snapshotHandle(Identifier identifier) throws 
Exception {
@@ -645,7 +644,7 @@ public class RESTCatalogServer {
                 new GetTableSnapshotResponse(snapshotOptional.get());
         return new MockResponse()
                 .setResponseCode(200)
-                
.setBody(OBJECT_MAPPER.writeValueAsString(getTableSnapshotResponse));
+                .setBody(RESTApi.toJson(getTableSnapshotResponse));
     }
 
     private Optional<MockResponse> checkTablePartitioned(Identifier 
identifier) {
@@ -665,8 +664,7 @@ public class RESTCatalogServer {
     }
 
     private MockResponse authTable(Identifier identifier, String data) throws 
Exception {
-        AuthTableQueryRequest requestBody =
-                OBJECT_MAPPER.readValue(data, AuthTableQueryRequest.class);
+        AuthTableQueryRequest requestBody = RESTApi.fromJson(data, 
AuthTableQueryRequest.class);
         if (noPermissionTables.contains(identifier.getFullName())) {
             throw new Catalog.TableNoPermissionException(identifier);
         }
@@ -688,7 +686,7 @@ public class RESTCatalogServer {
     }
 
     private MockResponse commitTableHandle(Identifier identifier, String data) 
throws Exception {
-        CommitTableRequest requestBody = OBJECT_MAPPER.readValue(data, 
CommitTableRequest.class);
+        CommitTableRequest requestBody = RESTApi.fromJson(data, 
CommitTableRequest.class);
         if (noPermissionTables.contains(identifier.getFullName())) {
             throw new Catalog.TableNoPermissionException(identifier);
         }
@@ -791,7 +789,7 @@ public class RESTCatalogServer {
                 return generateFinalListFunctionsResponse(parameters, 
functions);
             case "POST":
                 CreateFunctionRequest requestBody =
-                        OBJECT_MAPPER.readValue(data, 
CreateFunctionRequest.class);
+                        RESTApi.fromJson(data, CreateFunctionRequest.class);
                 String functionName = requestBody.name();
                 if (!functionStore.containsKey(functionName)) {
                     Function function =
@@ -846,7 +844,7 @@ public class RESTCatalogServer {
                 return mockResponse(response, 200);
             case "POST":
                 AlterFunctionRequest requestBody =
-                        OBJECT_MAPPER.readValue(data, 
AlterFunctionRequest.class);
+                        RESTApi.fromJson(data, AlterFunctionRequest.class);
                 HashMap<String, FunctionDefinition> newDefinitions =
                         new HashMap<>(function.definitions());
                 Map<String, String> newOptions = new 
HashMap<>(function.options());
@@ -919,7 +917,7 @@ public class RESTCatalogServer {
                 return generateFinalListDatabasesResponse(parameters, 
databases);
             case "POST":
                 CreateDatabaseRequest requestBody =
-                        OBJECT_MAPPER.readValue(data, 
CreateDatabaseRequest.class);
+                        RESTApi.fromJson(data, CreateDatabaseRequest.class);
                 String databaseName = requestBody.getName();
                 if (noPermissionDatabases.contains(databaseName)) {
                     throw new 
Catalog.DatabaseNoPermissionException(databaseName);
@@ -1055,7 +1053,7 @@ public class RESTCatalogServer {
                     return new MockResponse().setResponseCode(200);
                 case "POST":
                     AlterDatabaseRequest requestBody =
-                            OBJECT_MAPPER.readValue(data, 
AlterDatabaseRequest.class);
+                            RESTApi.fromJson(data, AlterDatabaseRequest.class);
                     List<PropertyChange> changes = new ArrayList<>();
                     for (String property : requestBody.getRemovals()) {
                         changes.add(PropertyChange.removeProperty(property));
@@ -1105,7 +1103,7 @@ public class RESTCatalogServer {
                     return generateFinalListTablesResponse(parameters, tables);
                 case "POST":
                     CreateTableRequest requestBody =
-                            OBJECT_MAPPER.readValue(data, 
CreateTableRequest.class);
+                            RESTApi.fromJson(data, CreateTableRequest.class);
                     Identifier identifier = requestBody.getIdentifier();
                     Schema schema = requestBody.getSchema();
                     TableMetadata tableMetadata;
@@ -1282,8 +1280,7 @@ public class RESTCatalogServer {
                                 "updated");
                 return mockResponse(response, 200);
             case "POST":
-                AlterTableRequest requestBody =
-                        OBJECT_MAPPER.readValue(data, AlterTableRequest.class);
+                AlterTableRequest requestBody = RESTApi.fromJson(data, 
AlterTableRequest.class);
                 alterTableImpl(identifier, requestBody.getChanges());
                 return new MockResponse().setResponseCode(200);
             case "DELETE":
@@ -1302,7 +1299,7 @@ public class RESTCatalogServer {
     }
 
     private MockResponse renameTableHandle(String data) throws Exception {
-        RenameTableRequest requestBody = OBJECT_MAPPER.readValue(data, 
RenameTableRequest.class);
+        RenameTableRequest requestBody = RESTApi.fromJson(data, 
RenameTableRequest.class);
         Identifier fromTable = requestBody.getSource();
         Identifier toTable = requestBody.getDestination();
         if (noPermissionTables.contains(fromTable.getFullName())) {
@@ -1388,7 +1385,7 @@ public class RESTCatalogServer {
                                 
tableLatestSnapshotStore.get(branchIdentifier.getFullName()));
                     } else {
                         CreateBranchRequest requestBody =
-                                OBJECT_MAPPER.readValue(data, 
CreateBranchRequest.class);
+                                RESTApi.fromJson(data, 
CreateBranchRequest.class);
                         branch = requestBody.branch();
                         if (requestBody.fromTag() == null) {
                             branchManager.createBranch(requestBody.branch());
@@ -1478,8 +1475,7 @@ public class RESTCatalogServer {
                 List<String> views = listViews(databaseName, parameters);
                 return generateFinalListViewsResponse(parameters, views);
             case "POST":
-                CreateViewRequest requestBody =
-                        OBJECT_MAPPER.readValue(data, CreateViewRequest.class);
+                CreateViewRequest requestBody = RESTApi.fromJson(data, 
CreateViewRequest.class);
                 Identifier identifier = requestBody.getIdentifier();
                 ViewSchema schema = requestBody.getSchema();
                 ViewImpl view =
@@ -1653,7 +1649,7 @@ public class RESTCatalogServer {
                 case "POST":
                     if (viewStore.containsKey(identifier.getFullName())) {
                         AlterViewRequest request =
-                                OBJECT_MAPPER.readValue(requestData, 
AlterViewRequest.class);
+                                RESTApi.fromJson(requestData, 
AlterViewRequest.class);
                         ViewImpl view = (ViewImpl) 
viewStore.get(identifier.getFullName());
                         HashMap<String, String> newDialects = new 
HashMap<>(view.dialects());
                         Map<String, String> newOptions = new 
HashMap<>(view.options());
@@ -1723,7 +1719,7 @@ public class RESTCatalogServer {
     }
 
     private MockResponse renameViewHandle(String data) throws Exception {
-        RenameTableRequest requestBody = OBJECT_MAPPER.readValue(data, 
RenameTableRequest.class);
+        RenameTableRequest requestBody = RESTApi.fromJson(data, 
RenameTableRequest.class);
         Identifier fromView = requestBody.getSource();
         Identifier toView = requestBody.getDestination();
         if (!viewStore.containsKey(fromView.getFullName())) {
@@ -1911,7 +1907,7 @@ public class RESTCatalogServer {
         try {
             return new MockResponse()
                     .setResponseCode(httpCode)
-                    .setBody(OBJECT_MAPPER.writeValueAsString(response))
+                    .setBody(RESTApi.toJson(response))
                     .addHeader("Content-Type", "application/json");
         } catch (JsonProcessingException e) {
             throw new RuntimeException(e);
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 29632df4a4..f6e259d2da 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
@@ -86,7 +86,7 @@ import static 
org.apache.paimon.CoreOptions.METASTORE_PARTITIONED_TABLE;
 import static org.apache.paimon.CoreOptions.METASTORE_TAG_TO_PARTITION;
 import static org.apache.paimon.CoreOptions.QUERY_AUTH_ENABLED;
 import static org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME;
-import static org.apache.paimon.rest.RESTCatalog.PAGE_TOKEN;
+import static org.apache.paimon.rest.RESTApi.PAGE_TOKEN;
 import static org.apache.paimon.rest.auth.DLFToken.TOKEN_DATE_FORMATTER;
 import static 
org.apache.paimon.utils.SnapshotManagerTest.createSnapshotWithMillis;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -1217,11 +1217,16 @@ public abstract class RESTCatalogTest extends 
CatalogTestBase {
         int maxResults = 2;
         AtomicInteger fetchTimes = new AtomicInteger(0);
         List<Integer> fetchData =
-                restCatalog.listDataFromPageApi(
-                        queryParams -> {
-                            return generateTestPagedResponse(
-                                    queryParams, testData, maxResults, 
fetchTimes, true);
-                        });
+                restCatalog
+                        .api()
+                        .listDataFromPageApi(
+                                queryParams ->
+                                        generateTestPagedResponse(
+                                                queryParams,
+                                                testData,
+                                                maxResults,
+                                                fetchTimes,
+                                                true));
         assertEquals(fetchTimes.get(), 4);
         assertThat(fetchData).containsSequence(testData);
     }
@@ -1232,11 +1237,13 @@ public abstract class RESTCatalogTest extends 
CatalogTestBase {
         int maxResults = 2;
         AtomicInteger fetchTimes = new AtomicInteger(0);
         List<Integer> fetchData =
-                restCatalog.listDataFromPageApi(
-                        queryParams -> {
-                            return generateTestPagedResponse(
-                                    queryParams, testData, maxResults, 
fetchTimes, false);
-                        });
+                restCatalog
+                        .api()
+                        .listDataFromPageApi(
+                                queryParams -> {
+                                    return generateTestPagedResponse(
+                                            queryParams, testData, maxResults, 
fetchTimes, false);
+                                });
 
         assertEquals(fetchTimes.get(), testData.size() / maxResults + 1);
         assertThat(fetchData).containsSequence(testData);
@@ -1567,7 +1574,7 @@ public abstract class RESTCatalogTest extends 
CatalogTestBase {
         String expiration = now.format(TOKEN_DATE_FORMATTER);
         String secret = UUID.randomUUID().toString();
         DLFToken token = new DLFToken("accessKeyId", secret, "securityToken", 
expiration);
-        String tokenStr = 
RESTObjectMapper.OBJECT_MAPPER.writeValueAsString(token);
+        String tokenStr = RESTApi.toJson(token);
         FileUtils.writeStringToFile(tokenFile, tokenStr);
     }
 }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/TestHttpWebServer.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/TestHttpWebServer.java
index 1e268c8d31..0f1ccecb55 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/TestHttpWebServer.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/TestHttpWebServer.java
@@ -27,8 +27,6 @@ import okhttp3.mockwebserver.RecordedRequest;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
-
 /** Mock a http web service locally. */
 public class TestHttpWebServer {
 
@@ -75,10 +73,10 @@ public class TestHttpWebServer {
     }
 
     public String createResponseBody(RESTResponse response) throws 
JsonProcessingException {
-        return OBJECT_MAPPER.writeValueAsString(response);
+        return RESTApi.toJson(response);
     }
 
     public <T> T readRequestBody(String body, Class<T> requestType) throws 
JsonProcessingException {
-        return OBJECT_MAPPER.readValue(body, requestType);
+        return RESTApi.fromJson(body, requestType);
     }
 }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthProviderTest.java 
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthProviderTest.java
index 770dadab5f..35780998c5 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthProviderTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthProviderTest.java
@@ -37,7 +37,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 
-import static 
org.apache.paimon.rest.RESTCatalog.TOKEN_EXPIRATION_SAFE_TIME_MILLIS;
+import static org.apache.paimon.rest.RESTApi.TOKEN_EXPIRATION_SAFE_TIME_MILLIS;
 import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_ID;
 import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_SECRET;
 import static org.apache.paimon.rest.RESTCatalogOptions.DLF_REGION;
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/DLFAuthSignatureTest.java
 
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/DLFAuthSignatureTest.java
index e58b360ee2..6292173b6f 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/DLFAuthSignatureTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/DLFAuthSignatureTest.java
@@ -19,7 +19,7 @@
 package org.apache.paimon.rest.auth;
 
 import org.apache.paimon.rest.MockRESTMessage;
-import org.apache.paimon.rest.RESTObjectMapper;
+import org.apache.paimon.rest.RESTApi;
 
 import org.junit.jupiter.api.Test;
 
@@ -39,9 +39,7 @@ public class DLFAuthSignatureTest {
         Map<String, String> parameters = new HashMap<>();
         parameters.put("k1", "v1");
         parameters.put("k2", "v2");
-        String data =
-                RESTObjectMapper.OBJECT_MAPPER.writeValueAsString(
-                        MockRESTMessage.createDatabaseRequest("database"));
+        String data = 
RESTApi.toJson(MockRESTMessage.createDatabaseRequest("database"));
         RESTAuthParameter restAuthParameter =
                 new RESTAuthParameter("/v1/paimon/databases", parameters, 
"POST", data);
         DLFToken token = new DLFToken("access-key-id", "access-key-secret", 
"securityToken", null);

Reply via email to