From 4d5a1fda769fd02d52d190a9f877ae3cd4549764 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 18 Nov 2024 15:29:17 +0800 Subject: [PATCH 01/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/RESTCatalog.java | 135 ++++++++++++++++++ 1 file changed, 135 insertions(+) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java 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 new file mode 100644 index 000000000000..7619d3e4f09c --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -0,0 +1,135 @@ +/* + * 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.catalog.Catalog; +import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.table.Table; + +import java.util.List; +import java.util.Map; + +public class RESTCatalog implements Catalog { + @Override + public String warehouse() { + throw new UnsupportedOperationException(); + } + + @Override + public Map options() { + throw new UnsupportedOperationException(); + } + + @Override + public FileIO fileIO() { + throw new UnsupportedOperationException(); + } + + @Override + public List listDatabases() { + throw new UnsupportedOperationException(); + } + + @Override + public void createDatabase(String name, boolean ignoreIfExists, Map properties) + throws DatabaseAlreadyExistException { + throw new UnsupportedOperationException(); + } + + @Override + public Database getDatabase(String name) throws DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException { + throw new UnsupportedOperationException(); + } + + @Override + public Table getTable(Identifier identifier) throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public Path getTableLocation(Identifier identifier) { + throw new UnsupportedOperationException(); + } + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropTable(Identifier identifier, boolean ignoreIfNotExists) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable( + Identifier identifier, List changes, boolean ignoreIfNotExists) + throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition(Identifier identifier, Map partitionSpec) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition(Identifier identifier, Map partitions) + throws TableNotExistException, PartitionNotExistException {} + + @Override + public List listPartitions(Identifier identifier) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean allowUpperCase() { + return false; + } + + @Override + public void close() throws Exception {} +} From 693b1e10a0302c892979ef11e96668a067553ec6 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 18 Nov 2024 16:56:04 +0800 Subject: [PATCH 02/60] [core] Add implementation to support REST Catalog --- paimon-core/pom.xml | 8 +- .../org/apache/paimon/rest/HttpClient.java | 50 ++++++++ .../apache/paimon/rest/HttpClientFactory.java | 61 ++++++++++ .../org/apache/paimon/rest/RESTCatalog.java | 13 ++- .../apache/paimon/rest/RESTCatalogApi.java | 33 ++++++ .../org/apache/paimon/rest/RESTClient.java | 23 ++++ .../org/apache/paimon/rest/RESTMessage.java | 21 ++++ .../org/apache/paimon/rest/RESTRequest.java | 21 ++++ .../org/apache/paimon/rest/RESTResponse.java | 21 ++++ .../apache/paimon/rest/RestClientUtil.java | 110 ++++++++++++++++++ .../paimon/rest/requests/ConfigRequest.java | 23 ++++ .../paimon/rest/responses/ConfigResponse.java | 31 +++++ .../paimon/rest/responses/ErrorResponse.java | 29 +++++ 13 files changed, 442 insertions(+), 2 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 399f0b5d6c19..41e4cb8893db 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -79,7 +79,13 @@ under the License. ${project.version} provided - + + + + com.squareup.retrofit2 + retrofit + 2.11.0 + org.apache.hadoop hadoop-common diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java new file mode 100644 index 000000000000..ace6c5b93f10 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -0,0 +1,50 @@ +/* + * 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 okhttp3.OkHttpClient; +import retrofit2.Retrofit; + +import static java.util.Objects.requireNonNull; + +public class HttpClient implements RESTClient { + + private final OkHttpClient okHttpClient; + private final String endpoint; + + public HttpClient(String endpoint) { + // todo: support config + this.okHttpClient = HttpClientFactory.createOkHttpClient(1, 3_000, 3_000); + this.endpoint = endpoint; + } + + @Override + public RESTCatalogApi getClient() { + return HttpClient.createAgentCallRetrofit(okHttpClient, endpoint) + .create(RESTCatalogApi.class); + } + + private static Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String baseUrl) { + return new Retrofit.Builder() + .client(requireNonNull(httpClient, "httpClient")) + .baseUrl(requireNonNull(baseUrl, "baseUrl").toString()) + .validateEagerly(true) + .build(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java new file mode 100644 index 000000000000..3429cd1d6937 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java @@ -0,0 +1,61 @@ +/* + * 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 com.google.common.util.concurrent.ThreadFactoryBuilder; +import okhttp3.Dispatcher; +import okhttp3.OkHttpClient; + +import java.util.Arrays; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static okhttp3.ConnectionSpec.CLEARTEXT; +import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; +import static okhttp3.ConnectionSpec.MODERN_TLS; + +public class HttpClientFactory { + + public static OkHttpClient createOkHttpClient( + int threadPoolSize, long connectTimeoutMillis, long readTimeoutMillis) { + ExecutorService executorService = + new ThreadPoolExecutor( + threadPoolSize, + threadPoolSize, + 60, + TimeUnit.SECONDS, + new SynchronousQueue<>(), + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("rest catalog http client %d") + .build()); + + OkHttpClient.Builder builder = + new OkHttpClient.Builder() + .connectTimeout(connectTimeoutMillis, TimeUnit.MILLISECONDS) + .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) + .dispatcher(new Dispatcher(executorService)) + .retryOnConnectionFailure(true) + .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); + + return builder.build(); + } +} 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 7619d3e4f09c..eac2c05ab1fa 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 @@ -24,6 +24,8 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.rest.requests.ConfigRequest; +import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; @@ -32,6 +34,12 @@ import java.util.Map; public class RESTCatalog implements Catalog { + private RESTClient client; + + public RESTCatalog(String endpoint) { + this.client = new HttpClient(endpoint); + } + @Override public String warehouse() { throw new UnsupportedOperationException(); @@ -39,7 +47,10 @@ public String warehouse() { @Override public Map options() { - throw new UnsupportedOperationException(); + ConfigResponse response = + RestClientUtil.getResponse(client.getClient().getConfig(new ConfigRequest())) + .body(); + return response.getDefaults(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java new file mode 100644 index 000000000000..2052aabd12d8 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java @@ -0,0 +1,33 @@ +/* + * 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.rest.requests.ConfigRequest; +import org.apache.paimon.rest.responses.ConfigResponse; + +import retrofit2.Call; +import retrofit2.http.Body; +import retrofit2.http.GET; + +public interface RESTCatalogApi { + String BASE_AGENT_PATH = "/api/v1/{prefix}"; + + @GET(BASE_AGENT_PATH + "/config") + Call getConfig(@Body ConfigRequest request); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java new file mode 100644 index 000000000000..3b2f139d89b5 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -0,0 +1,23 @@ +/* + * 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; + +public interface RESTClient { + RESTCatalogApi getClient(); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java new file mode 100644 index 000000000000..64fa8bf0f3f8 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +public interface RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java new file mode 100644 index 000000000000..6fb03f8a3b39 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +public interface RESTRequest extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java new file mode 100644 index 000000000000..7fa35d68d69a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +public interface RESTResponse extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java new file mode 100644 index 000000000000..2a9850447344 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java @@ -0,0 +1,110 @@ +/* + * 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.rest.responses.ErrorResponse; + +import com.fasterxml.jackson.databind.ObjectMapper; +import okhttp3.HttpUrl; +import okhttp3.Request; +import okhttp3.ResponseBody; +import retrofit2.Call; +import retrofit2.Response; + +import java.io.IOException; + +public class RestClientUtil { + private static final org.slf4j.Logger log = + org.slf4j.LoggerFactory.getLogger(RestClientUtil.class); + private static final ObjectMapper mapper = null; + + public static Response getResponse(Call call) { + long start = System.currentTimeMillis(); + try { + return call.execute(); + } catch (IOException e) { + throw new RuntimeException("Failed to execute call " + call.request().url(), e); + } finally { + long cost = System.currentTimeMillis() - start; + + if (log.isDebugEnabled()) { + log.debug("{} {}, cost:{}ms", call.request().method(), call.request().url(), cost); + } + } + } + + public static T getResponseBody(Call call) { + Request request = call.request(); + if (log.isDebugEnabled()) { + log.debug("Begin: http call [{}]", request); + } + + Response response = getResponse(call); + T body = response.body(); + + if (log.isDebugEnabled()) { + log.debug( + "Http call success [{}], http code [{}]", + response.isSuccessful(), + response.code()); + } + + if (response.code() == 403) { + throw new UnsupportedOperationException( + "This call is Forbidden, as cluster name is diff with call cluster."); + } + if (!response.isSuccessful() || body == null) { + try (ResponseBody errorBody = response.errorBody()) { + HttpUrl url = request.url(); + if (errorBody != null) { + String errorBodyStr = null; + try { + errorBodyStr = errorBody.string(); + ErrorResponse errorResponse = + mapper.readValue(errorBody.toString(), ErrorResponse.class); + if (errorResponse != null + && errorResponse.getErrors() != null + && errorResponse.getErrors().size() > 0) { + errorBodyStr = String.join(",", errorResponse.getErrors()); + } + log.error("Http call Error body: [{}]", errorBodyStr); + } catch (Throwable e) { + String logErrorBodyStr = errorBodyStr != null ? errorBodyStr : "empty"; + log.warn( + "Http call parse errorBody [{}] from response failed:", + logErrorBodyStr, + e); + } + + if (errorBodyStr != null) { + throw new IllegalStateException( + String.format( + "Missing body in response for call %s. Error body: %s", + url, errorBodyStr)); + } + log.warn("Http call [{}] get errorBodyStr from response is empty", url); + } + log.error("Http call [{}] errorBody is empty", url); + throw new IllegalStateException( + String.format("Missing body in response for call %s", url)); + } + } + return body; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java new file mode 100644 index 000000000000..d61584c72a3a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.rest.RESTRequest; + +public class ConfigRequest implements RESTRequest {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java new file mode 100644 index 000000000000..cdbe5294f452 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.responses; + +import org.apache.paimon.rest.RESTResponse; + +import java.util.Map; + +public class ConfigResponse implements RESTResponse { + private Map defaults; + + public Map getDefaults() { + return defaults; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java new file mode 100644 index 000000000000..6479ea8da03b --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.responses; + +import java.util.List; + +public class ErrorResponse { + List errors; + + public List getErrors() { + return errors; + } +} From f632a06726823639fdaa6b10b50bc67e441c7130 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 18 Nov 2024 17:00:17 +0800 Subject: [PATCH 03/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/HttpClient.java | 43 ++++++++++++- .../apache/paimon/rest/HttpClientFactory.java | 61 ------------------- 2 files changed, 42 insertions(+), 62 deletions(-) delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index ace6c5b93f10..ce7e11e7ff16 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -18,10 +18,21 @@ package org.apache.paimon.rest; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import okhttp3.Dispatcher; import okhttp3.OkHttpClient; import retrofit2.Retrofit; +import java.util.Arrays; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + import static java.util.Objects.requireNonNull; +import static okhttp3.ConnectionSpec.CLEARTEXT; +import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; +import static okhttp3.ConnectionSpec.MODERN_TLS; public class HttpClient implements RESTClient { @@ -30,7 +41,12 @@ public class HttpClient implements RESTClient { public HttpClient(String endpoint) { // todo: support config - this.okHttpClient = HttpClientFactory.createOkHttpClient(1, 3_000, 3_000); + this.okHttpClient = createHttpClient(1, 3_000, 3_000); + this.endpoint = endpoint; + } + + public HttpClient(OkHttpClient okHttpClient, String endpoint) { + this.okHttpClient = okHttpClient; this.endpoint = endpoint; } @@ -47,4 +63,29 @@ private static Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String .validateEagerly(true) .build(); } + + private static OkHttpClient createHttpClient( + int threadPoolSize, long connectTimeoutMillis, long readTimeoutMillis) { + ExecutorService executorService = + new ThreadPoolExecutor( + threadPoolSize, + threadPoolSize, + 60, + TimeUnit.SECONDS, + new SynchronousQueue<>(), + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("rest catalog http client %d") + .build()); + + OkHttpClient.Builder builder = + new OkHttpClient.Builder() + .connectTimeout(connectTimeoutMillis, TimeUnit.MILLISECONDS) + .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) + .dispatcher(new Dispatcher(executorService)) + .retryOnConnectionFailure(true) + .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); + + return builder.build(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java deleted file mode 100644 index 3429cd1d6937..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import okhttp3.Dispatcher; -import okhttp3.OkHttpClient; - -import java.util.Arrays; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import static okhttp3.ConnectionSpec.CLEARTEXT; -import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; -import static okhttp3.ConnectionSpec.MODERN_TLS; - -public class HttpClientFactory { - - public static OkHttpClient createOkHttpClient( - int threadPoolSize, long connectTimeoutMillis, long readTimeoutMillis) { - ExecutorService executorService = - new ThreadPoolExecutor( - threadPoolSize, - threadPoolSize, - 60, - TimeUnit.SECONDS, - new SynchronousQueue<>(), - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("rest catalog http client %d") - .build()); - - OkHttpClient.Builder builder = - new OkHttpClient.Builder() - .connectTimeout(connectTimeoutMillis, TimeUnit.MILLISECONDS) - .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) - .dispatcher(new Dispatcher(executorService)) - .retryOnConnectionFailure(true) - .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); - - return builder.build(); - } -} From 0a87c4d1e81d21b695e01f0ef4918eed1728aafa Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 18 Nov 2024 17:50:09 +0800 Subject: [PATCH 04/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/RESTCatalog.java | 3 +- .../apache/paimon/rest/RestClientUtil.java | 34 +++++++++---------- 2 files changed, 18 insertions(+), 19 deletions(-) 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 eac2c05ab1fa..53b66bd7cfaa 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 @@ -48,8 +48,7 @@ public String warehouse() { @Override public Map options() { ConfigResponse response = - RestClientUtil.getResponse(client.getClient().getConfig(new ConfigRequest())) - .body(); + RestClientUtil.getResponse(client.getClient().getConfig(new ConfigRequest())); return response.getDefaults(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java index 2a9850447344..cc17852f61af 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java @@ -34,28 +34,13 @@ public class RestClientUtil { org.slf4j.LoggerFactory.getLogger(RestClientUtil.class); private static final ObjectMapper mapper = null; - public static Response getResponse(Call call) { - long start = System.currentTimeMillis(); - try { - return call.execute(); - } catch (IOException e) { - throw new RuntimeException("Failed to execute call " + call.request().url(), e); - } finally { - long cost = System.currentTimeMillis() - start; - - if (log.isDebugEnabled()) { - log.debug("{} {}, cost:{}ms", call.request().method(), call.request().url(), cost); - } - } - } - - public static T getResponseBody(Call call) { + public static T getResponse(Call call) { Request request = call.request(); if (log.isDebugEnabled()) { log.debug("Begin: http call [{}]", request); } - Response response = getResponse(call); + Response response = getResponseUtil(call); T body = response.body(); if (log.isDebugEnabled()) { @@ -107,4 +92,19 @@ public static T getResponseBody(Call call) { } return body; } + + private static Response getResponseUtil(Call call) { + long start = System.currentTimeMillis(); + try { + return call.execute(); + } catch (IOException e) { + throw new RuntimeException("Failed to execute call " + call.request().url(), e); + } finally { + long cost = System.currentTimeMillis() - start; + + if (log.isDebugEnabled()) { + log.debug("{} {}, cost:{}ms", call.request().method(), call.request().url(), cost); + } + } + } } From 52f07b6768c80df8ce4a56623da78669dac5b81f Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 18 Nov 2024 18:05:12 +0800 Subject: [PATCH 05/60] [core] Add implementation to support REST Catalog --- .../apache/paimon/rest/HttpClientFactory.java | 61 +++++++++++++++++++ .../org/apache/paimon/rest/RESTCatalog.java | 4 +- .../paimon/rest/RESTCatalogFactory.java | 39 ++++++++++++ .../paimon/rest/RESTCatalogOptions.java | 30 +++++++++ .../org.apache.paimon.factories.Factory | 1 + 5 files changed, 134 insertions(+), 1 deletion(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java new file mode 100644 index 000000000000..3429cd1d6937 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java @@ -0,0 +1,61 @@ +/* + * 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 com.google.common.util.concurrent.ThreadFactoryBuilder; +import okhttp3.Dispatcher; +import okhttp3.OkHttpClient; + +import java.util.Arrays; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static okhttp3.ConnectionSpec.CLEARTEXT; +import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; +import static okhttp3.ConnectionSpec.MODERN_TLS; + +public class HttpClientFactory { + + public static OkHttpClient createOkHttpClient( + int threadPoolSize, long connectTimeoutMillis, long readTimeoutMillis) { + ExecutorService executorService = + new ThreadPoolExecutor( + threadPoolSize, + threadPoolSize, + 60, + TimeUnit.SECONDS, + new SynchronousQueue<>(), + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("rest catalog http client %d") + .build()); + + OkHttpClient.Builder builder = + new OkHttpClient.Builder() + .connectTimeout(connectTimeoutMillis, TimeUnit.MILLISECONDS) + .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) + .dispatcher(new Dispatcher(executorService)) + .retryOnConnectionFailure(true) + .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); + + return builder.build(); + } +} 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 53b66bd7cfaa..1a8112d80e87 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 @@ -24,6 +24,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.options.Options; import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.schema.Schema; @@ -36,7 +37,8 @@ public class RESTCatalog implements Catalog { private RESTClient client; - public RESTCatalog(String endpoint) { + public RESTCatalog(Options options) { + String endpoint = options.get(RESTCatalogOptions.ENDPOINT); this.client = new HttpClient(endpoint); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java new file mode 100644 index 000000000000..e6fd8f2b13ea --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -0,0 +1,39 @@ +/* + * 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.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; + +public class RESTCatalogFactory implements CatalogFactory { + public static final String IDENTIFIER = "rest"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public Catalog create(FileIO fileIO, Path warehouse, CatalogContext context) { + return new RESTCatalog(context.options()); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java new file mode 100644 index 000000000000..8ed5e7e21c79 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -0,0 +1,30 @@ +/* + * 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.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +public class RESTCatalogOptions { + public static final ConfigOption ENDPOINT = + ConfigOptions.key("endpoint") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog server's endpoint."); +} diff --git a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index ac6cc98fed6b..3b98eef52c85 100644 --- a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -36,3 +36,4 @@ org.apache.paimon.mergetree.compact.aggregate.factory.FieldRoaringBitmap32AggFac org.apache.paimon.mergetree.compact.aggregate.factory.FieldRoaringBitmap64AggFactory org.apache.paimon.mergetree.compact.aggregate.factory.FieldSumAggFactory org.apache.paimon.mergetree.compact.aggregate.factory.FieldThetaSketchAggFactory +org.apache.paimon.rest.RESTCatalogFactory From 1c3cc245627e18a5e207ce52ff42633aa4134c99 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 11:16:27 +0800 Subject: [PATCH 06/60] [core] Add implementation to support REST Catalog --- paimon-core/pom.xml | 20 +++++ .../org/apache/paimon/rest/HttpClient.java | 11 ++- .../apache/paimon/rest/HttpClientFactory.java | 61 --------------- .../org/apache/paimon/rest/RESTCatalog.java | 5 +- .../apache/paimon/rest/RESTCatalogApi.java | 5 +- .../paimon/rest/RESTCatalogFactory.java | 1 + .../paimon/rest/RESTCatalogOptions.java | 1 + .../org/apache/paimon/rest/RESTClient.java | 1 + .../org/apache/paimon/rest/RESTMessage.java | 1 + .../apache/paimon/rest/RESTObjectMapper.java | 32 ++++++++ .../org/apache/paimon/rest/RESTRequest.java | 1 + .../org/apache/paimon/rest/RESTResponse.java | 1 + .../apache/paimon/rest/RestClientUtil.java | 1 + .../paimon/rest/requests/ConfigRequest.java | 1 + .../paimon/rest/responses/ConfigResponse.java | 7 ++ .../paimon/rest/responses/ErrorResponse.java | 1 + .../paimon/rest/RESTCatalogApiTest.java | 77 +++++++++++++++++++ 17 files changed, 157 insertions(+), 70 deletions(-) delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 41e4cb8893db..ae36771e10e2 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -86,6 +86,26 @@ under the License. retrofit 2.11.0 + + + com.squareup.okhttp3 + okhttp + 4.12.0 + + + + com.squareup.okhttp3 + mockwebserver + 4.12.0 + test + + + + com.squareup.retrofit2 + converter-jackson + 2.11.0 + + org.apache.hadoop hadoop-common diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index ce7e11e7ff16..29151af9963e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -18,10 +18,12 @@ package org.apache.paimon.rest; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.ThreadFactoryBuilder; import okhttp3.Dispatcher; import okhttp3.OkHttpClient; import retrofit2.Retrofit; +import retrofit2.converter.jackson.JacksonConverterFactory; import java.util.Arrays; import java.util.concurrent.ExecutorService; @@ -34,10 +36,12 @@ import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; import static okhttp3.ConnectionSpec.MODERN_TLS; +/** HTTP client for REST catalog. */ public class HttpClient implements RESTClient { private final OkHttpClient okHttpClient; private final String endpoint; + private final ObjectMapper mapper = RESTObjectMapper.create(); public HttpClient(String endpoint) { // todo: support config @@ -52,14 +56,15 @@ public HttpClient(OkHttpClient okHttpClient, String endpoint) { @Override public RESTCatalogApi getClient() { - return HttpClient.createAgentCallRetrofit(okHttpClient, endpoint) - .create(RESTCatalogApi.class); + return createAgentCallRetrofit(okHttpClient, endpoint).create(RESTCatalogApi.class); } - private static Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String baseUrl) { + private Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String baseUrl) { return new Retrofit.Builder() .client(requireNonNull(httpClient, "httpClient")) .baseUrl(requireNonNull(baseUrl, "baseUrl").toString()) + // todo: need define mapper + .addConverterFactory(JacksonConverterFactory.create(mapper)) .validateEagerly(true) .build(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java deleted file mode 100644 index 3429cd1d6937..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientFactory.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import okhttp3.Dispatcher; -import okhttp3.OkHttpClient; - -import java.util.Arrays; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import static okhttp3.ConnectionSpec.CLEARTEXT; -import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; -import static okhttp3.ConnectionSpec.MODERN_TLS; - -public class HttpClientFactory { - - public static OkHttpClient createOkHttpClient( - int threadPoolSize, long connectTimeoutMillis, long readTimeoutMillis) { - ExecutorService executorService = - new ThreadPoolExecutor( - threadPoolSize, - threadPoolSize, - 60, - TimeUnit.SECONDS, - new SynchronousQueue<>(), - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("rest catalog http client %d") - .build()); - - OkHttpClient.Builder builder = - new OkHttpClient.Builder() - .connectTimeout(connectTimeoutMillis, TimeUnit.MILLISECONDS) - .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) - .dispatcher(new Dispatcher(executorService)) - .retryOnConnectionFailure(true) - .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); - - return builder.build(); - } -} 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 1a8112d80e87..c2652ddffa0a 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 @@ -25,7 +25,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.Options; -import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -34,6 +33,7 @@ import java.util.List; import java.util.Map; +/** REST catalog. */ public class RESTCatalog implements Catalog { private RESTClient client; @@ -49,8 +49,7 @@ public String warehouse() { @Override public Map options() { - ConfigResponse response = - RestClientUtil.getResponse(client.getClient().getConfig(new ConfigRequest())); + ConfigResponse response = RestClientUtil.getResponse(client.getClient().getConfig()); return response.getDefaults(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java index 2052aabd12d8..f52dc3af3a6b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java @@ -18,16 +18,15 @@ package org.apache.paimon.rest; -import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import retrofit2.Call; -import retrofit2.http.Body; import retrofit2.http.GET; +/** REST catalog API. */ public interface RESTCatalogApi { String BASE_AGENT_PATH = "/api/v1/{prefix}"; @GET(BASE_AGENT_PATH + "/config") - Call getConfig(@Body ConfigRequest request); + Call getConfig(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index e6fd8f2b13ea..4b6d77c9f5a3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -24,6 +24,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +/** REST catalog factory. */ public class RESTCatalogFactory implements CatalogFactory { public static final String IDENTIFIER = "rest"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 8ed5e7e21c79..32356d5ce55d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -21,6 +21,7 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; +/** REST catalog options. */ public class RESTCatalogOptions { public static final ConfigOption ENDPOINT = ConfigOptions.key("endpoint") diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java index 3b2f139d89b5..5173756cfd92 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +/** REST client. */ public interface RESTClient { RESTCatalogApi getClient(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java index 64fa8bf0f3f8..110fb7978079 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java @@ -18,4 +18,5 @@ package org.apache.paimon.rest; +/** REST message. */ public interface RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java new file mode 100644 index 000000000000..d7d9a899ac5b --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -0,0 +1,32 @@ +/* + * 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 com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; + +public class RESTObjectMapper { + public static ObjectMapper create() { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.registerModule(new JavaTimeModule()); + return mapper; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java index 6fb03f8a3b39..44bfa5ae31f8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java @@ -18,4 +18,5 @@ package org.apache.paimon.rest; +/** REST message. */ public interface RESTRequest extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java index 7fa35d68d69a..2e40fe645beb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java @@ -18,4 +18,5 @@ package org.apache.paimon.rest; +/** REST response. */ public interface RESTResponse extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java index cc17852f61af..a4ebcb8a8405 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java @@ -29,6 +29,7 @@ import java.io.IOException; +/** REST client util. */ public class RestClientUtil { private static final org.slf4j.Logger log = org.slf4j.LoggerFactory.getLogger(RestClientUtil.class); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java index d61584c72a3a..e7d670ca8a5e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java @@ -20,4 +20,5 @@ import org.apache.paimon.rest.RESTRequest; +/** Request to get config. */ public class ConfigRequest implements RESTRequest {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index cdbe5294f452..ae5a5b9bbc14 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -22,9 +22,16 @@ import java.util.Map; +/** Response to get config. */ public class ConfigResponse implements RESTResponse { private Map defaults; + public ConfigResponse() {} + + public ConfigResponse(Map defaults) { + this.defaults = defaults; + } + public Map getDefaults() { return defaults; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index 6479ea8da03b..2db4521689c4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -20,6 +20,7 @@ import java.util.List; +/** Response for error. */ public class ErrorResponse { List errors; diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java new file mode 100644 index 000000000000..706ebbe2b9fe --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java @@ -0,0 +1,77 @@ +/* + * 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.rest.responses.ConfigResponse; + +import okhttp3.mockwebserver.MockResponse; +import okhttp3.mockwebserver.MockWebServer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import retrofit2.Response; +import retrofit2.Retrofit; +import retrofit2.converter.jackson.JacksonConverterFactory; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +public class RESTCatalogApiTest { + private MockWebServer mockWebServer; + private RESTCatalogApi apiService; + + @Before + public void setUp() throws IOException { + mockWebServer = new MockWebServer(); + mockWebServer.start(); + + String baseUrl = mockWebServer.url("/").toString(); + + Retrofit retrofit = + new Retrofit.Builder() + .baseUrl(baseUrl) + .addConverterFactory( + JacksonConverterFactory.create(RESTObjectMapper.create())) + .build(); + + apiService = retrofit.create(RESTCatalogApi.class); + } + + @After + public void tearDown() throws IOException { + mockWebServer.shutdown(); + } + + @Test + public void testGetData() throws IOException { + String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; + MockResponse mockResponseObj = + new MockResponse() + .setBody(mockResponse) + .addHeader("Content-Type", "application/json"); + mockWebServer.enqueue(mockResponseObj); + + Response response = apiService.getConfig().execute(); + ConfigResponse data = response.body(); + + assertEquals(200, response.code()); + assertEquals("b", data.getDefaults().get("a")); + } +} From b5c0cdb0fb6a9a68045c73483043924e0894ac68 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 11:27:30 +0800 Subject: [PATCH 07/60] [core] Add implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTCatalog.java | 4 +++- .../main/java/org/apache/paimon/rest/RESTCatalogApi.java | 8 +++++--- .../java/org/apache/paimon/rest/RESTObjectMapper.java | 2 ++ .../org/apache/paimon/rest/requests/ConfigRequest.java | 4 +++- .../java/org/apache/paimon/rest/RESTCatalogApiTest.java | 5 +++-- 5 files changed, 16 insertions(+), 7 deletions(-) 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 c2652ddffa0a..c158fe03cdd8 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 @@ -25,6 +25,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.Options; +import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -49,7 +50,8 @@ public String warehouse() { @Override public Map options() { - ConfigResponse response = RestClientUtil.getResponse(client.getClient().getConfig()); + ConfigResponse response = + RestClientUtil.getResponse(client.getClient().getConfig(new ConfigRequest())); return response.getDefaults(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java index f52dc3af3a6b..301acb68aaf2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java @@ -18,15 +18,17 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import retrofit2.Call; -import retrofit2.http.GET; +import retrofit2.http.Body; +import retrofit2.http.POST; /** REST catalog API. */ public interface RESTCatalogApi { String BASE_AGENT_PATH = "/api/v1/{prefix}"; - @GET(BASE_AGENT_PATH + "/config") - Call getConfig(); + @POST(BASE_AGENT_PATH + "/config") + Call getConfig(@Body ConfigRequest request); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java index d7d9a899ac5b..7fe4f1eb8454 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -20,12 +20,14 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; public class RESTObjectMapper { public static ObjectMapper create() { ObjectMapper mapper = new ObjectMapper(); mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); mapper.registerModule(new JavaTimeModule()); return mapper; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java index e7d670ca8a5e..7f07e3297a75 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java @@ -21,4 +21,6 @@ import org.apache.paimon.rest.RESTRequest; /** Request to get config. */ -public class ConfigRequest implements RESTRequest {} +public class ConfigRequest implements RESTRequest { + public ConfigRequest() {} +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java index 706ebbe2b9fe..8638673a4230 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import okhttp3.mockwebserver.MockResponse; @@ -67,8 +68,8 @@ public void testGetData() throws IOException { .setBody(mockResponse) .addHeader("Content-Type", "application/json"); mockWebServer.enqueue(mockResponseObj); - - Response response = apiService.getConfig().execute(); + ConfigRequest request = new ConfigRequest(); + Response response = apiService.getConfig(request).execute(); ConfigResponse data = response.body(); assertEquals(200, response.code()); From 2ef4bd8a6a90760627a786db65ee009c90973e53 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 11:45:26 +0800 Subject: [PATCH 08/60] [core] Add implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTObjectMapper.java | 1 + .../test/java/org/apache/paimon/rest/RESTCatalogApiTest.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java index 7fe4f1eb8454..e7a43e55cd77 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +/** REST object mapper. */ public class RESTObjectMapper { public static ObjectMapper create() { ObjectMapper mapper = new ObjectMapper(); diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java index 8638673a4230..2d9f7b3435c6 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java @@ -61,7 +61,7 @@ public void tearDown() throws IOException { } @Test - public void testGetData() throws IOException { + public void testGetConfig() throws IOException { String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; MockResponse mockResponseObj = new MockResponse() From 1730e0603335151569c87e676a3f87647f20debd Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 14:47:02 +0800 Subject: [PATCH 09/60] [core] Add implementation to support REST Catalog --- .../rest/AuthenticationInterceptor.java | 59 +++++++++++++++++++ .../org/apache/paimon/rest/HttpClient.java | 11 ++-- .../org/apache/paimon/rest/RESTCatalog.java | 3 +- .../apache/paimon/rest/RestClientUtil.java | 1 - .../paimon/rest/RESTCatalogApiTest.java | 19 ++---- 5 files changed, 73 insertions(+), 20 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java b/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java new file mode 100644 index 000000000000..cd879ac033e5 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import okhttp3.Interceptor; +import okhttp3.Request; +import okhttp3.Response; + +import java.io.IOException; + +/** Authentication interceptor. */ +public class AuthenticationInterceptor implements Interceptor { + private String refreshToken; + + public AuthenticationInterceptor(String refreshToken) { + this.refreshToken = refreshToken; + } + + @Override + public Response intercept(Chain chain) throws IOException { + Request request = chain.request(); + Request.Builder requestBuilder = + request.newBuilder().header("Authorization", "Bearer " + refreshToken); + Response response = chain.proceed(requestBuilder.build()); + // Unauthorized + if (response.code() == 401) { + // Refresh the token here + refreshToken = refreshTokenFromServer(); + requestBuilder = request.newBuilder().header("Authorization", "Bearer " + refreshToken); + + response = chain.proceed(requestBuilder.build()); + } + response = + response.newBuilder().addHeader("Authorization", "Bearer " + refreshToken).build(); + return response; + } + + private String refreshTokenFromServer() { + // Make a request to the server to refresh the token + // and return the new token + return "new_refreshed_token"; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 29151af9963e..0cb50cb608c1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -43,9 +43,9 @@ public class HttpClient implements RESTClient { private final String endpoint; private final ObjectMapper mapper = RESTObjectMapper.create(); - public HttpClient(String endpoint) { + public HttpClient(String endpoint, String initToken) { // todo: support config - this.okHttpClient = createHttpClient(1, 3_000, 3_000); + this.okHttpClient = createHttpClient(1, 3_000, 3_000, initToken); this.endpoint = endpoint; } @@ -63,14 +63,16 @@ private Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String baseUrl return new Retrofit.Builder() .client(requireNonNull(httpClient, "httpClient")) .baseUrl(requireNonNull(baseUrl, "baseUrl").toString()) - // todo: need define mapper .addConverterFactory(JacksonConverterFactory.create(mapper)) .validateEagerly(true) .build(); } private static OkHttpClient createHttpClient( - int threadPoolSize, long connectTimeoutMillis, long readTimeoutMillis) { + int threadPoolSize, + long connectTimeoutMillis, + long readTimeoutMillis, + String initToken) { ExecutorService executorService = new ThreadPoolExecutor( threadPoolSize, @@ -89,6 +91,7 @@ private static OkHttpClient createHttpClient( .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) + .addInterceptor(new AuthenticationInterceptor(initToken)) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); return builder.build(); 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 c158fe03cdd8..12c7cf42260b 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 @@ -40,7 +40,8 @@ public class RESTCatalog implements Catalog { public RESTCatalog(Options options) { String endpoint = options.get(RESTCatalogOptions.ENDPOINT); - this.client = new HttpClient(endpoint); + // todo: token need config + this.client = new HttpClient(endpoint, "init_token"); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java index a4ebcb8a8405..47283c60a75d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java @@ -43,7 +43,6 @@ public static T getResponse(Call call) { Response response = getResponseUtil(call); T body = response.body(); - if (log.isDebugEnabled()) { log.debug( "Http call success [{}], http code [{}]", diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java index 2d9f7b3435c6..b109f84248f5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java @@ -21,14 +21,13 @@ import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; +import okhttp3.Headers; import okhttp3.mockwebserver.MockResponse; import okhttp3.mockwebserver.MockWebServer; import org.junit.After; import org.junit.Before; import org.junit.Test; import retrofit2.Response; -import retrofit2.Retrofit; -import retrofit2.converter.jackson.JacksonConverterFactory; import java.io.IOException; @@ -37,22 +36,14 @@ public class RESTCatalogApiTest { private MockWebServer mockWebServer; private RESTCatalogApi apiService; + private final String initToken = "init_token"; @Before public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); - String baseUrl = mockWebServer.url("/").toString(); - - Retrofit retrofit = - new Retrofit.Builder() - .baseUrl(baseUrl) - .addConverterFactory( - JacksonConverterFactory.create(RESTObjectMapper.create())) - .build(); - - apiService = retrofit.create(RESTCatalogApi.class); + apiService = (new HttpClient(baseUrl, initToken)).getClient(); } @After @@ -71,8 +62,8 @@ public void testGetConfig() throws IOException { ConfigRequest request = new ConfigRequest(); Response response = apiService.getConfig(request).execute(); ConfigResponse data = response.body(); - - assertEquals(200, response.code()); + Headers headers = response.headers(); assertEquals("b", data.getDefaults().get("a")); + assertEquals("Bearer " + initToken, headers.get("Authorization")); } } From cf5407f553b2220cc741576f1d31ea233d71e6c5 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 14:51:33 +0800 Subject: [PATCH 10/60] [core] Add implementation to support REST Catalog --- .../rest/AuthenticationInterceptor.java | 2 +- .../paimon/rest/RESTCatalogApiTest.java | 23 +++++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java b/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java index cd879ac033e5..484e436bad7c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java @@ -43,7 +43,7 @@ public Response intercept(Chain chain) throws IOException { // Refresh the token here refreshToken = refreshTokenFromServer(); requestBuilder = request.newBuilder().header("Authorization", "Bearer " + refreshToken); - + response.close(); response = chain.proceed(requestBuilder.build()); } response = diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java index b109f84248f5..f157cdb11175 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java @@ -32,6 +32,7 @@ import java.io.IOException; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; public class RESTCatalogApiTest { private MockWebServer mockWebServer; @@ -66,4 +67,26 @@ public void testGetConfig() throws IOException { assertEquals("b", data.getDefaults().get("a")); assertEquals("Bearer " + initToken, headers.get("Authorization")); } + + @Test + public void testNeedAuth() throws IOException { + String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; + MockResponse mockResponseObj401 = + new MockResponse() + .setBody(mockResponse) + .setResponseCode(401) + .addHeader("Content-Type", "application/json"); + MockResponse mockResponseObj200 = + new MockResponse() + .setBody(mockResponse) + .addHeader("Content-Type", "application/json"); + mockWebServer.enqueue(mockResponseObj401); + mockWebServer.enqueue(mockResponseObj200); + ConfigRequest request = new ConfigRequest(); + Response response = apiService.getConfig(request).execute(); + ConfigResponse data = response.body(); + Headers headers = response.headers(); + assertEquals("b", data.getDefaults().get("a")); + assertNotEquals("Bearer " + initToken, headers.get("Authorization")); + } } From 3171f0e623b40e82c64b7acc01612d1a52917932 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 17:31:46 +0800 Subject: [PATCH 11/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/HttpClient.java | 7 +- .../apache/paimon/rest/RESTObjectMapper.java | 8 +- .../apache/paimon/rest/RestClientUtil.java | 3 +- .../paimon/rest/responses/ConfigResponse.java | 4 + .../paimon/rest/RESTCatalogApiTest.java | 1 + paimon-open-api/pom.xml | 84 +++++++++++++++++++ .../paimon/open/api/OpenApiApplication.java | 31 +++++++ .../open/api/RESTCatalogController.java | 71 ++++++++++++++++ .../paimon/open/api/config/OpenAPIConfig.java | 59 +++++++++++++ .../src/main/resources/application.properties | 21 +++++ pom.xml | 1 + 11 files changed, 282 insertions(+), 8 deletions(-) create mode 100644 paimon-open-api/pom.xml create mode 100644 paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java create mode 100644 paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java create mode 100644 paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java create mode 100644 paimon-open-api/src/main/resources/application.properties diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 0cb50cb608c1..0c80a0bfa680 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -18,8 +18,9 @@ package org.apache.paimon.rest; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + import okhttp3.Dispatcher; import okhttp3.OkHttpClient; import retrofit2.Retrofit; @@ -63,7 +64,7 @@ private Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String baseUrl return new Retrofit.Builder() .client(requireNonNull(httpClient, "httpClient")) .baseUrl(requireNonNull(baseUrl, "baseUrl").toString()) - .addConverterFactory(JacksonConverterFactory.create(mapper)) + .addConverterFactory(JacksonConverterFactory.create()) .validateEagerly(true) .build(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java index e7a43e55cd77..173cb861b007 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -18,10 +18,10 @@ package org.apache.paimon.rest; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.SerializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; /** REST object mapper. */ public class RESTObjectMapper { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java index 47283c60a75d..0c5676881286 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java @@ -20,7 +20,8 @@ import org.apache.paimon.rest.responses.ErrorResponse; -import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + import okhttp3.HttpUrl; import okhttp3.Request; import okhttp3.ResponseBody; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index ae5a5b9bbc14..1c65d3076049 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -35,4 +35,8 @@ public ConfigResponse(Map defaults) { public Map getDefaults() { return defaults; } + + public void setDefaults(Map defaults) { + this.defaults = defaults; + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java index f157cdb11175..d09a6ede6110 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java @@ -34,6 +34,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +/** REST catalog api test. */ public class RESTCatalogApiTest { private MockWebServer mockWebServer; private RESTCatalogApi apiService; diff --git a/paimon-open-api/pom.xml b/paimon-open-api/pom.xml new file mode 100644 index 000000000000..b0eacaf28d6d --- /dev/null +++ b/paimon-open-api/pom.xml @@ -0,0 +1,84 @@ + + + + 4.0.0 + + org.apache.paimon + paimon-parent + 1.0-SNAPSHOT + + + paimon-open-api + + + 8 + 8 + UTF-8 + + + + org.springframework.boot + spring-boot-starter-web + 2.7.18 + + + ch.qos.logback + logback-classic + + + + + + + org.springdoc + springdoc-openapi-ui + 1.7.0 + + + org.apache.paimon + paimon-core + ${project.version} + + + io.swagger.core.v3 + swagger-annotations + 2.2.20 + + + + + + org.springframework.boot + spring-boot-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + \ No newline at end of file diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java new file mode 100644 index 000000000000..76ce4cbf83c6 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java @@ -0,0 +1,31 @@ +/* + * 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.open.api; + +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.SpringBootApplication; + +/** OpenAPI application. */ +@SpringBootApplication +public class OpenApiApplication { + + public static void main(String[] args) { + SpringApplication.run(OpenApiApplication.class, args); + } +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java new file mode 100644 index 000000000000..d51a37fb0882 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -0,0 +1,71 @@ +/* + * 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.open.api; + +import org.apache.paimon.rest.requests.ConfigRequest; +import org.apache.paimon.rest.responses.ConfigResponse; + +import io.swagger.v3.oas.annotations.Operation; +import io.swagger.v3.oas.annotations.media.Content; +import io.swagger.v3.oas.annotations.media.Schema; +import io.swagger.v3.oas.annotations.responses.ApiResponse; +import io.swagger.v3.oas.annotations.responses.ApiResponses; +import org.springframework.http.HttpStatus; +import org.springframework.http.ResponseEntity; +import org.springframework.web.bind.annotation.CrossOrigin; +import org.springframework.web.bind.annotation.PostMapping; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RestController; + +import java.util.HashMap; +import java.util.Map; + +/** * RESTCatalog management APIs. */ +@CrossOrigin(origins = "http://localhost:8081") +@RestController +@RequestMapping("/api/v1/{prefix}") +public class RESTCatalogController { + + @Operation( + summary = "Get Config", + tags = {"config"}) + @ApiResponses({ + @ApiResponse( + responseCode = "201", + content = { + @Content( + schema = @Schema(implementation = ConfigResponse.class), + mediaType = "application/json") + }), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/config") + public ResponseEntity getConfig(@RequestBody ConfigRequest request) { + try { + Map defaults = new HashMap<>(); + ConfigResponse response = new ConfigResponse(defaults); + return new ResponseEntity<>(response, HttpStatus.CREATED); + } catch (Exception e) { + return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); + } + } +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java new file mode 100644 index 000000000000..8bbccbc8d2eb --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.open.api.config; + +import io.swagger.v3.oas.models.OpenAPI; +import io.swagger.v3.oas.models.info.Info; +import io.swagger.v3.oas.models.info.License; +import io.swagger.v3.oas.models.servers.Server; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +import java.util.ArrayList; +import java.util.List; + +/** OpenAPI config. */ +@Configuration +public class OpenAPIConfig { + + @Value("${openapi.dev-url}") + private String devUrl; + + @Bean + public OpenAPI myOpenAPI() { + Server devServer = new Server(); + devServer.setUrl(devUrl); + devServer.setDescription("Server URL in Development environment"); + + License mitLicense = + new License().name("MIT License").url("https://choosealicense.com/licenses/mit/"); + + Info info = + new Info() + .title("Tutorial Management API") + .version("1.0") + .description("This API exposes endpoints to manage tutorials.") + .termsOfService("https://www.bezkoder.com/terms") + .license(mitLicense); + List servers = new ArrayList<>(); + servers.add(devServer); + return new OpenAPI().info(info).servers(servers); + } +} diff --git a/paimon-open-api/src/main/resources/application.properties b/paimon-open-api/src/main/resources/application.properties new file mode 100644 index 000000000000..f0b377ffff8f --- /dev/null +++ b/paimon-open-api/src/main/resources/application.properties @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +springdoc.swagger-ui.path=/swagger-api +springdoc.api-docs.path=/swagger-api-docs +springdoc.swagger-ui.tryItOutEnabled=true +springdoc.swagger-ui.filter=true +openapi.dev-url=http://localhost:8080 diff --git a/pom.xml b/pom.xml index 24f292331950..1292e1c5d0a0 100644 --- a/pom.xml +++ b/pom.xml @@ -69,6 +69,7 @@ under the License. paimon-test-utils paimon-arrow tools/ci/paimon-ci-tools + paimon-open-api From 484da3c1793f73ccc822a619711fc26f2cfff1e3 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 18:13:28 +0800 Subject: [PATCH 12/60] [core] Add implementation to support REST Catalog --- paimon-core/pom.xml | 13 -- .../org/apache/paimon/rest/HttpClient.java | 40 +++++-- .../org/apache/paimon/rest/RESTCatalog.java | 2 +- .../apache/paimon/rest/RESTCatalogApi.java | 34 ------ .../org/apache/paimon/rest/RESTClient.java | 17 ++- .../apache/paimon/rest/RestClientUtil.java | 111 ------------------ ...talogApiTest.java => RESTCatalogTest.java} | 37 +++--- 7 files changed, 58 insertions(+), 196 deletions(-) delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java rename paimon-core/src/test/java/org/apache/paimon/rest/{RESTCatalogApiTest.java => RESTCatalogTest.java} (65%) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index ae36771e10e2..3377414662b3 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -81,12 +81,6 @@ under the License. - - com.squareup.retrofit2 - retrofit - 2.11.0 - - com.squareup.okhttp3 okhttp @@ -99,13 +93,6 @@ under the License. 4.12.0 test - - - com.squareup.retrofit2 - converter-jackson - 2.11.0 - - org.apache.hadoop hadoop-common diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 0c80a0bfa680..386f1fa14e8f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -19,20 +19,24 @@ package org.apache.paimon.rest; import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import okhttp3.Dispatcher; +import okhttp3.MediaType; import okhttp3.OkHttpClient; -import retrofit2.Retrofit; -import retrofit2.converter.jackson.JacksonConverterFactory; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; +import java.io.IOException; import java.util.Arrays; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import static java.util.Objects.requireNonNull; import static okhttp3.ConnectionSpec.CLEARTEXT; import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; import static okhttp3.ConnectionSpec.MODERN_TLS; @@ -56,17 +60,29 @@ public HttpClient(OkHttpClient okHttpClient, String endpoint) { } @Override - public RESTCatalogApi getClient() { - return createAgentCallRetrofit(okHttpClient, endpoint).create(RESTCatalogApi.class); + public T post( + String path, RESTRequest body, Class responseType, Map headers) { + try { + RequestBody requestBody = buildRequestBody(body); + Request request = new Request.Builder().url(endpoint + path).post(requestBody).build(); + try (Response response = okHttpClient.newCall(request).execute()) { + String responseBody = response.body().string(); + return mapper.readValue(responseBody, responseType); + } + } catch (Exception e) { + throw new RuntimeException(e); + } } - private Retrofit createAgentCallRetrofit(OkHttpClient httpClient, String baseUrl) { - return new Retrofit.Builder() - .client(requireNonNull(httpClient, "httpClient")) - .baseUrl(requireNonNull(baseUrl, "baseUrl").toString()) - .addConverterFactory(JacksonConverterFactory.create()) - .validateEagerly(true) - .build(); + @Override + public void close() throws IOException { + okHttpClient.dispatcher().cancelAll(); + okHttpClient.connectionPool().evictAll(); + } + + private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingException { + return RequestBody.create( + MediaType.parse("application/json"), mapper.writeValueAsString(body)); } private static OkHttpClient createHttpClient( 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 12c7cf42260b..bbfee94178ce 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -52,7 +52,7 @@ public String warehouse() { @Override public Map options() { ConfigResponse response = - RestClientUtil.getResponse(client.getClient().getConfig(new ConfigRequest())); + client.post("config", new ConfigRequest(), ConfigResponse.class, () -> null); return response.getDefaults(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java deleted file mode 100644 index 301acb68aaf2..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogApi.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest; - -import org.apache.paimon.rest.requests.ConfigRequest; -import org.apache.paimon.rest.responses.ConfigResponse; - -import retrofit2.Call; -import retrofit2.http.Body; -import retrofit2.http.POST; - -/** REST catalog API. */ -public interface RESTCatalogApi { - String BASE_AGENT_PATH = "/api/v1/{prefix}"; - - @POST(BASE_AGENT_PATH + "/config") - Call getConfig(@Body ConfigRequest request); -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java index 5173756cfd92..71a8d65ddaa4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -18,7 +18,20 @@ package org.apache.paimon.rest; +import java.io.Closeable; +import java.util.Map; +import java.util.function.Supplier; + /** REST client. */ -public interface RESTClient { - RESTCatalogApi getClient(); +public interface RESTClient extends Closeable { + default T post( + String path, + RESTRequest body, + Class responseType, + Supplier> headers) { + return post(path, body, responseType, headers.get()); + } + + T post( + String path, RESTRequest body, Class responseType, Map headers); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java deleted file mode 100644 index 0c5676881286..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RestClientUtil.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest; - -import org.apache.paimon.rest.responses.ErrorResponse; - -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - -import okhttp3.HttpUrl; -import okhttp3.Request; -import okhttp3.ResponseBody; -import retrofit2.Call; -import retrofit2.Response; - -import java.io.IOException; - -/** REST client util. */ -public class RestClientUtil { - private static final org.slf4j.Logger log = - org.slf4j.LoggerFactory.getLogger(RestClientUtil.class); - private static final ObjectMapper mapper = null; - - public static T getResponse(Call call) { - Request request = call.request(); - if (log.isDebugEnabled()) { - log.debug("Begin: http call [{}]", request); - } - - Response response = getResponseUtil(call); - T body = response.body(); - if (log.isDebugEnabled()) { - log.debug( - "Http call success [{}], http code [{}]", - response.isSuccessful(), - response.code()); - } - - if (response.code() == 403) { - throw new UnsupportedOperationException( - "This call is Forbidden, as cluster name is diff with call cluster."); - } - if (!response.isSuccessful() || body == null) { - try (ResponseBody errorBody = response.errorBody()) { - HttpUrl url = request.url(); - if (errorBody != null) { - String errorBodyStr = null; - try { - errorBodyStr = errorBody.string(); - ErrorResponse errorResponse = - mapper.readValue(errorBody.toString(), ErrorResponse.class); - if (errorResponse != null - && errorResponse.getErrors() != null - && errorResponse.getErrors().size() > 0) { - errorBodyStr = String.join(",", errorResponse.getErrors()); - } - log.error("Http call Error body: [{}]", errorBodyStr); - } catch (Throwable e) { - String logErrorBodyStr = errorBodyStr != null ? errorBodyStr : "empty"; - log.warn( - "Http call parse errorBody [{}] from response failed:", - logErrorBodyStr, - e); - } - - if (errorBodyStr != null) { - throw new IllegalStateException( - String.format( - "Missing body in response for call %s. Error body: %s", - url, errorBodyStr)); - } - log.warn("Http call [{}] get errorBodyStr from response is empty", url); - } - log.error("Http call [{}] errorBody is empty", url); - throw new IllegalStateException( - String.format("Missing body in response for call %s", url)); - } - } - return body; - } - - private static Response getResponseUtil(Call call) { - long start = System.currentTimeMillis(); - try { - return call.execute(); - } catch (IOException e) { - throw new RuntimeException("Failed to execute call " + call.request().url(), e); - } finally { - long cost = System.currentTimeMillis() - start; - - if (log.isDebugEnabled()) { - log.debug("{} {}, cost:{}ms", call.request().method(), call.request().url(), cost); - } - } - } -} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java similarity index 65% rename from paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java rename to paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java index d09a6ede6110..04e14dfde7b3 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogApiTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -18,34 +18,33 @@ package org.apache.paimon.rest; -import org.apache.paimon.rest.requests.ConfigRequest; -import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.options.Options; -import okhttp3.Headers; import okhttp3.mockwebserver.MockResponse; import okhttp3.mockwebserver.MockWebServer; import org.junit.After; import org.junit.Before; import org.junit.Test; -import retrofit2.Response; import java.io.IOException; +import java.util.Map; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; /** REST catalog api test. */ -public class RESTCatalogApiTest { +public class RESTCatalogTest { private MockWebServer mockWebServer; - private RESTCatalogApi apiService; + private RESTCatalog restCatalog; private final String initToken = "init_token"; @Before public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); - String baseUrl = mockWebServer.url("/").toString(); - apiService = (new HttpClient(baseUrl, initToken)).getClient(); + String baseUrl = mockWebServer.url("").toString(); + Options options = new Options(); + options.set(RESTCatalogOptions.ENDPOINT, baseUrl); + restCatalog = new RESTCatalog(options); } @After @@ -54,23 +53,19 @@ public void tearDown() throws IOException { } @Test - public void testGetConfig() throws IOException { + public void testGetConfig() { String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; MockResponse mockResponseObj = new MockResponse() .setBody(mockResponse) .addHeader("Content-Type", "application/json"); mockWebServer.enqueue(mockResponseObj); - ConfigRequest request = new ConfigRequest(); - Response response = apiService.getConfig(request).execute(); - ConfigResponse data = response.body(); - Headers headers = response.headers(); - assertEquals("b", data.getDefaults().get("a")); - assertEquals("Bearer " + initToken, headers.get("Authorization")); + Map response = restCatalog.options(); + assertEquals("b", response.get("a")); } @Test - public void testNeedAuth() throws IOException { + public void testNeedAuth() { String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; MockResponse mockResponseObj401 = new MockResponse() @@ -83,11 +78,7 @@ public void testNeedAuth() throws IOException { .addHeader("Content-Type", "application/json"); mockWebServer.enqueue(mockResponseObj401); mockWebServer.enqueue(mockResponseObj200); - ConfigRequest request = new ConfigRequest(); - Response response = apiService.getConfig(request).execute(); - ConfigResponse data = response.body(); - Headers headers = response.headers(); - assertEquals("b", data.getDefaults().get("a")); - assertNotEquals("Bearer " + initToken, headers.get("Authorization")); + Map response = restCatalog.options(); + assertEquals("b", response.get("a")); } } From 5c85fda9c6a77b51acf641925710c57cd36e5341 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 19 Nov 2024 18:22:36 +0800 Subject: [PATCH 13/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/HttpClient.java | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 386f1fa14e8f..adddee32d6aa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -23,6 +23,7 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import okhttp3.Dispatcher; +import okhttp3.Headers; import okhttp3.MediaType; import okhttp3.OkHttpClient; import okhttp3.Request; @@ -64,11 +65,13 @@ public T post( String path, RESTRequest body, Class responseType, Map headers) { try { RequestBody requestBody = buildRequestBody(body); - Request request = new Request.Builder().url(endpoint + path).post(requestBody).build(); - try (Response response = okHttpClient.newCall(request).execute()) { - String responseBody = response.body().string(); - return mapper.readValue(responseBody, responseType); - } + Request request = + new Request.Builder() + .url(endpoint + path) + .post(requestBody) + .headers(Headers.of(headers)) + .build(); + return exec(request, responseType); } catch (Exception e) { throw new RuntimeException(e); } @@ -80,6 +83,16 @@ public void close() throws IOException { okHttpClient.connectionPool().evictAll(); } + private T exec(Request request, Class responseType) { + try (Response response = okHttpClient.newCall(request).execute()) { + String responseBody = response.body().string(); + return mapper.readValue(responseBody, responseType); + // todo: need handle exception + } catch (Exception e) { + throw new RuntimeException(e); + } + } + private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingException { return RequestBody.create( MediaType.parse("application/json"), mapper.writeValueAsString(body)); From 6613160553900b3e04dbb341442013a635a7158a Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 10:35:47 +0800 Subject: [PATCH 14/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/HttpClient.java | 31 ++++---- .../paimon/rest/HttpClientBuildParameter.java | 72 +++++++++++++++++++ .../org/apache/paimon/rest/RESTCatalog.java | 13 +++- 3 files changed, 101 insertions(+), 15 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index adddee32d6aa..2d41bc07f113 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -47,17 +47,19 @@ public class HttpClient implements RESTClient { private final OkHttpClient okHttpClient; private final String endpoint; - private final ObjectMapper mapper = RESTObjectMapper.create(); + private final ObjectMapper mapper; - public HttpClient(String endpoint, String initToken) { + public HttpClient(HttpClientBuildParameter httpClientBuildParameter) { // todo: support config - this.okHttpClient = createHttpClient(1, 3_000, 3_000, initToken); - this.endpoint = endpoint; + this.endpoint = httpClientBuildParameter.getEndpoint(); + this.mapper = httpClientBuildParameter.getMapper(); + this.okHttpClient = createHttpClient(1, 3_000, 3_000, interceptor); } - public HttpClient(OkHttpClient okHttpClient, String endpoint) { + public HttpClient(OkHttpClient okHttpClient, String endpoint, ObjectMapper mapper) { this.okHttpClient = okHttpClient; this.endpoint = endpoint; + this.mapper = mapper; } @Override @@ -99,14 +101,11 @@ private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingExce } private static OkHttpClient createHttpClient( - int threadPoolSize, - long connectTimeoutMillis, - long readTimeoutMillis, - String initToken) { + HttpClientBuildParameter httpClientBuildParameter) { ExecutorService executorService = new ThreadPoolExecutor( - threadPoolSize, - threadPoolSize, + httpClientBuildParameter.getThreadPoolSize(), + httpClientBuildParameter.getThreadPoolSize(), 60, TimeUnit.SECONDS, new SynchronousQueue<>(), @@ -117,11 +116,15 @@ private static OkHttpClient createHttpClient( OkHttpClient.Builder builder = new OkHttpClient.Builder() - .connectTimeout(connectTimeoutMillis, TimeUnit.MILLISECONDS) - .readTimeout(readTimeoutMillis, TimeUnit.MILLISECONDS) + .connectTimeout( + httpClientBuildParameter.getConnectTimeoutMillis(), + TimeUnit.MILLISECONDS) + .readTimeout( + httpClientBuildParameter.getReadTimeoutMillis(), + TimeUnit.MILLISECONDS) .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) - .addInterceptor(new AuthenticationInterceptor(initToken)) + .addInterceptor(httpClientBuildParameter.getInterceptor()) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); return builder.build(); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java new file mode 100644 index 000000000000..d70fdf3cbf13 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java @@ -0,0 +1,72 @@ +/* + * 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.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.Interceptor; + +/** HTTP client build parameter. */ +public class HttpClientBuildParameter { + private final String endpoint; + private final int connectTimeoutMillis; + private final int readTimeoutMillis; + private final Interceptor interceptor; + private final ObjectMapper mapper; + private final int threadPoolSize; + + public HttpClientBuildParameter( + String endpoint, + int connectTimeoutMillis, + int readTimeoutMillis, + Interceptor interceptor, + ObjectMapper mapper, + int threadPoolSize) { + this.endpoint = endpoint; + this.connectTimeoutMillis = connectTimeoutMillis; + this.readTimeoutMillis = readTimeoutMillis; + this.interceptor = interceptor; + this.mapper = mapper; + this.threadPoolSize = threadPoolSize; + } + + public String getEndpoint() { + return endpoint; + } + + public int getConnectTimeoutMillis() { + return connectTimeoutMillis; + } + + public int getReadTimeoutMillis() { + return readTimeoutMillis; + } + + public Interceptor getInterceptor() { + return interceptor; + } + + public ObjectMapper getMapper() { + return mapper; + } + + public int getThreadPoolSize() { + return threadPoolSize; + } +} 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 bbfee94178ce..f58dfe0a7659 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,17 +31,28 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + import java.util.List; import java.util.Map; /** REST catalog. */ public class RESTCatalog implements Catalog { private RESTClient client; + private final ObjectMapper objectMapper = RESTObjectMapper.create(); public RESTCatalog(Options options) { String endpoint = options.get(RESTCatalogOptions.ENDPOINT); // todo: token need config - this.client = new HttpClient(endpoint, "init_token"); + HttpClientBuildParameter httpClientBuildParameter = + new HttpClientBuildParameter( + endpoint, + 3_000, + 3_000, + new AuthenticationInterceptor("init_token"), + objectMapper, + 1); + this.client = new HttpClient(httpClientBuildParameter); } @Override From 3f39e7e1eb9035209021826acd516d8fb8879289 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 10:50:31 +0800 Subject: [PATCH 15/60] [core] Add implementation to support REST Catalog --- .../org/apache/paimon/rest/HttpClient.java | 2 +- .../org/apache/paimon/rest/RESTCatalog.java | 13 ++++++----- .../paimon/rest/RESTCatalogOptions.java | 22 ++++++++++++++++++- 3 files changed, 30 insertions(+), 7 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 2d41bc07f113..ab97d0576fa1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -53,7 +53,7 @@ public HttpClient(HttpClientBuildParameter httpClientBuildParameter) { // todo: support config this.endpoint = httpClientBuildParameter.getEndpoint(); this.mapper = httpClientBuildParameter.getMapper(); - this.okHttpClient = createHttpClient(1, 3_000, 3_000, interceptor); + this.okHttpClient = createHttpClient(httpClientBuildParameter); } public HttpClient(OkHttpClient okHttpClient, String endpoint, ObjectMapper mapper) { 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 f58dfe0a7659..550ccd341a4a 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 @@ -43,15 +43,18 @@ public class RESTCatalog implements Catalog { public RESTCatalog(Options options) { String endpoint = options.get(RESTCatalogOptions.ENDPOINT); - // todo: token need config + String token = options.get(RESTCatalogOptions.TOKEN); + Integer connectTimeoutMillis = options.get(RESTCatalogOptions.CONNECT_TIMEOUT_MILLIS); + Integer readTimeoutMillis = options.get(RESTCatalogOptions.READ_TIMEOUT_MILLIS); + Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); HttpClientBuildParameter httpClientBuildParameter = new HttpClientBuildParameter( endpoint, - 3_000, - 3_000, - new AuthenticationInterceptor("init_token"), + connectTimeoutMillis, + readTimeoutMillis, + new AuthenticationInterceptor(token), objectMapper, - 1); + threadPoolSize); this.client = new HttpClient(httpClientBuildParameter); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 32356d5ce55d..0ec5bbb0d7c3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -24,8 +24,28 @@ /** REST catalog options. */ public class RESTCatalogOptions { public static final ConfigOption ENDPOINT = - ConfigOptions.key("endpoint") + ConfigOptions.key("rest.catalog.endpoint") .stringType() .noDefaultValue() .withDescription("REST Catalog server's endpoint."); + public static final ConfigOption TOKEN = + ConfigOptions.key("rest.catalog.token") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog server's auth token."); + public static final ConfigOption CONNECT_TIMEOUT_MILLIS = + ConfigOptions.key("rest.catalog.connect.timeout.millis") + .intType() + .defaultValue(3_000) + .withDescription("REST Catalog server connect timeout in mills."); + public static final ConfigOption READ_TIMEOUT_MILLIS = + ConfigOptions.key("rest.catalog.read.timeout.millis") + .intType() + .defaultValue(3_000) + .withDescription("REST Catalog server read timeout in mills."); + public static final ConfigOption THREAD_POOL_SIZE = + ConfigOptions.key("rest.catalog.thread.size") + .intType() + .defaultValue(1) + .withDescription("REST Catalog server thread size."); } From 4f993272490b3bf781e1164613b1a15343f09fd9 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 13:51:04 +0800 Subject: [PATCH 16/60] [core] Add implementation to support REST Catalog --- .../paimon/rest/DefaultErrorHandler.java | 61 +++++++++++++++++++ .../org/apache/paimon/rest/ErrorHandler.java | 25 ++++++++ .../org/apache/paimon/rest/HttpClient.java | 21 +++++-- .../org/apache/paimon/rest/RESTCatalog.java | 8 ++- .../rest/exceptions/BadRequestException.java | 31 ++++++++++ .../rest/exceptions/ForbiddenException.java | 26 ++++++++ .../exceptions/NotAuthorizedException.java | 26 ++++++++ .../paimon/rest/exceptions/RESTException.java | 30 +++++++++ .../exceptions/ServiceFailureException.java | 26 ++++++++ .../ServiceUnavailableException.java | 26 ++++++++ .../paimon/rest/responses/ErrorResponse.java | 38 +++++++++++- .../apache/paimon/rest/RESTCatalogTest.java | 2 + 12 files changed, 312 insertions(+), 8 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java new file mode 100644 index 000000000000..a1fbee127c32 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java @@ -0,0 +1,61 @@ +/* + * 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.rest.exceptions.BadRequestException; +import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NotAuthorizedException; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.exceptions.ServiceFailureException; +import org.apache.paimon.rest.exceptions.ServiceUnavailableException; +import org.apache.paimon.rest.responses.ErrorResponse; + +/** Default error handler. */ +public class DefaultErrorHandler extends ErrorHandler { + private static final ErrorHandler INSTANCE = new DefaultErrorHandler(); + + public static ErrorHandler getInstance() { + return INSTANCE; + } + + @Override + public void accept(ErrorResponse error) { + switch (error.getCode()) { + case 400: + throw new BadRequestException( + String.format("Malformed request: %s", error.getMessage())); + case 401: + throw new NotAuthorizedException("Not authorized: %s", error.getMessage()); + case 403: + throw new ForbiddenException("Forbidden: %s", error.getMessage()); + case 405: + case 406: + break; + case 500: + throw new ServiceFailureException("Server error: %s", error.getMessage()); + case 501: + throw new UnsupportedOperationException(error.getMessage()); + case 503: + throw new ServiceUnavailableException( + "Service unavailable: %s", error.getMessage()); + } + + throw new RESTException("Unable to process: %s", error.getMessage()); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java new file mode 100644 index 000000000000..17c9116407ae --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java @@ -0,0 +1,25 @@ +/* + * 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.rest.responses.ErrorResponse; + +import java.util.function.Consumer; + +public abstract class ErrorHandler implements Consumer {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index ab97d0576fa1..83072fbcc3a8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -18,6 +18,9 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.responses.ErrorResponse; + import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -87,11 +90,21 @@ public void close() throws IOException { private T exec(Request request, Class responseType) { try (Response response = okHttpClient.newCall(request).execute()) { - String responseBody = response.body().string(); - return mapper.readValue(responseBody, responseType); - // todo: need handle exception + String responseBodyStr = response.body() != null ? response.body().string() : null; + if (!response.isSuccessful()) { + ErrorResponse error = + new ErrorResponse( + responseBodyStr != null ? responseBodyStr : "response body is null", + response.code(), + null); + DefaultErrorHandler.getInstance().accept(error); + } + if (responseBodyStr == null) { + throw new RESTException("response body is null."); + } + return mapper.readValue(responseBodyStr, responseType); } catch (Exception e) { - throw new RuntimeException(e); + throw new RESTException(e, "rest exception"); } } 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 550ccd341a4a..918c19892cd4 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 @@ -33,6 +33,7 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -66,7 +67,7 @@ public String warehouse() { @Override public Map options() { ConfigResponse response = - client.post("config", new ConfigRequest(), ConfigResponse.class, () -> null); + client.post("config", new ConfigRequest(), ConfigResponse.class, headers()); return response.getDefaults(); } @@ -160,4 +161,9 @@ public boolean allowUpperCase() { @Override public void close() throws Exception {} + + private Map headers() { + Map header = new HashMap<>(); + return header; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java new file mode 100644 index 000000000000..6a4b0b76eb30 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java @@ -0,0 +1,31 @@ +/* + * 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.exceptions; + +/** Exception thrown on HTTP 400 - Bad Request */ +public class BadRequestException extends RESTException { + + public BadRequestException(String message, Object... args) { + super(message, args); + } + + public BadRequestException(Throwable cause, String message, Object... args) { + super(cause, message, args); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java new file mode 100644 index 000000000000..3982e5b70417 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java @@ -0,0 +1,26 @@ +/* + * 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.exceptions; + +/** Exception thrown on HTTP 403 Forbidden. */ +public class ForbiddenException extends RESTException { + public ForbiddenException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java new file mode 100644 index 000000000000..43c13b1a1c97 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java @@ -0,0 +1,26 @@ +/* + * 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.exceptions; + +/** Exception thrown on HTTP 401 Unauthorized. */ +public class NotAuthorizedException extends RESTException { + public NotAuthorizedException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java new file mode 100644 index 000000000000..9dee5aecdf37 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java @@ -0,0 +1,30 @@ +/* + * 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.exceptions; + +/** Base class for REST client exceptions */ +public class RESTException extends RuntimeException { + public RESTException(String message, Object... args) { + super(String.format(message, args)); + } + + public RESTException(Throwable cause, String message, Object... args) { + super(String.format(message, args), cause); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java new file mode 100644 index 000000000000..b2091578067f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java @@ -0,0 +1,26 @@ +/* + * 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.exceptions; + +/** Exception thrown on HTTP 500 - Bad Request */ +public class ServiceFailureException extends RESTException { + public ServiceFailureException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java new file mode 100644 index 000000000000..b1ca8bb88fa0 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java @@ -0,0 +1,26 @@ +/* + * 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.exceptions; + +/** Exception thrown on HTTP 503 - service is unavailable */ +public class ServiceUnavailableException extends RESTException { + public ServiceUnavailableException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index 2db4521689c4..b0954f4add55 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -18,13 +18,45 @@ package org.apache.paimon.rest.responses; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Arrays; import java.util.List; /** Response for error. */ public class ErrorResponse { - List errors; + private final String message; + private final int code; + private List stack; - public List getErrors() { - return errors; + public ErrorResponse(String message, int code, List stack) { + this.message = message; + this.code = code; + this.stack = stack; + } + + public String getMessage() { + return message; + } + + public int getCode() { + return code; + } + + public List getStack() { + return stack; + } + + public void setStack(Throwable throwable) { + StringWriter sw = new StringWriter(); + try (PrintWriter pw = new PrintWriter(sw)) { + throwable.printStackTrace(pw); + } + + this.stack = Arrays.asList(sw.toString().split("\n")); + } + + public void setStack(List trace) { + this.stack = trace; } } 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 04e14dfde7b3..88b0f99799a8 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 @@ -44,6 +44,8 @@ public void setUp() throws IOException { String baseUrl = mockWebServer.url("").toString(); Options options = new Options(); options.set(RESTCatalogOptions.ENDPOINT, baseUrl); + options.set(RESTCatalogOptions.TOKEN, initToken); + options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); restCatalog = new RESTCatalog(options); } From a52eb443a87c9eba2872d96beea4b580f7894903 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 13:53:52 +0800 Subject: [PATCH 17/60] [core] Add implementation to support REST Catalog --- .../apache/paimon/rest/exceptions/BadRequestException.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java index 6a4b0b76eb30..2a35b2ff5e21 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java @@ -24,8 +24,4 @@ public class BadRequestException extends RESTException { public BadRequestException(String message, Object... args) { super(message, args); } - - public BadRequestException(Throwable cause, String message, Object... args) { - super(cause, message, args); - } } From a5886f5782971379cd1647a33681cd0a86feaa2e Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 14:06:51 +0800 Subject: [PATCH 18/60] [core] Add implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/ErrorHandler.java | 1 + .../org/apache/paimon/rest/exceptions/BadRequestException.java | 2 +- .../java/org/apache/paimon/rest/exceptions/RESTException.java | 2 +- .../apache/paimon/rest/exceptions/ServiceFailureException.java | 2 +- .../paimon/rest/exceptions/ServiceUnavailableException.java | 2 +- 5 files changed, 5 insertions(+), 4 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java index 17c9116407ae..cdfa4bcdfaac 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java @@ -22,4 +22,5 @@ import java.util.function.Consumer; +/** Error handler for REST client. */ public abstract class ErrorHandler implements Consumer {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java index 2a35b2ff5e21..301f3bd63f88 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java @@ -18,7 +18,7 @@ package org.apache.paimon.rest.exceptions; -/** Exception thrown on HTTP 400 - Bad Request */ +/** Exception thrown on HTTP 400 - Bad Request. */ public class BadRequestException extends RESTException { public BadRequestException(String message, Object... args) { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java index 9dee5aecdf37..532936f43032 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java @@ -18,7 +18,7 @@ package org.apache.paimon.rest.exceptions; -/** Base class for REST client exceptions */ +/** Base class for REST client exceptions. */ public class RESTException extends RuntimeException { public RESTException(String message, Object... args) { super(String.format(message, args)); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java index b2091578067f..45c48ec0de09 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java @@ -18,7 +18,7 @@ package org.apache.paimon.rest.exceptions; -/** Exception thrown on HTTP 500 - Bad Request */ +/** Exception thrown on HTTP 500 - Bad Request. */ public class ServiceFailureException extends RESTException { public ServiceFailureException(String message, Object... args) { super(String.format(message, args)); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java index b1ca8bb88fa0..fb6a05e89f9f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java @@ -18,7 +18,7 @@ package org.apache.paimon.rest.exceptions; -/** Exception thrown on HTTP 503 - service is unavailable */ +/** Exception thrown on HTTP 503 - service is unavailable. */ public class ServiceUnavailableException extends RESTException { public ServiceUnavailableException(String message, Object... args) { super(String.format(message, args)); From da2a2cf838218f921dfd63fbe72611b93a8fb74f Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 14:40:32 +0800 Subject: [PATCH 19/60] [core] Add basic implementation to support REST Catalog --- paimon-open-api/rest-catalog-open-api.yaml | 62 +++++++++++++++++++ .../paimon/open/api/config/OpenAPIConfig.java | 15 +++-- .../src/main/resources/application.properties | 3 +- 3 files changed, 71 insertions(+), 9 deletions(-) create mode 100644 paimon-open-api/rest-catalog-open-api.yaml diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml new file mode 100644 index 000000000000..5aa1f36cb2b2 --- /dev/null +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -0,0 +1,62 @@ +# +# 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. +# + +--- +openapi: 3.0.1 +info: + title: RESTCatalog API + description: This API exposes endpoints to RESTCatalog. + license: + name: MIT License + url: https://choosealicense.com/licenses/mit/ + version: '1.0' +servers: + - url: http://localhost:8080 + description: Server URL in Development environment +paths: + /api/v1/{prefix}/config: + post: + tags: + - config + summary: Get Config + operationId: getConfig + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/ConfigRequest' + required: true + responses: + '201': + description: Created + content: + application/json: + schema: + $ref: '#/components/schemas/ConfigResponse' + '500': + description: Internal Server Error +components: + schemas: + ConfigRequest: + type: object + ConfigResponse: + type: object + properties: + defaults: + type: object + additionalProperties: + type: string diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java index 8bbccbc8d2eb..72c29a178afd 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -33,27 +33,26 @@ @Configuration public class OpenAPIConfig { - @Value("${openapi.dev-url}") + @Value("${openapi.url}") private String devUrl; @Bean public OpenAPI myOpenAPI() { - Server devServer = new Server(); - devServer.setUrl(devUrl); - devServer.setDescription("Server URL in Development environment"); + Server server = new Server(); + server.setUrl(devUrl); + server.setDescription("Server URL in Development environment"); License mitLicense = new License().name("MIT License").url("https://choosealicense.com/licenses/mit/"); Info info = new Info() - .title("Tutorial Management API") + .title("RESTCatalog API") .version("1.0") - .description("This API exposes endpoints to manage tutorials.") - .termsOfService("https://www.bezkoder.com/terms") + .description("This API exposes endpoints to RESTCatalog.") .license(mitLicense); List servers = new ArrayList<>(); - servers.add(devServer); + servers.add(server); return new OpenAPI().info(info).servers(servers); } } diff --git a/paimon-open-api/src/main/resources/application.properties b/paimon-open-api/src/main/resources/application.properties index f0b377ffff8f..58a975161145 100644 --- a/paimon-open-api/src/main/resources/application.properties +++ b/paimon-open-api/src/main/resources/application.properties @@ -16,6 +16,7 @@ # springdoc.swagger-ui.path=/swagger-api springdoc.api-docs.path=/swagger-api-docs +springdoc.swagger-ui.deepLinking=true springdoc.swagger-ui.tryItOutEnabled=true springdoc.swagger-ui.filter=true -openapi.dev-url=http://localhost:8080 +openapi.url=http://localhost:8080 From d70fe359d8e855d0c5f09b4c662ab58a615f2ff4 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 15:01:44 +0800 Subject: [PATCH 20/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 30 +++++----- .../rest/AuthenticationInterceptor.java | 59 ------------------- .../org/apache/paimon/rest/HttpClient.java | 1 - .../paimon/rest/HttpClientBuildParameter.java | 9 --- .../org/apache/paimon/rest/RESTCatalog.java | 6 +- .../apache/paimon/rest/RESTCatalogTest.java | 18 ------ 6 files changed, 20 insertions(+), 103 deletions(-) delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 3377414662b3..41d435235abb 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -63,6 +63,14 @@ under the License. provided + + + + com.squareup.okhttp3 + okhttp + 4.12.0 + + @@ -79,20 +87,7 @@ under the License. ${project.version} provided - - - - com.squareup.okhttp3 - okhttp - 4.12.0 - - - - com.squareup.okhttp3 - mockwebserver - 4.12.0 - test - + org.apache.hadoop hadoop-common @@ -217,6 +212,13 @@ under the License. test + + com.squareup.okhttp3 + mockwebserver + 4.12.0 + test + + diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java b/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java deleted file mode 100644 index 484e436bad7c..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/AuthenticationInterceptor.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest; - -import okhttp3.Interceptor; -import okhttp3.Request; -import okhttp3.Response; - -import java.io.IOException; - -/** Authentication interceptor. */ -public class AuthenticationInterceptor implements Interceptor { - private String refreshToken; - - public AuthenticationInterceptor(String refreshToken) { - this.refreshToken = refreshToken; - } - - @Override - public Response intercept(Chain chain) throws IOException { - Request request = chain.request(); - Request.Builder requestBuilder = - request.newBuilder().header("Authorization", "Bearer " + refreshToken); - Response response = chain.proceed(requestBuilder.build()); - // Unauthorized - if (response.code() == 401) { - // Refresh the token here - refreshToken = refreshTokenFromServer(); - requestBuilder = request.newBuilder().header("Authorization", "Bearer " + refreshToken); - response.close(); - response = chain.proceed(requestBuilder.build()); - } - response = - response.newBuilder().addHeader("Authorization", "Bearer " + refreshToken).build(); - return response; - } - - private String refreshTokenFromServer() { - // Make a request to the server to refresh the token - // and return the new token - return "new_refreshed_token"; - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 83072fbcc3a8..af5630debb65 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -137,7 +137,6 @@ private static OkHttpClient createHttpClient( TimeUnit.MILLISECONDS) .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) - .addInterceptor(httpClientBuildParameter.getInterceptor()) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); return builder.build(); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java index d70fdf3cbf13..ae5257739e3e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java @@ -20,14 +20,11 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import okhttp3.Interceptor; - /** HTTP client build parameter. */ public class HttpClientBuildParameter { private final String endpoint; private final int connectTimeoutMillis; private final int readTimeoutMillis; - private final Interceptor interceptor; private final ObjectMapper mapper; private final int threadPoolSize; @@ -35,13 +32,11 @@ public HttpClientBuildParameter( String endpoint, int connectTimeoutMillis, int readTimeoutMillis, - Interceptor interceptor, ObjectMapper mapper, int threadPoolSize) { this.endpoint = endpoint; this.connectTimeoutMillis = connectTimeoutMillis; this.readTimeoutMillis = readTimeoutMillis; - this.interceptor = interceptor; this.mapper = mapper; this.threadPoolSize = threadPoolSize; } @@ -58,10 +53,6 @@ public int getReadTimeoutMillis() { return readTimeoutMillis; } - public Interceptor getInterceptor() { - return interceptor; - } - public ObjectMapper getMapper() { return mapper; } 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 918c19892cd4..a6bff575f7d5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -41,10 +41,11 @@ public class RESTCatalog implements Catalog { private RESTClient client; private final ObjectMapper objectMapper = RESTObjectMapper.create(); + private String token; public RESTCatalog(Options options) { String endpoint = options.get(RESTCatalogOptions.ENDPOINT); - String token = options.get(RESTCatalogOptions.TOKEN); + token = options.get(RESTCatalogOptions.TOKEN); Integer connectTimeoutMillis = options.get(RESTCatalogOptions.CONNECT_TIMEOUT_MILLIS); Integer readTimeoutMillis = options.get(RESTCatalogOptions.READ_TIMEOUT_MILLIS); Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); @@ -53,7 +54,6 @@ public RESTCatalog(Options options) { endpoint, connectTimeoutMillis, readTimeoutMillis, - new AuthenticationInterceptor(token), objectMapper, threadPoolSize); this.client = new HttpClient(httpClientBuildParameter); @@ -163,7 +163,9 @@ public boolean allowUpperCase() { public void close() throws Exception {} private Map headers() { + // todo: need refresh token Map header = new HashMap<>(); + header.put("Authorization", "Bearer " + token); return header; } } 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 88b0f99799a8..0e24f5b5528a 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 @@ -65,22 +65,4 @@ public void testGetConfig() { Map response = restCatalog.options(); assertEquals("b", response.get("a")); } - - @Test - public void testNeedAuth() { - String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; - MockResponse mockResponseObj401 = - new MockResponse() - .setBody(mockResponse) - .setResponseCode(401) - .addHeader("Content-Type", "application/json"); - MockResponse mockResponseObj200 = - new MockResponse() - .setBody(mockResponse) - .addHeader("Content-Type", "application/json"); - mockWebServer.enqueue(mockResponseObj401); - mockWebServer.enqueue(mockResponseObj200); - Map response = restCatalog.options(); - assertEquals("b", response.get("a")); - } } From 289fbb95b59b0ac7aabe6e63fe805237ad9ec738 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 20 Nov 2024 15:20:47 +0800 Subject: [PATCH 21/60] [core] Add basic implementation to support REST Catalog --- paimon-open-api/rest-catalog-open-api.yaml | 10 ++++++++-- .../apache/paimon/open/api/RESTCatalogController.java | 4 +++- .../apache/paimon/open/api/config/OpenAPIConfig.java | 4 +++- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 5aa1f36cb2b2..e886b46a30ca 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -21,8 +21,8 @@ info: title: RESTCatalog API description: This API exposes endpoints to RESTCatalog. license: - name: MIT License - url: https://choosealicense.com/licenses/mit/ + name: Apache 2.0 + url: https://www.apache.org/licenses/LICENSE-2.0.html version: '1.0' servers: - url: http://localhost:8080 @@ -34,6 +34,12 @@ paths: - config summary: Get Config operationId: getConfig + parameters: + - name: prefix + in: path + required: true + schema: + type: string requestBody: content: application/json: diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java index d51a37fb0882..aa61735fb142 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -29,6 +29,7 @@ import org.springframework.http.HttpStatus; import org.springframework.http.ResponseEntity; import org.springframework.web.bind.annotation.CrossOrigin; +import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.PostMapping; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; @@ -59,7 +60,8 @@ public class RESTCatalogController { content = {@Content(schema = @Schema())}) }) @PostMapping("/config") - public ResponseEntity getConfig(@RequestBody ConfigRequest request) { + public ResponseEntity getConfig( + @PathVariable String prefix, @RequestBody ConfigRequest request) { try { Map defaults = new HashMap<>(); ConfigResponse response = new ConfigResponse(defaults); diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java index 72c29a178afd..f458d21dc774 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -43,7 +43,9 @@ public OpenAPI myOpenAPI() { server.setDescription("Server URL in Development environment"); License mitLicense = - new License().name("MIT License").url("https://choosealicense.com/licenses/mit/"); + new License() + .name("Apache 2.0") + .url("https://www.apache.org/licenses/LICENSE-2.0.html"); Info info = new Info() From b0faa008fe6324a60f1a80e7b3474fadc563e434 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 11:20:53 +0800 Subject: [PATCH 22/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 41d435235abb..9aa0fbaeddfe 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -63,8 +63,8 @@ under the License. provided - - + + com.squareup.okhttp3 okhttp @@ -234,6 +234,27 @@ under the License. + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + okhttp3 + org.apache.paimon.shade.okhttp + + + + + + From f46cff37e088f9774ef1f3740e8cb5c477aa4368 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 11:22:39 +0800 Subject: [PATCH 23/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 9aa0fbaeddfe..bb1f0285b007 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -248,7 +248,7 @@ under the License. okhttp3 - org.apache.paimon.shade.okhttp + org.apache.paimon.shade.okhttp3 From fc3fdb57bf5c4f0eb590e04b885560589684e10a Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 13:48:45 +0800 Subject: [PATCH 24/60] [core] Add basic implementation to support REST Catalog --- .../paimon/rest/DefaultErrorHandler.java | 17 ++++++++--------- .../org/apache/paimon/rest/HttpClient.java | 14 ++++++-------- .../paimon/rest/HttpClientBuildParameter.java | 10 +++++----- .../org/apache/paimon/rest/RESTCatalog.java | 4 ++-- .../apache/paimon/rest/RESTCatalogFactory.java | 2 +- .../apache/paimon/rest/RESTCatalogOptions.java | 2 +- .../org/apache/paimon/rest/RESTClient.java | 2 +- .../org/apache/paimon/rest/RESTMessage.java | 2 +- .../apache/paimon/rest/RESTObjectMapper.java | 2 +- .../org/apache/paimon/rest/RESTRequest.java | 2 +- .../org/apache/paimon/rest/RESTResponse.java | 2 +- .../paimon/rest/responses/ConfigResponse.java | 10 ++-------- .../paimon/rest/responses/ErrorResponse.java | 18 +++++++----------- .../apache/paimon/rest/RESTCatalogTest.java | 2 +- .../paimon/open/api/config/OpenAPIConfig.java | 4 ++-- 15 files changed, 40 insertions(+), 53 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java index a1fbee127c32..607bcde49ecf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java @@ -36,26 +36,25 @@ public static ErrorHandler getInstance() { @Override public void accept(ErrorResponse error) { - switch (error.getCode()) { + switch (error.code()) { case 400: throw new BadRequestException( - String.format("Malformed request: %s", error.getMessage())); + String.format("Malformed request: %s", error.message())); case 401: - throw new NotAuthorizedException("Not authorized: %s", error.getMessage()); + throw new NotAuthorizedException("Not authorized: %s", error.message()); case 403: - throw new ForbiddenException("Forbidden: %s", error.getMessage()); + throw new ForbiddenException("Forbidden: %s", error.message()); case 405: case 406: break; case 500: - throw new ServiceFailureException("Server error: %s", error.getMessage()); + throw new ServiceFailureException("Server error: %s", error.message()); case 501: - throw new UnsupportedOperationException(error.getMessage()); + throw new UnsupportedOperationException(error.message()); case 503: - throw new ServiceUnavailableException( - "Service unavailable: %s", error.getMessage()); + throw new ServiceUnavailableException("Service unavailable: %s", error.message()); } - throw new RESTException("Unable to process: %s", error.getMessage()); + throw new RESTException("Unable to process: %s", error.message()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index af5630debb65..60ab794077d5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -53,9 +53,8 @@ public class HttpClient implements RESTClient { private final ObjectMapper mapper; public HttpClient(HttpClientBuildParameter httpClientBuildParameter) { - // todo: support config - this.endpoint = httpClientBuildParameter.getEndpoint(); - this.mapper = httpClientBuildParameter.getMapper(); + this.endpoint = httpClientBuildParameter.endpoint(); + this.mapper = httpClientBuildParameter.mapper(); this.okHttpClient = createHttpClient(httpClientBuildParameter); } @@ -117,8 +116,8 @@ private static OkHttpClient createHttpClient( HttpClientBuildParameter httpClientBuildParameter) { ExecutorService executorService = new ThreadPoolExecutor( - httpClientBuildParameter.getThreadPoolSize(), - httpClientBuildParameter.getThreadPoolSize(), + httpClientBuildParameter.threadPoolSize(), + httpClientBuildParameter.threadPoolSize(), 60, TimeUnit.SECONDS, new SynchronousQueue<>(), @@ -130,11 +129,10 @@ private static OkHttpClient createHttpClient( OkHttpClient.Builder builder = new OkHttpClient.Builder() .connectTimeout( - httpClientBuildParameter.getConnectTimeoutMillis(), + httpClientBuildParameter.connectTimeoutMillis(), TimeUnit.MILLISECONDS) .readTimeout( - httpClientBuildParameter.getReadTimeoutMillis(), - TimeUnit.MILLISECONDS) + httpClientBuildParameter.readTimeoutMillis(), TimeUnit.MILLISECONDS) .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java index ae5257739e3e..7fbab1806eb7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java @@ -41,23 +41,23 @@ public HttpClientBuildParameter( this.threadPoolSize = threadPoolSize; } - public String getEndpoint() { + public String endpoint() { return endpoint; } - public int getConnectTimeoutMillis() { + public int connectTimeoutMillis() { return connectTimeoutMillis; } - public int getReadTimeoutMillis() { + public int readTimeoutMillis() { return readTimeoutMillis; } - public ObjectMapper getMapper() { + public ObjectMapper mapper() { return mapper; } - public int getThreadPoolSize() { + public int threadPoolSize() { return threadPoolSize; } } 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 a6bff575f7d5..1684189de356 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 @@ -37,7 +37,7 @@ import java.util.List; import java.util.Map; -/** REST catalog. */ +/** A catalog implementation for REST. */ public class RESTCatalog implements Catalog { private RESTClient client; private final ObjectMapper objectMapper = RESTObjectMapper.create(); @@ -68,7 +68,7 @@ public String warehouse() { public Map options() { ConfigResponse response = client.post("config", new ConfigRequest(), ConfigResponse.class, headers()); - return response.getDefaults(); + return response.defaults(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index 4b6d77c9f5a3..ca0f3c0ddf9c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -24,7 +24,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; -/** REST catalog factory. */ +/** Factory to create {@link RESTCatalog}. */ public class RESTCatalogFactory implements CatalogFactory { public static final String IDENTIFIER = "rest"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 0ec5bbb0d7c3..14e97ad5c326 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -21,7 +21,7 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; -/** REST catalog options. */ +/** Options for REST Catalog. */ public class RESTCatalogOptions { public static final ConfigOption ENDPOINT = ConfigOptions.key("rest.catalog.endpoint") diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java index 71a8d65ddaa4..4b3039d4ce53 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -22,7 +22,7 @@ import java.util.Map; import java.util.function.Supplier; -/** REST client. */ +/** Interface for a basic HTTP Client for interfacing with the REST catalog. */ public interface RESTClient extends Closeable { default T post( String path, diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java index 110fb7978079..6cb0b6fa6573 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java @@ -18,5 +18,5 @@ package org.apache.paimon.rest; -/** REST message. */ +/** Interface to mark both REST requests and responses. */ public interface RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java index 173cb861b007..b1c83e90224a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -23,7 +23,7 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.SerializationFeature; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; -/** REST object mapper. */ +/** Object mapper for REST request and response. */ public class RESTObjectMapper { public static ObjectMapper create() { ObjectMapper mapper = new ObjectMapper(); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java index 44bfa5ae31f8..9c6758df14f0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java @@ -18,5 +18,5 @@ package org.apache.paimon.rest; -/** REST message. */ +/** Interface to mark a REST request. */ public interface RESTRequest extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java index 2e40fe645beb..a4149d3fda14 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java @@ -18,5 +18,5 @@ package org.apache.paimon.rest; -/** REST response. */ +/** Interface to mark a REST response. */ public interface RESTResponse extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index 1c65d3076049..ea7f0d76858a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -22,21 +22,15 @@ import java.util.Map; -/** Response to get config. */ +/** Response for getting config. */ public class ConfigResponse implements RESTResponse { private Map defaults; - public ConfigResponse() {} - public ConfigResponse(Map defaults) { this.defaults = defaults; } - public Map getDefaults() { + public Map defaults() { return defaults; } - - public void setDefaults(Map defaults) { - this.defaults = defaults; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index b0954f4add55..dc53976c1f9f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -29,34 +29,30 @@ public class ErrorResponse { private final int code; private List stack; - public ErrorResponse(String message, int code, List stack) { + public ErrorResponse(String message, int code, Throwable throwable) { this.message = message; this.code = code; - this.stack = stack; + this.stack = getStackFromThrowable(throwable); } - public String getMessage() { + public String message() { return message; } - public int getCode() { + public int code() { return code; } - public List getStack() { + public List stack() { return stack; } - public void setStack(Throwable throwable) { + private List getStackFromThrowable(Throwable throwable) { StringWriter sw = new StringWriter(); try (PrintWriter pw = new PrintWriter(sw)) { throwable.printStackTrace(pw); } - this.stack = Arrays.asList(sw.toString().split("\n")); - } - - public void setStack(List trace) { - this.stack = trace; + return Arrays.asList(sw.toString().split("\n")); } } 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 0e24f5b5528a..3ff7b3e324b9 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 @@ -31,7 +31,7 @@ import static org.junit.Assert.assertEquals; -/** REST catalog api test. */ +/** Test for REST Catalog. */ public class RESTCatalogTest { private MockWebServer mockWebServer; private RESTCatalog restCatalog; diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java index f458d21dc774..01234c41bbff 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -29,7 +29,7 @@ import java.util.ArrayList; import java.util.List; -/** OpenAPI config. */ +/** Config for OpenAPI. */ @Configuration public class OpenAPIConfig { @@ -37,7 +37,7 @@ public class OpenAPIConfig { private String devUrl; @Bean - public OpenAPI myOpenAPI() { + public OpenAPI restCatalogOpenAPI() { Server server = new Server(); server.setUrl(devUrl); server.setDescription("Server URL in Development environment"); From 903f71a814c83b829e7d44c35624e9eb82127b5a Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 14:12:30 +0800 Subject: [PATCH 25/60] [core] Add basic implementation to support REST Catalog --- .../paimon/rest/responses/ErrorResponse.java | 4 + .../paimon/rest/DefaultErrorHandlerTest.java | 76 +++++++++++++++++++ 2 files changed, 80 insertions(+) create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index dc53976c1f9f..df33768f8c31 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -20,6 +20,7 @@ import java.io.PrintWriter; import java.io.StringWriter; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -48,6 +49,9 @@ public List stack() { } private List getStackFromThrowable(Throwable throwable) { + if (throwable == null) { + return new ArrayList(); + } StringWriter sw = new StringWriter(); try (PrintWriter pw = new PrintWriter(sw)) { throwable.printStackTrace(pw); diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java new file mode 100644 index 000000000000..fd2658a3849e --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java @@ -0,0 +1,76 @@ +/* + * 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.rest.exceptions.BadRequestException; +import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NotAuthorizedException; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.exceptions.ServiceFailureException; +import org.apache.paimon.rest.exceptions.ServiceUnavailableException; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertThrows; + +/** Test for {@link DefaultErrorHandler}. */ +public class DefaultErrorHandlerTest { + private ErrorHandler defaultErrorHandler; + + @Before + public void setUp() throws IOException { + defaultErrorHandler = DefaultErrorHandler.getInstance(); + } + + @Test + public void testHandleErrorResponse() { + assertThrows( + BadRequestException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(400))); + assertThrows( + NotAuthorizedException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(401))); + assertThrows( + ForbiddenException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(403))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(405))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(406))); + assertThrows( + ServiceFailureException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(500))); + assertThrows( + UnsupportedOperationException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(501))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(502))); + assertThrows( + ServiceUnavailableException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(503))); + } + + private ErrorResponse generateErrorResponse(int code) { + return new ErrorResponse("message", code, null); + } +} From acb64fc40dd976ab836de85c15d6cdc39ee3282e Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 14:58:39 +0800 Subject: [PATCH 26/60] [core] Add basic implementation to support REST Catalog --- .../paimon/rest/requests/ConfigRequest.java | 5 +++++ .../paimon/rest/responses/ConfigResponse.java | 9 ++++++++ .../apache/paimon/rest/HttpClientTest.java | 22 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java index 7f07e3297a75..c671a944301c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java @@ -20,7 +20,12 @@ import org.apache.paimon.rest.RESTRequest; +import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + /** Request to get config. */ +@JsonIgnoreProperties(ignoreUnknown = true) public class ConfigRequest implements RESTRequest { + @JsonCreator public ConfigRequest() {} } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index ea7f0d76858a..f5b5a1b84612 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -20,12 +20,21 @@ import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import java.beans.ConstructorProperties; import java.util.Map; /** Response for getting config. */ +@JsonIgnoreProperties(ignoreUnknown = true) public class ConfigResponse implements RESTResponse { + private static final String FIELD_DEFAULTS = "defaults"; + + @JsonProperty(FIELD_DEFAULTS) private Map defaults; + @ConstructorProperties({FIELD_DEFAULTS}) public ConfigResponse(Map defaults) { this.defaults = defaults; } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java new file mode 100644 index 000000000000..c3c2802c2ce1 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -0,0 +1,22 @@ +/* + * 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; + +/** Test for {@link HttpClient}. */ +public class HttpClientTest {} From d5f253cbf2f40ffb37d50c3f38d9d1bf1815bfb7 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 15:24:26 +0800 Subject: [PATCH 27/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 6 ++ .../paimon/rest/HttpClientBuildParameter.java | 9 ++- .../org/apache/paimon/rest/RESTCatalog.java | 3 +- .../apache/paimon/rest/HttpClientTest.java | 63 ++++++++++++++++++- .../org/apache/paimon/rest/MockRESTData.java | 41 ++++++++++++ 5 files changed, 119 insertions(+), 3 deletions(-) create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index bb1f0285b007..9c8eb3cf45d7 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -218,6 +218,12 @@ under the License. 4.12.0 test + + org.mockito + mockito-core + 5.3.1 + test + diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java index 7fbab1806eb7..1d61e4ef6370 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java @@ -27,18 +27,21 @@ public class HttpClientBuildParameter { private final int readTimeoutMillis; private final ObjectMapper mapper; private final int threadPoolSize; + private final ErrorHandler errorHandler; public HttpClientBuildParameter( String endpoint, int connectTimeoutMillis, int readTimeoutMillis, ObjectMapper mapper, - int threadPoolSize) { + int threadPoolSize, + ErrorHandler errorHandler) { this.endpoint = endpoint; this.connectTimeoutMillis = connectTimeoutMillis; this.readTimeoutMillis = readTimeoutMillis; this.mapper = mapper; this.threadPoolSize = threadPoolSize; + this.errorHandler = errorHandler; } public String endpoint() { @@ -60,4 +63,8 @@ public ObjectMapper mapper() { public int threadPoolSize() { return threadPoolSize; } + + public ErrorHandler errorHandler() { + return errorHandler; + } } 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 1684189de356..c4b8dbf71c01 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 @@ -55,7 +55,8 @@ public RESTCatalog(Options options) { connectTimeoutMillis, readTimeoutMillis, objectMapper, - threadPoolSize); + threadPoolSize, + DefaultErrorHandler.getInstance()); this.client = new HttpClient(httpClientBuildParameter); } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index c3c2802c2ce1..ed95258f2cb4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -18,5 +18,66 @@ package org.apache.paimon.rest; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.mockwebserver.MockResponse; +import okhttp3.mockwebserver.MockWebServer; +import org.junit.After; +import org.junit.Before; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + /** Test for {@link HttpClient}. */ -public class HttpClientTest {} +public class HttpClientTest { + private MockWebServer mockWebServer; + private HttpClient httpClient; + private ObjectMapper objectMapper = RESTObjectMapper.create(); + private ErrorHandler errorHandler; + + @Before + public void setUp() throws IOException { + mockWebServer = new MockWebServer(); + mockWebServer.start(); + String baseUrl = mockWebServer.url("").toString(); + errorHandler = mock(ErrorHandler.class); + HttpClientBuildParameter httpClientBuildParameter = + new HttpClientBuildParameter(baseUrl, 1000, 1000, objectMapper, 1, errorHandler); + httpClient = new HttpClient(httpClientBuildParameter); + } + + @After + public void tearDown() throws IOException { + mockWebServer.shutdown(); + } + + @Test + public void testPostSuccess() throws Exception { + MockRESTData mockResponseData = new MockRESTData("test"); + String mockResponse = objectMapper.writeValueAsString(mockResponseData); + MockResponse mockResponseObj = + new MockResponse() + .setBody(mockResponse) + .addHeader("Content-Type", "application/json"); + mockWebServer.enqueue(mockResponseObj); + MockRESTData response = + httpClient.post("/test", mockResponseData, MockRESTData.class, headers("token")); + verify(errorHandler, times(0)).accept(any()); + assertEquals(mockResponseData.data(), response.data()); + } + + private Map headers(String token) { + // todo: need refresh token + Map header = new HashMap<>(); + header.put("Authorization", "Bearer " + token); + return header; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java new file mode 100644 index 000000000000..ddcdf0831005 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -0,0 +1,41 @@ +/* + * 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.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import java.beans.ConstructorProperties; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class MockRESTData implements RESTRequest, RESTResponse { + private static final String FIELD_DATA = "data"; + + @JsonProperty(FIELD_DATA) + private String data; + + @ConstructorProperties({FIELD_DATA}) + public MockRESTData(String data) { + this.data = data; + } + + public String data() { + return data; + } +} From 6088452e56b88032b8cc1c2698d1cf4a4e15f9fe Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 15:47:55 +0800 Subject: [PATCH 28/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 3 ++- .../main/java/org/apache/paimon/rest/HttpClient.java | 10 +++------- .../main/java/org/apache/paimon/rest/RESTClient.java | 8 -------- .../java/org/apache/paimon/rest/HttpClientTest.java | 10 ++++------ .../test/java/org/apache/paimon/rest/MockRESTData.java | 6 ++++++ 5 files changed, 15 insertions(+), 22 deletions(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 9c8eb3cf45d7..d6c05f587186 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -221,7 +221,8 @@ under the License. org.mockito mockito-core - 5.3.1 + ${mockito.version} + jar test diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 60ab794077d5..96ec5b462b2d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -51,17 +51,13 @@ public class HttpClient implements RESTClient { private final OkHttpClient okHttpClient; private final String endpoint; private final ObjectMapper mapper; + private final ErrorHandler errorHandler; public HttpClient(HttpClientBuildParameter httpClientBuildParameter) { this.endpoint = httpClientBuildParameter.endpoint(); this.mapper = httpClientBuildParameter.mapper(); this.okHttpClient = createHttpClient(httpClientBuildParameter); - } - - public HttpClient(OkHttpClient okHttpClient, String endpoint, ObjectMapper mapper) { - this.okHttpClient = okHttpClient; - this.endpoint = endpoint; - this.mapper = mapper; + this.errorHandler = httpClientBuildParameter.errorHandler(); } @Override @@ -96,7 +92,7 @@ private T exec(Request request, Class responseType) responseBodyStr != null ? responseBodyStr : "response body is null", response.code(), null); - DefaultErrorHandler.getInstance().accept(error); + errorHandler.accept(error); } if (responseBodyStr == null) { throw new RESTException("response body is null."); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java index 4b3039d4ce53..bc1e7ccdb39d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -20,17 +20,9 @@ import java.io.Closeable; import java.util.Map; -import java.util.function.Supplier; /** Interface for a basic HTTP Client for interfacing with the REST catalog. */ public interface RESTClient extends Closeable { - default T post( - String path, - RESTRequest body, - Class responseType, - Supplier> headers) { - return post(path, body, responseType, headers.get()); - } T post( String path, RESTRequest body, Class responseType, Map headers); diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index ed95258f2cb4..1df9caed35ec 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -24,7 +24,7 @@ import okhttp3.mockwebserver.MockWebServer; import org.junit.After; import org.junit.Before; -import org.junit.jupiter.api.Test; +import org.junit.Test; import java.io.IOException; import java.util.HashMap; @@ -60,22 +60,20 @@ public void tearDown() throws IOException { } @Test - public void testPostSuccess() throws Exception { + public void testPostSuccess() { MockRESTData mockResponseData = new MockRESTData("test"); - String mockResponse = objectMapper.writeValueAsString(mockResponseData); MockResponse mockResponseObj = new MockResponse() - .setBody(mockResponse) + .setBody(mockResponseData.toString()) .addHeader("Content-Type", "application/json"); mockWebServer.enqueue(mockResponseObj); MockRESTData response = - httpClient.post("/test", mockResponseData, MockRESTData.class, headers("token")); + httpClient.post("test", mockResponseData, MockRESTData.class, headers("token")); verify(errorHandler, times(0)).accept(any()); assertEquals(mockResponseData.data(), response.data()); } private Map headers(String token) { - // todo: need refresh token Map header = new HashMap<>(); header.put("Authorization", "Bearer " + token); return header; diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java index ddcdf0831005..155018a519bc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -23,6 +23,7 @@ import java.beans.ConstructorProperties; +/** Mock REST data. */ @JsonIgnoreProperties(ignoreUnknown = true) public class MockRESTData implements RESTRequest, RESTResponse { private static final String FIELD_DATA = "data"; @@ -38,4 +39,9 @@ public MockRESTData(String data) { public String data() { return data; } + + @Override + public String toString() { + return String.format("{\"data\": \"%s\"}", data); + } } From cd2ffd664868e61d1d1201f2615f68af8e9871f3 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 16:07:01 +0800 Subject: [PATCH 29/60] [core] Add basic implementation to support REST Catalog --- .../apache/paimon/rest/HttpClientTest.java | 29 +++++++++++++++---- .../org/apache/paimon/rest/MockRESTData.java | 10 ++++--- 2 files changed, 29 insertions(+), 10 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index 1df9caed35ec..c0bb9eda8ff7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -42,6 +42,9 @@ public class HttpClientTest { private HttpClient httpClient; private ObjectMapper objectMapper = RESTObjectMapper.create(); private ErrorHandler errorHandler; + private MockRESTData mockResponseData; + private String mockResponseDataStr; + private Map headers; @Before public void setUp() throws IOException { @@ -51,7 +54,10 @@ public void setUp() throws IOException { errorHandler = mock(ErrorHandler.class); HttpClientBuildParameter httpClientBuildParameter = new HttpClientBuildParameter(baseUrl, 1000, 1000, objectMapper, 1, errorHandler); + mockResponseData = new MockRESTData("test"); + mockResponseDataStr = objectMapper.writeValueAsString(mockResponseData); httpClient = new HttpClient(httpClientBuildParameter); + headers = headers("token"); } @After @@ -61,21 +67,32 @@ public void tearDown() throws IOException { @Test public void testPostSuccess() { - MockRESTData mockResponseData = new MockRESTData("test"); - MockResponse mockResponseObj = - new MockResponse() - .setBody(mockResponseData.toString()) - .addHeader("Content-Type", "application/json"); + MockResponse mockResponseObj = generateMockResponse(mockResponseDataStr, 200); mockWebServer.enqueue(mockResponseObj); MockRESTData response = - httpClient.post("test", mockResponseData, MockRESTData.class, headers("token")); + httpClient.post("test", mockResponseData, MockRESTData.class, headers); verify(errorHandler, times(0)).accept(any()); assertEquals(mockResponseData.data(), response.data()); } + @Test + public void testPostFail() { + MockResponse mockResponseObj = generateMockResponse(mockResponseDataStr, 400); + mockWebServer.enqueue(mockResponseObj); + httpClient.post("test", mockResponseData, MockRESTData.class, headers); + verify(errorHandler, times(1)).accept(any()); + } + private Map headers(String token) { Map header = new HashMap<>(); header.put("Authorization", "Bearer " + token); return header; } + + private MockResponse generateMockResponse(String data, Integer code) { + return new MockResponse() + .setResponseCode(code) + .setBody(data) + .addHeader("Content-Type", "application/json"); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java index 155018a519bc..4fe1d1899f8d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -19,6 +19,7 @@ package org.apache.paimon.rest; import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import java.beans.ConstructorProperties; @@ -36,12 +37,13 @@ public MockRESTData(String data) { this.data = data; } + @JsonGetter(FIELD_DATA) public String data() { return data; } - @Override - public String toString() { - return String.format("{\"data\": \"%s\"}", data); - } + // @Override + // public String toString() { + // return String.format("{\"data\": \"%s\"}", data); + // } } From ef7e9c3e9a38e3dfe2609e94ad40b33ef7dd10a5 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 16:24:59 +0800 Subject: [PATCH 30/60] [core] Add basic implementation to support REST Catalog --- .../paimon/rest/responses/ConfigResponse.java | 2 + .../paimon/rest/responses/ErrorResponse.java | 29 ++++++++- .../paimon/rest/DefaultErrorHandlerTest.java | 3 +- .../paimon/rest/RESTObjectMapperTest.java | 59 +++++++++++++++++++ 4 files changed, 89 insertions(+), 4 deletions(-) create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index f5b5a1b84612..b4b32a8e728d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -21,6 +21,7 @@ import org.apache.paimon.rest.RESTResponse; import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import java.beans.ConstructorProperties; @@ -39,6 +40,7 @@ public ConfigResponse(Map defaults) { this.defaults = defaults; } + @JsonGetter(FIELD_DEFAULTS) public Map defaults() { return defaults; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index df33768f8c31..b7bfbed33039 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -18,6 +18,10 @@ package org.apache.paimon.rest.responses; +import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; + +import java.beans.ConstructorProperties; import java.io.PrintWriter; import java.io.StringWriter; import java.util.ArrayList; @@ -26,9 +30,25 @@ /** Response for error. */ public class ErrorResponse { + private static final String FIELD_MESSAGE = "message"; + private static final String FIELD_CODE = "code"; + private static final String FIELD_STACK = "stack"; + + @JsonProperty(FIELD_MESSAGE) private final String message; - private final int code; - private List stack; + + @JsonProperty(FIELD_CODE) + private final Integer code; + + @JsonProperty(FIELD_STACK) + private final List stack; + + @ConstructorProperties({FIELD_MESSAGE, FIELD_CODE, FIELD_STACK}) + public ErrorResponse(String message, int code, List stack) { + this.message = message; + this.code = code; + this.stack = stack; + } public ErrorResponse(String message, int code, Throwable throwable) { this.message = message; @@ -36,14 +56,17 @@ public ErrorResponse(String message, int code, Throwable throwable) { this.stack = getStackFromThrowable(throwable); } + @JsonGetter(FIELD_MESSAGE) public String message() { return message; } - public int code() { + @JsonGetter(FIELD_CODE) + public Integer code() { return code; } + @JsonGetter(FIELD_STACK) public List stack() { return stack; } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java index fd2658a3849e..1f1b9c01aace 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java @@ -30,6 +30,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.ArrayList; import static org.junit.Assert.assertThrows; @@ -71,6 +72,6 @@ public void testHandleErrorResponse() { } private ErrorResponse generateErrorResponse(int code) { - return new ErrorResponse("message", code, null); + return new ErrorResponse("message", code, new ArrayList()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java new file mode 100644 index 000000000000..5260ecfbd8e3 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** Test for {@link RESTObjectMapper}. */ +public class RESTObjectMapperTest { + private ObjectMapper mapper = RESTObjectMapper.create(); + + @Test + public void configResponseParseTest() throws Exception { + String confKey = "a"; + Map conf = new HashMap<>(); + conf.put(confKey, "b"); + ConfigResponse response = new ConfigResponse(conf); + String responseStr = mapper.writeValueAsString(response); + ConfigResponse parseData = mapper.readValue(responseStr, ConfigResponse.class); + assertEquals(conf.get(confKey), parseData.defaults().get(confKey)); + } + + @Test + public void errorResponseParseTest() throws Exception { + String message = "message"; + Integer code = 400; + ErrorResponse response = new ErrorResponse(message, code, new ArrayList()); + String responseStr = mapper.writeValueAsString(response); + ErrorResponse parseData = mapper.readValue(responseStr, ErrorResponse.class); + assertEquals(message, parseData.message()); + assertEquals(code, parseData.code()); + } +} From cb2e7ef16eb3ae52113588badb84ebc74ad8c5a0 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 16:55:10 +0800 Subject: [PATCH 31/60] [core] Add basic implementation to support REST Catalog --- .../paimon/rest/DefaultErrorHandler.java | 3 +- .../org/apache/paimon/rest/HttpClient.java | 3 +- .../org/apache/paimon/rest/RESTCatalog.java | 9 +++- .../paimon/rest/RESTCatalogOptions.java | 5 +++ .../org/apache/paimon/rest/ResourcePaths.java | 41 +++++++++++++++++++ .../paimon/rest/responses/ErrorResponse.java | 6 +++ 6 files changed, 63 insertions(+), 4 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java index 607bcde49ecf..1a8618c1c603 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java @@ -36,7 +36,8 @@ public static ErrorHandler getInstance() { @Override public void accept(ErrorResponse error) { - switch (error.code()) { + int code = error.code(); + switch (code) { case 400: throw new BadRequestException( String.format("Malformed request: %s", error.message())); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 96ec5b462b2d..fcf2f04d0b9a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -90,8 +90,7 @@ private T exec(Request request, Class responseType) ErrorResponse error = new ErrorResponse( responseBodyStr != null ? responseBodyStr : "response body is null", - response.code(), - null); + response.code()); errorHandler.accept(error); } if (responseBodyStr == null) { 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 c4b8dbf71c01..f5ce75873551 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 @@ -42,6 +42,7 @@ public class RESTCatalog implements Catalog { private RESTClient client; private final ObjectMapper objectMapper = RESTObjectMapper.create(); private String token; + private ResourcePaths resourcePaths; public RESTCatalog(Options options) { String endpoint = options.get(RESTCatalogOptions.ENDPOINT); @@ -58,6 +59,8 @@ public RESTCatalog(Options options) { threadPoolSize, DefaultErrorHandler.getInstance()); this.client = new HttpClient(httpClientBuildParameter); + this.resourcePaths = + ResourcePaths.forCatalogProperties(options.get(RESTCatalogOptions.ENDPOINT_PREFIX)); } @Override @@ -68,7 +71,11 @@ public String warehouse() { @Override public Map options() { ConfigResponse response = - client.post("config", new ConfigRequest(), ConfigResponse.class, headers()); + client.post( + resourcePaths.config(), + new ConfigRequest(), + ConfigResponse.class, + headers()); return response.defaults(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 14e97ad5c326..38353afd7d67 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -28,6 +28,11 @@ public class RESTCatalogOptions { .stringType() .noDefaultValue() .withDescription("REST Catalog server's endpoint."); + public static final ConfigOption ENDPOINT_PREFIX = + ConfigOptions.key("rest.catalog.endpoint.prefix") + .stringType() + .defaultValue("default") + .withDescription("REST Catalog server's endpoint orefix."); public static final ConfigOption TOKEN = ConfigOptions.key("rest.catalog.token") .stringType() diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java new file mode 100644 index 000000000000..9facf6a6cdcd --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -0,0 +1,41 @@ +/* + * 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 java.util.StringJoiner; + +/** Resource paths for REST catalog. */ +public class ResourcePaths { + private static final StringJoiner SLASH = new StringJoiner("/"); + public static final String V1_CONFIG = "/v1/{prefix}/config"; + + public static ResourcePaths forCatalogProperties(String prefix) { + return new ResourcePaths(prefix); + } + + private final String prefix; + + public ResourcePaths(String prefix) { + this.prefix = prefix; + } + + public String config() { + return SLASH.add("v1").add(prefix).add("config").toString(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index b7bfbed33039..0e4b23486732 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -43,6 +43,12 @@ public class ErrorResponse { @JsonProperty(FIELD_STACK) private final List stack; + public ErrorResponse(String message, Integer code) { + this.code = code; + this.message = message; + this.stack = new ArrayList(); + } + @ConstructorProperties({FIELD_MESSAGE, FIELD_CODE, FIELD_STACK}) public ErrorResponse(String message, int code, List stack) { this.message = message; From ce226e39185dc99d2760028521749848f50e917b Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 21 Nov 2024 18:09:17 +0800 Subject: [PATCH 32/60] [core] Add basic implementation to support REST Catalog --- .../org/apache/paimon/rest/ResourcePaths.java | 2 +- paimon-open-api/README.md | 30 +++++++++++++++++++ paimon-open-api/pom.xml | 1 + paimon-open-api/rest-catalog-open-api.yaml | 9 +++--- 4 files changed, 37 insertions(+), 5 deletions(-) create mode 100644 paimon-open-api/README.md diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index 9facf6a6cdcd..cdba171ca2ae 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -23,7 +23,7 @@ /** Resource paths for REST catalog. */ public class ResourcePaths { private static final StringJoiner SLASH = new StringJoiner("/"); - public static final String V1_CONFIG = "/v1/{prefix}/config"; + public static final String V1_CONFIG = "/api/v1/{prefix}/config"; public static ResourcePaths forCatalogProperties(String prefix) { return new ResourcePaths(prefix); diff --git a/paimon-open-api/README.md b/paimon-open-api/README.md new file mode 100644 index 000000000000..708772c1e81b --- /dev/null +++ b/paimon-open-api/README.md @@ -0,0 +1,30 @@ + + +# Open API spec + +The `rest-catalog-open-api.yaml` defines the REST catalog interface. + +## Generate Open API Spec +```sh +mvn spring-boot:run & +curl -s "http://localhost:8080/swagger-api-docs" | jq -M > /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.json +yq --prettyPrint -o=yaml /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.json > /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.yaml +rm -rf /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.json +``` \ No newline at end of file diff --git a/paimon-open-api/pom.xml b/paimon-open-api/pom.xml index b0eacaf28d6d..b5cee29fe4e7 100644 --- a/paimon-open-api/pom.xml +++ b/paimon-open-api/pom.xml @@ -69,6 +69,7 @@ under the License. org.springframework.boot spring-boot-maven-plugin + 2.7.6 org.apache.maven.plugins diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index e886b46a30ca..ec3f88a9a840 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -23,7 +23,7 @@ info: license: name: Apache 2.0 url: https://www.apache.org/licenses/LICENSE-2.0.html - version: '1.0' + version: "1.0" servers: - url: http://localhost:8080 description: Server URL in Development environment @@ -47,14 +47,14 @@ paths: $ref: '#/components/schemas/ConfigRequest' required: true responses: - '201': + "500": + description: Internal Server Error + "201": description: Created content: application/json: schema: $ref: '#/components/schemas/ConfigResponse' - '500': - description: Internal Server Error components: schemas: ConfigRequest: @@ -66,3 +66,4 @@ components: type: object additionalProperties: type: string + writeOnly: true From 9a928294ec9d557e3883810b30bdf0b41fa867df Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 22 Nov 2024 10:11:48 +0800 Subject: [PATCH 33/60] [core] Add basic implementation to support REST Catalog --- paimon-open-api/Makefile | 25 +++++++++++++++++++++ paimon-open-api/README.md | 26 +++------------------ paimon-open-api/generate.sh | 45 +++++++++++++++++++++++++++++++++++++ 3 files changed, 73 insertions(+), 23 deletions(-) create mode 100644 paimon-open-api/Makefile create mode 100755 paimon-open-api/generate.sh diff --git a/paimon-open-api/Makefile b/paimon-open-api/Makefile new file mode 100644 index 000000000000..c3264c83dbd0 --- /dev/null +++ b/paimon-open-api/Makefile @@ -0,0 +1,25 @@ +# +# 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. +# + +# See: https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features + + +install: + brew install yq + +generate: + @sh generate.sh diff --git a/paimon-open-api/README.md b/paimon-open-api/README.md index 708772c1e81b..9d14a7cdd364 100644 --- a/paimon-open-api/README.md +++ b/paimon-open-api/README.md @@ -1,30 +1,10 @@ - - # Open API spec The `rest-catalog-open-api.yaml` defines the REST catalog interface. ## Generate Open API Spec ```sh -mvn spring-boot:run & -curl -s "http://localhost:8080/swagger-api-docs" | jq -M > /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.json -yq --prettyPrint -o=yaml /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.json > /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.yaml -rm -rf /Users/jerry/code/paimon/yt-paimon/paimon/paimon-open-api/rest-catalog-open-api.json +make install +cd paimon-open-api +make generate ``` \ No newline at end of file diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh new file mode 100755 index 000000000000..cde8eb69a2c0 --- /dev/null +++ b/paimon-open-api/generate.sh @@ -0,0 +1,45 @@ +#!/bin/bash +# +# 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. + +# Start the application +mvn spring-boot:run & +SPRING_PID=$! +# Wait for the application to be ready +RETRY_COUNT=0 +MAX_RETRIES=10 +SLEEP_DURATION=5 + +until $(curl -s -o /dev/null -w "%{http_code}" http://localhost:8080/swagger-api-docs | grep -q "200"); do + ((RETRY_COUNT++)) + if [ $RETRY_COUNT -gt $MAX_RETRIES ]; then + echo "Failed to start the application after $MAX_RETRIES retries." + exit 1 + fi + echo "Application not ready yet. Retrying in $SLEEP_DURATION seconds..." + sleep $SLEEP_DURATION +done + +echo "Application is ready". + +# Generate the OpenAPI specification file +curl -s "http://localhost:8080/swagger-api-docs" | jq -M > ./rest-catalog-open-api.json +yq --prettyPrint -o=yaml ./rest-catalog-open-api.json > ./rest-catalog-open-api.yaml +rm -rf ./rest-catalog-open-api.json +mvn spotless:apply +# Stop the application +echo "Stopping application..." +kill $SPRING_PID \ No newline at end of file From 084e3382b5c52769ee2fdc4201ddf3e8980bac46 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 22 Nov 2024 16:38:06 +0800 Subject: [PATCH 34/60] [core] Add basic implementation to support REST Catalog --- .../org/apache/paimon/rest/ResourcePaths.java | 2 +- .../apache/paimon/rest/ResourcePathsTest.java | 40 +++++++++++++++++++ .../open/api/RESTCatalogController.java | 5 +-- 3 files changed, 43 insertions(+), 4 deletions(-) create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index cdba171ca2ae..edad689aec11 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -36,6 +36,6 @@ public ResourcePaths(String prefix) { } public String config() { - return SLASH.add("v1").add(prefix).add("config").toString(); + return SLASH.add("api").add("v1").add(prefix).add("config").toString(); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java new file mode 100644 index 000000000000..d87f16a0e9e2 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java @@ -0,0 +1,40 @@ +/* + * 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.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** Test for {@link ResourcePaths}. */ +public class ResourcePathsTest { + private final String prefix = "test"; + private ResourcePaths resourcePaths = ResourcePaths.forCatalogProperties(prefix); + + @Test + public void configPathTest() { + ResourcePaths resourcePaths = ResourcePaths.forCatalogProperties(prefix); + String expected = replacePrefix(ResourcePaths.V1_CONFIG, prefix); + assertEquals(expected, "/" + resourcePaths.config()); + } + + private String replacePrefix(String path, String prefix) { + return path.replace("{prefix}", prefix); + } +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java index aa61735fb142..47845bd04a90 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -18,6 +18,7 @@ package org.apache.paimon.open.api; +import org.apache.paimon.rest.ResourcePaths; import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; @@ -32,7 +33,6 @@ import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.PostMapping; import org.springframework.web.bind.annotation.RequestBody; -import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RestController; import java.util.HashMap; @@ -41,7 +41,6 @@ /** * RESTCatalog management APIs. */ @CrossOrigin(origins = "http://localhost:8081") @RestController -@RequestMapping("/api/v1/{prefix}") public class RESTCatalogController { @Operation( @@ -59,7 +58,7 @@ public class RESTCatalogController { responseCode = "500", content = {@Content(schema = @Schema())}) }) - @PostMapping("/config") + @PostMapping(ResourcePaths.V1_CONFIG) public ResponseEntity getConfig( @PathVariable String prefix, @RequestBody ConfigRequest request) { try { From 6353a8965cc7603653f8df412617a18d4378500e Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 11:10:33 +0800 Subject: [PATCH 35/60] [core] Add basic implementation to support REST Catalog --- NOTICE | 17 +++++++ .../apache/paimon/options/ConfigOptions.java | 6 +++ .../apache/paimon/utils/ThreadPoolUtils.java | 14 ++++-- paimon-core/pom.xml | 5 ++ .../org/apache/paimon/rest/HttpClient.java | 49 ++++++++----------- ...dParameter.java => HttpClientOptions.java} | 41 ++++++++++------ .../org/apache/paimon/rest/RESTCatalog.java | 24 +++++---- .../paimon/rest/RESTCatalogOptions.java | 40 +++++++++------ .../apache/paimon/rest/HttpClientTest.java | 17 +++++-- .../apache/paimon/rest/RESTCatalogTest.java | 3 +- 10 files changed, 138 insertions(+), 78 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/rest/{HttpClientBuildParameter.java => HttpClientOptions.java} (68%) diff --git a/NOTICE b/NOTICE index 30bd8fc78e26..4de1ec1ea399 100644 --- a/NOTICE +++ b/NOTICE @@ -52,4 +52,21 @@ Apache Flink, Flink®, Apache®, the squirrel logo, and the Apache feather logo Chroma Copyright (C) 2017 Alec Thomas +---------------------------------------------------------- + +OkHttp +Copyright 2019 Square, Inc. + +Licensed 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. + ---------------------------------------------------------- \ No newline at end of file diff --git a/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java index caef58946dbd..fc016b2cead3 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java @@ -21,6 +21,7 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.options.description.Description; +import java.net.URI; import java.time.Duration; import java.util.Map; @@ -164,6 +165,11 @@ public TypedConfigOptionBuilder> mapType() { return new TypedConfigOptionBuilder<>(key, PROPERTIES_MAP_CLASS); } + /** Defines that the value of the option should be of {@link URI} type. */ + public TypedConfigOptionBuilder uriType() { + return new TypedConfigOptionBuilder<>(key, URI.class); + } + /** * Creates a ConfigOption with the given default value. * diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java index 02b5d73fcf2c..f768f6b26a6b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Queue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -46,21 +47,28 @@ /** Utils for thread pool. */ public class ThreadPoolUtils { + /** Create a thread pool with max thread number and default queue. */ + public static ThreadPoolExecutor createCachedThreadPool(int threadNum, String namePrefix) { + return createCachedThreadPool(threadNum, namePrefix, new LinkedBlockingQueue<>()); + } + /** - * Create a thread pool with max thread number. Inactive threads will automatically exit. + * Create a thread pool with max thread number and user define queue. Inactive threads will + * automatically exit. * *

The {@link Executors#newCachedThreadPool} cannot limit max thread number. Non-core threads * must be used with {@link SynchronousQueue}, but synchronous queue will be blocked when there * is max thread number. */ - public static ThreadPoolExecutor createCachedThreadPool(int threadNum, String namePrefix) { + public static ThreadPoolExecutor createCachedThreadPool( + int threadNum, String namePrefix, BlockingQueue workQueue) { ThreadPoolExecutor executor = new ThreadPoolExecutor( threadNum, threadNum, 1, TimeUnit.MINUTES, - new LinkedBlockingQueue<>(), + workQueue, newDaemonThreadFactory(namePrefix)); executor.allowCoreThreadTimeOut(true); return executor; diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index d6c05f587186..577dae7cde34 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -252,6 +252,11 @@ under the License. shade + + + com.squareup.okhttp3:okhttp + + okhttp3 diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index fcf2f04d0b9a..214c8f8d2bf6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -21,7 +21,6 @@ import org.apache.paimon.rest.exceptions.RESTException; import org.apache.paimon.rest.responses.ErrorResponse; -import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -34,30 +33,34 @@ import okhttp3.Response; import java.io.IOException; +import java.net.URI; import java.util.Arrays; import java.util.Map; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.LinkedBlockingQueue; import static okhttp3.ConnectionSpec.CLEARTEXT; import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; import static okhttp3.ConnectionSpec.MODERN_TLS; +import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; /** HTTP client for REST catalog. */ public class HttpClient implements RESTClient { private final OkHttpClient okHttpClient; - private final String endpoint; + private final URI endpoint; private final ObjectMapper mapper; private final ErrorHandler errorHandler; - public HttpClient(HttpClientBuildParameter httpClientBuildParameter) { - this.endpoint = httpClientBuildParameter.endpoint(); - this.mapper = httpClientBuildParameter.mapper(); - this.okHttpClient = createHttpClient(httpClientBuildParameter); - this.errorHandler = httpClientBuildParameter.errorHandler(); + private static final String thread_name = "REST-CATALOG-HTTP-CLIENT-THREAD-POOL"; + private static final MediaType MEDIA_TYPE = MediaType.parse("application/json"); + + public HttpClient(HttpClientOptions httpClientOptions) { + this.endpoint = httpClientOptions.endpoint(); + this.mapper = httpClientOptions.mapper(); + this.okHttpClient = createHttpClient(httpClientOptions); + this.errorHandler = httpClientOptions.errorHandler(); } @Override @@ -103,31 +106,19 @@ private T exec(Request request, Class responseType) } private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingException { - return RequestBody.create( - MediaType.parse("application/json"), mapper.writeValueAsString(body)); + return RequestBody.create(mapper.writeValueAsBytes(body), MEDIA_TYPE); } - private static OkHttpClient createHttpClient( - HttpClientBuildParameter httpClientBuildParameter) { + private static OkHttpClient createHttpClient(HttpClientOptions httpClientOptions) { + BlockingQueue workQueue = + new LinkedBlockingQueue<>(httpClientOptions.queueSize()); ExecutorService executorService = - new ThreadPoolExecutor( - httpClientBuildParameter.threadPoolSize(), - httpClientBuildParameter.threadPoolSize(), - 60, - TimeUnit.SECONDS, - new SynchronousQueue<>(), - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("rest catalog http client %d") - .build()); + createCachedThreadPool(httpClientOptions.threadPoolSize(), thread_name, workQueue); OkHttpClient.Builder builder = new OkHttpClient.Builder() - .connectTimeout( - httpClientBuildParameter.connectTimeoutMillis(), - TimeUnit.MILLISECONDS) - .readTimeout( - httpClientBuildParameter.readTimeoutMillis(), TimeUnit.MILLISECONDS) + .connectTimeout(httpClientOptions.connectTimeout()) + .readTimeout(httpClientOptions.readTimeout()) .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java similarity index 68% rename from paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java rename to paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java index 1d61e4ef6370..22a08c67df39 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientBuildParameter.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java @@ -20,40 +20,47 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import java.net.URI; +import java.time.Duration; + /** HTTP client build parameter. */ -public class HttpClientBuildParameter { - private final String endpoint; - private final int connectTimeoutMillis; - private final int readTimeoutMillis; +public class HttpClientOptions { + + private final URI endpoint; + private final Duration connectTimeout; + private final Duration readTimeout; private final ObjectMapper mapper; private final int threadPoolSize; private final ErrorHandler errorHandler; + private final int queueSize; - public HttpClientBuildParameter( - String endpoint, - int connectTimeoutMillis, - int readTimeoutMillis, + public HttpClientOptions( + URI endpoint, + Duration connectTimeout, + Duration readTimeout, ObjectMapper mapper, int threadPoolSize, + int queueSize, ErrorHandler errorHandler) { this.endpoint = endpoint; - this.connectTimeoutMillis = connectTimeoutMillis; - this.readTimeoutMillis = readTimeoutMillis; + this.connectTimeout = connectTimeout; + this.readTimeout = readTimeout; this.mapper = mapper; this.threadPoolSize = threadPoolSize; this.errorHandler = errorHandler; + this.queueSize = queueSize; } - public String endpoint() { + public URI endpoint() { return endpoint; } - public int connectTimeoutMillis() { - return connectTimeoutMillis; + public Duration connectTimeout() { + return connectTimeout; } - public int readTimeoutMillis() { - return readTimeoutMillis; + public Duration readTimeout() { + return readTimeout; } public ObjectMapper mapper() { @@ -64,6 +71,10 @@ public int threadPoolSize() { return threadPoolSize; } + public int queueSize() { + return queueSize; + } + public ErrorHandler errorHandler() { return errorHandler; } 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 f5ce75873551..5e6132cedb03 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 @@ -33,6 +33,8 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import java.net.URI; +import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -40,25 +42,28 @@ /** A catalog implementation for REST. */ public class RESTCatalog implements Catalog { private RESTClient client; - private final ObjectMapper objectMapper = RESTObjectMapper.create(); private String token; private ResourcePaths resourcePaths; + private static final ObjectMapper objectMapper = RESTObjectMapper.create(); + public RESTCatalog(Options options) { - String endpoint = options.get(RESTCatalogOptions.ENDPOINT); + URI endpoint = options.get(RESTCatalogOptions.ENDPOINT); token = options.get(RESTCatalogOptions.TOKEN); - Integer connectTimeoutMillis = options.get(RESTCatalogOptions.CONNECT_TIMEOUT_MILLIS); - Integer readTimeoutMillis = options.get(RESTCatalogOptions.READ_TIMEOUT_MILLIS); + Duration connectTimeout = options.get(RESTCatalogOptions.CONNECT_TIMEOUT); + Duration readTimeout = options.get(RESTCatalogOptions.CONNECT_TIMEOUT); Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); - HttpClientBuildParameter httpClientBuildParameter = - new HttpClientBuildParameter( + int queueSize = options.get(RESTCatalogOptions.THREAD_POOL_QUEUE_SIZE); + HttpClientOptions httpClientOptions = + new HttpClientOptions( endpoint, - connectTimeoutMillis, - readTimeoutMillis, + connectTimeout, + readTimeout, objectMapper, threadPoolSize, + queueSize, DefaultErrorHandler.getInstance()); - this.client = new HttpClient(httpClientBuildParameter); + this.client = new HttpClient(httpClientOptions); this.resourcePaths = ResourcePaths.forCatalogProperties(options.get(RESTCatalogOptions.ENDPOINT_PREFIX)); } @@ -171,7 +176,6 @@ public boolean allowUpperCase() { public void close() throws Exception {} private Map headers() { - // todo: need refresh token Map header = new HashMap<>(); header.put("Authorization", "Bearer " + token); return header; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 38353afd7d67..d174d5f93bf4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -21,36 +21,44 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; +import java.net.URI; +import java.time.Duration; + /** Options for REST Catalog. */ public class RESTCatalogOptions { - public static final ConfigOption ENDPOINT = + public static final ConfigOption ENDPOINT = ConfigOptions.key("rest.catalog.endpoint") - .stringType() + .uriType() .noDefaultValue() .withDescription("REST Catalog server's endpoint."); public static final ConfigOption ENDPOINT_PREFIX = ConfigOptions.key("rest.catalog.endpoint.prefix") .stringType() .defaultValue("default") - .withDescription("REST Catalog server's endpoint orefix."); + .withDescription("REST Catalog server's endpoint prefix."); public static final ConfigOption TOKEN = - ConfigOptions.key("rest.catalog.token") + ConfigOptions.key("rest.catalog.auth.token") .stringType() .noDefaultValue() .withDescription("REST Catalog server's auth token."); - public static final ConfigOption CONNECT_TIMEOUT_MILLIS = - ConfigOptions.key("rest.catalog.connect.timeout.millis") - .intType() - .defaultValue(3_000) - .withDescription("REST Catalog server connect timeout in mills."); - public static final ConfigOption READ_TIMEOUT_MILLIS = - ConfigOptions.key("rest.catalog.read.timeout.millis") - .intType() - .defaultValue(3_000) - .withDescription("REST Catalog server read timeout in mills."); + public static final ConfigOption CONNECT_TIMEOUT = + ConfigOptions.key("rest.catalog.connect-timeout") + .durationType() + .defaultValue(Duration.ofMillis(3_000)) + .withDescription("REST Catalog http client connect timeout."); + public static final ConfigOption READ_TIMEOUT = + ConfigOptions.key("rest.catalog.read-timeout") + .durationType() + .defaultValue(Duration.ofMillis(3_000)) + .withDescription("REST Catalog http client read timeout."); public static final ConfigOption THREAD_POOL_SIZE = - ConfigOptions.key("rest.catalog.thread.size") + ConfigOptions.key("rest.client.num-threads") .intType() .defaultValue(1) - .withDescription("REST Catalog server thread size."); + .withDescription("REST Catalog http client thread num."); + public static final ConfigOption THREAD_POOL_QUEUE_SIZE = + ConfigOptions.key("rest.client.num-queue") + .intType() + .defaultValue(500) + .withDescription("REST Catalog http client thread queue size."); } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index c0bb9eda8ff7..e66b82a6223e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -27,6 +27,8 @@ import org.junit.Test; import java.io.IOException; +import java.net.URI; +import java.time.Duration; import java.util.HashMap; import java.util.Map; @@ -50,13 +52,20 @@ public class HttpClientTest { public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); - String baseUrl = mockWebServer.url("").toString(); + URI baseUrl = mockWebServer.url("").uri(); errorHandler = mock(ErrorHandler.class); - HttpClientBuildParameter httpClientBuildParameter = - new HttpClientBuildParameter(baseUrl, 1000, 1000, objectMapper, 1, errorHandler); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + baseUrl, + Duration.ofSeconds(3), + Duration.ofSeconds(3), + objectMapper, + 1, + 10, + errorHandler); mockResponseData = new MockRESTData("test"); mockResponseDataStr = objectMapper.writeValueAsString(mockResponseData); - httpClient = new HttpClient(httpClientBuildParameter); + httpClient = new HttpClient(httpClientOptions); headers = headers("token"); } 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 3ff7b3e324b9..411910b0c7b3 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 @@ -27,6 +27,7 @@ import org.junit.Test; import java.io.IOException; +import java.net.URI; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -41,7 +42,7 @@ public class RESTCatalogTest { public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); - String baseUrl = mockWebServer.url("").toString(); + URI baseUrl = mockWebServer.url("").uri(); Options options = new Options(); options.set(RESTCatalogOptions.ENDPOINT, baseUrl); options.set(RESTCatalogOptions.TOKEN, initToken); From eee9863e9e8547482d248780c6797cea4690f141 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 11:26:51 +0800 Subject: [PATCH 36/60] [core] Add basic implementation to support REST Catalog --- NOTICE | 8 +++----- .../apache/paimon/rest/RESTCatalogOptions.java | 4 ++-- .../paimon/rest/responses/ConfigResponse.java | 18 +++++++++--------- .../org/apache/paimon/rest/MockRESTData.java | 5 ----- .../apache/paimon/rest/RESTCatalogTest.java | 2 +- .../paimon/rest/RESTObjectMapperTest.java | 2 +- paimon-open-api/rest-catalog-open-api.yaml | 2 +- 7 files changed, 17 insertions(+), 24 deletions(-) diff --git a/NOTICE b/NOTICE index 4de1ec1ea399..3fe6df8bccbf 100644 --- a/NOTICE +++ b/NOTICE @@ -57,11 +57,9 @@ Copyright (C) 2017 Alec Thomas OkHttp Copyright 2019 Square, Inc. -Licensed 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 +Licensed 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, diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index d174d5f93bf4..432d028ef82b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -42,12 +42,12 @@ public class RESTCatalogOptions { .noDefaultValue() .withDescription("REST Catalog server's auth token."); public static final ConfigOption CONNECT_TIMEOUT = - ConfigOptions.key("rest.catalog.connect-timeout") + ConfigOptions.key("rest.client.connect-timeout") .durationType() .defaultValue(Duration.ofMillis(3_000)) .withDescription("REST Catalog http client connect timeout."); public static final ConfigOption READ_TIMEOUT = - ConfigOptions.key("rest.catalog.read-timeout") + ConfigOptions.key("rest.client.read-timeout") .durationType() .defaultValue(Duration.ofMillis(3_000)) .withDescription("REST Catalog http client read timeout."); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index b4b32a8e728d..666b7288102c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -30,18 +30,18 @@ /** Response for getting config. */ @JsonIgnoreProperties(ignoreUnknown = true) public class ConfigResponse implements RESTResponse { - private static final String FIELD_DEFAULTS = "defaults"; + private static final String FIELD_OPTIONS = "options"; - @JsonProperty(FIELD_DEFAULTS) - private Map defaults; + @JsonProperty(FIELD_OPTIONS) + private Map options; - @ConstructorProperties({FIELD_DEFAULTS}) - public ConfigResponse(Map defaults) { - this.defaults = defaults; + @ConstructorProperties({FIELD_OPTIONS}) + public ConfigResponse(Map options) { + this.options = options; } - @JsonGetter(FIELD_DEFAULTS) - public Map defaults() { - return defaults; + @JsonGetter(FIELD_OPTIONS) + public Map options() { + return options; } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java index 4fe1d1899f8d..55c5165ada48 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -41,9 +41,4 @@ public MockRESTData(String data) { public String data() { return data; } - - // @Override - // public String toString() { - // return String.format("{\"data\": \"%s\"}", data); - // } } 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 411910b0c7b3..bf84f9e64acd 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 @@ -57,7 +57,7 @@ public void tearDown() throws IOException { @Test public void testGetConfig() { - String mockResponse = "{\"defaults\": {\"a\": \"b\"}}"; + String mockResponse = "{\"options\": {\"a\": \"b\"}}"; MockResponse mockResponseObj = new MockResponse() .setBody(mockResponse) diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java index 5260ecfbd8e3..8a1705f2e226 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -43,7 +43,7 @@ public void configResponseParseTest() throws Exception { ConfigResponse response = new ConfigResponse(conf); String responseStr = mapper.writeValueAsString(response); ConfigResponse parseData = mapper.readValue(responseStr, ConfigResponse.class); - assertEquals(conf.get(confKey), parseData.defaults().get(confKey)); + assertEquals(conf.get(confKey), parseData.options().get(confKey)); } @Test diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index ec3f88a9a840..2a86807d3615 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -62,7 +62,7 @@ components: ConfigResponse: type: object properties: - defaults: + optionsoptions: type: object additionalProperties: type: string From 3186300119c4c9354e400e39810c5030d64dd719 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 11:39:37 +0800 Subject: [PATCH 37/60] [core] Add basic implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5e6132cedb03..333b33c5729d 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 @@ -81,7 +81,7 @@ public Map options() { new ConfigRequest(), ConfigResponse.class, headers()); - return response.defaults(); + return response.options(); } @Override From fdd093c56bf22771ed0ca08097c733128fbcecad Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 13:28:57 +0800 Subject: [PATCH 38/60] [core] Add basic implementation to support REST Catalog --- .../java/org/apache/paimon/options/OptionsUtils.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java b/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java index a625454f3996..1f9b5e701cdd 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java @@ -20,6 +20,7 @@ import org.apache.paimon.utils.TimeUtils; +import java.net.URI; import java.time.Duration; import java.util.Arrays; import java.util.HashMap; @@ -74,11 +75,21 @@ public static T convertValue(Object rawValue, Class clazz) { return (T) convertToMemorySize(rawValue); } else if (clazz == Map.class) { return (T) convertToProperties(rawValue); + } else if (clazz == URI.class) { + return (T) convertToUri(rawValue); } throw new IllegalArgumentException("Unsupported type: " + clazz); } + static URI convertToUri(Object o) { + if (o.getClass() == URI.class) { + return (URI) o; + } + + return URI.create(o.toString()); + } + @SuppressWarnings("unchecked") static Map convertToProperties(Object o) { if (o instanceof Map) { From 93652b1870c1998f351faa3d9e84bf6855c9b607 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 13:37:48 +0800 Subject: [PATCH 39/60] [core] Add basic implementation to support REST Catalog --- paimon-open-api/rest-catalog-open-api.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 2a86807d3615..fe987fc35c02 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -62,7 +62,7 @@ components: ConfigResponse: type: object properties: - optionsoptions: + options: type: object additionalProperties: type: string From b884933ba001abc4e4ebefd7ab69c17adf8fb80d Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 14:01:40 +0800 Subject: [PATCH 40/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 577dae7cde34..08a6a177b544 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -252,6 +252,14 @@ under the License. shade + + + * + + NOTICE + + + com.squareup.okhttp3:okhttp From cb1de2657d02da22a842a5e61f9024f980bacd15 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 15:06:37 +0800 Subject: [PATCH 41/60] [core] Add basic implementation to support REST Catalog --- NOTICE | 15 ------ .../apache/paimon/options/ConfigOptions.java | 6 --- .../apache/paimon/options/OptionsUtils.java | 2 - .../org/apache/paimon/rest/HttpClient.java | 15 +++--- .../apache/paimon/rest/HttpClientOptions.java | 24 +++++----- .../org/apache/paimon/rest/RESTCatalog.java | 48 ++++++++++++++----- .../rest/RESTCatalogInternalOptions.java | 30 ++++++++++++ .../paimon/rest/RESTCatalogOptions.java | 18 +++---- .../org/apache/paimon/rest/ResourcePaths.java | 6 +-- paimon-core/src/main/resources/NOTICE | 8 ++++ .../apache/paimon/rest/HttpClientTest.java | 8 ++-- .../apache/paimon/rest/RESTCatalogTest.java | 18 ++++--- 12 files changed, 119 insertions(+), 79 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java create mode 100644 paimon-core/src/main/resources/NOTICE diff --git a/NOTICE b/NOTICE index 3fe6df8bccbf..30bd8fc78e26 100644 --- a/NOTICE +++ b/NOTICE @@ -52,19 +52,4 @@ Apache Flink, Flink®, Apache®, the squirrel logo, and the Apache feather logo Chroma Copyright (C) 2017 Alec Thomas ----------------------------------------------------------- - -OkHttp -Copyright 2019 Square, Inc. - -Licensed 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. - ---------------------------------------------------------- \ No newline at end of file diff --git a/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java index fc016b2cead3..caef58946dbd 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/ConfigOptions.java @@ -21,7 +21,6 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.options.description.Description; -import java.net.URI; import java.time.Duration; import java.util.Map; @@ -165,11 +164,6 @@ public TypedConfigOptionBuilder> mapType() { return new TypedConfigOptionBuilder<>(key, PROPERTIES_MAP_CLASS); } - /** Defines that the value of the option should be of {@link URI} type. */ - public TypedConfigOptionBuilder uriType() { - return new TypedConfigOptionBuilder<>(key, URI.class); - } - /** * Creates a ConfigOption with the given default value. * diff --git a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java b/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java index 1f9b5e701cdd..1ea7bf37796e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java @@ -75,8 +75,6 @@ public static T convertValue(Object rawValue, Class clazz) { return (T) convertToMemorySize(rawValue); } else if (clazz == Map.class) { return (T) convertToProperties(rawValue); - } else if (clazz == URI.class) { - return (T) convertToUri(rawValue); } throw new IllegalArgumentException("Unsupported type: " + clazz); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 214c8f8d2bf6..3ce1875f6348 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -33,7 +33,6 @@ import okhttp3.Response; import java.io.IOException; -import java.net.URI; import java.util.Arrays; import java.util.Map; import java.util.concurrent.BlockingQueue; @@ -49,7 +48,7 @@ public class HttpClient implements RESTClient { private final OkHttpClient okHttpClient; - private final URI endpoint; + private final String uri; private final ObjectMapper mapper; private final ErrorHandler errorHandler; @@ -57,7 +56,7 @@ public class HttpClient implements RESTClient { private static final MediaType MEDIA_TYPE = MediaType.parse("application/json"); public HttpClient(HttpClientOptions httpClientOptions) { - this.endpoint = httpClientOptions.endpoint(); + this.uri = httpClientOptions.uri(); this.mapper = httpClientOptions.mapper(); this.okHttpClient = createHttpClient(httpClientOptions); this.errorHandler = httpClientOptions.errorHandler(); @@ -70,7 +69,7 @@ public T post( RequestBody requestBody = buildRequestBody(body); Request request = new Request.Builder() - .url(endpoint + path) + .url(uri + path) .post(requestBody) .headers(Headers.of(headers)) .build(); @@ -117,11 +116,15 @@ private static OkHttpClient createHttpClient(HttpClientOptions httpClientOptions OkHttpClient.Builder builder = new OkHttpClient.Builder() - .connectTimeout(httpClientOptions.connectTimeout()) - .readTimeout(httpClientOptions.readTimeout()) .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); + if (httpClientOptions.connectTimeout().isPresent()) { + builder.connectTimeout(httpClientOptions.connectTimeout().get()); + } + if (httpClientOptions.readTimeout().isPresent()) { + builder.readTimeout(httpClientOptions.readTimeout().get()); + } return builder.build(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java index 22a08c67df39..1c620b8e68c8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java @@ -20,29 +20,29 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import java.net.URI; import java.time.Duration; +import java.util.Optional; /** HTTP client build parameter. */ public class HttpClientOptions { - private final URI endpoint; - private final Duration connectTimeout; - private final Duration readTimeout; + private final String uri; + private final Optional connectTimeout; + private final Optional readTimeout; private final ObjectMapper mapper; private final int threadPoolSize; private final ErrorHandler errorHandler; private final int queueSize; public HttpClientOptions( - URI endpoint, - Duration connectTimeout, - Duration readTimeout, + String uri, + Optional connectTimeout, + Optional readTimeout, ObjectMapper mapper, int threadPoolSize, int queueSize, ErrorHandler errorHandler) { - this.endpoint = endpoint; + this.uri = uri; this.connectTimeout = connectTimeout; this.readTimeout = readTimeout; this.mapper = mapper; @@ -51,15 +51,15 @@ public HttpClientOptions( this.queueSize = queueSize; } - public URI endpoint() { - return endpoint; + public String uri() { + return uri; } - public Duration connectTimeout() { + public Optional connectTimeout() { return connectTimeout; } - public Duration readTimeout() { + public Optional readTimeout() { return readTimeout; } 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 333b33c5729d..6117dc2f18d9 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 @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; @@ -31,32 +32,36 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; +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.databind.ObjectMapper; -import java.net.URI; import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Optional; /** A catalog implementation for REST. */ public class RESTCatalog implements Catalog { private RESTClient client; private String token; private ResourcePaths resourcePaths; + private Map options; private static final ObjectMapper objectMapper = RESTObjectMapper.create(); public RESTCatalog(Options options) { - URI endpoint = options.get(RESTCatalogOptions.ENDPOINT); + String uri = options.get(RESTCatalogOptions.URI); token = options.get(RESTCatalogOptions.TOKEN); - Duration connectTimeout = options.get(RESTCatalogOptions.CONNECT_TIMEOUT); - Duration readTimeout = options.get(RESTCatalogOptions.CONNECT_TIMEOUT); + Optional connectTimeout = options.getOptional(RESTCatalogOptions.CONNECT_TIMEOUT); + Optional readTimeout = options.getOptional(RESTCatalogOptions.CONNECT_TIMEOUT); Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); int queueSize = options.get(RESTCatalogOptions.THREAD_POOL_QUEUE_SIZE); HttpClientOptions httpClientOptions = new HttpClientOptions( - endpoint, + uri, connectTimeout, readTimeout, objectMapper, @@ -64,8 +69,10 @@ public RESTCatalog(Options options) { queueSize, DefaultErrorHandler.getInstance()); this.client = new HttpClient(httpClientOptions); + this.options = mergeOptions(optionsInner(), options.toMap()); this.resourcePaths = - ResourcePaths.forCatalogProperties(options.get(RESTCatalogOptions.ENDPOINT_PREFIX)); + ResourcePaths.forCatalogProperties( + this.options.get(RESTCatalogInternalOptions.PREFIX)); } @Override @@ -75,13 +82,7 @@ public String warehouse() { @Override public Map options() { - ConfigResponse response = - client.post( - resourcePaths.config(), - new ConfigRequest(), - ConfigResponse.class, - headers()); - return response.options(); + return this.options; } @Override @@ -175,6 +176,27 @@ public boolean allowUpperCase() { @Override public void close() throws Exception {} + @VisibleForTesting + Map optionsInner() { + ConfigResponse response = + client.post( + ResourcePaths.config(), + new ConfigRequest(), + ConfigResponse.class, + headers()); + return response.options(); + } + + public Map mergeOptions( + Map propertiesFromServer, Map clientProperties) { + Map merged = + propertiesFromServer != null + ? Maps.newHashMap(propertiesFromServer) + : Maps.newHashMap(); + merged.putAll(clientProperties); + return ImmutableMap.copyOf(Maps.filterValues(merged, Objects::nonNull)); + } + private Map headers() { Map header = new HashMap<>(); header.put("Authorization", "Bearer " + token); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java new file mode 100644 index 000000000000..6011aa562106 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java @@ -0,0 +1,30 @@ +/* + * 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.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +public class RESTCatalogInternalOptions { + public static final ConfigOption PREFIX = + ConfigOptions.key("prefix") + .stringType() + .defaultValue("default") + .withDescription("REST Catalog server's endpoint prefix."); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 432d028ef82b..79a35981632f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -21,21 +21,15 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; -import java.net.URI; import java.time.Duration; /** Options for REST Catalog. */ public class RESTCatalogOptions { - public static final ConfigOption ENDPOINT = - ConfigOptions.key("rest.catalog.endpoint") - .uriType() - .noDefaultValue() - .withDescription("REST Catalog server's endpoint."); - public static final ConfigOption ENDPOINT_PREFIX = - ConfigOptions.key("rest.catalog.endpoint.prefix") + public static final ConfigOption URI = + ConfigOptions.key("uri") .stringType() - .defaultValue("default") - .withDescription("REST Catalog server's endpoint prefix."); + .noDefaultValue() + .withDescription("REST Catalog server's uri."); public static final ConfigOption TOKEN = ConfigOptions.key("rest.catalog.auth.token") .stringType() @@ -44,12 +38,12 @@ public class RESTCatalogOptions { public static final ConfigOption CONNECT_TIMEOUT = ConfigOptions.key("rest.client.connect-timeout") .durationType() - .defaultValue(Duration.ofMillis(3_000)) + .noDefaultValue() .withDescription("REST Catalog http client connect timeout."); public static final ConfigOption READ_TIMEOUT = ConfigOptions.key("rest.client.read-timeout") .durationType() - .defaultValue(Duration.ofMillis(3_000)) + .noDefaultValue() .withDescription("REST Catalog http client read timeout."); public static final ConfigOption THREAD_POOL_SIZE = ConfigOptions.key("rest.client.num-threads") diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index edad689aec11..fb54bea3b7c9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -23,7 +23,7 @@ /** Resource paths for REST catalog. */ public class ResourcePaths { private static final StringJoiner SLASH = new StringJoiner("/"); - public static final String V1_CONFIG = "/api/v1/{prefix}/config"; + public static final String V1_CONFIG = "/api/v1/config"; public static ResourcePaths forCatalogProperties(String prefix) { return new ResourcePaths(prefix); @@ -35,7 +35,7 @@ public ResourcePaths(String prefix) { this.prefix = prefix; } - public String config() { - return SLASH.add("api").add("v1").add(prefix).add("config").toString(); + public static String config() { + return SLASH.add("api").add("v1").add("config").toString(); } } diff --git a/paimon-core/src/main/resources/NOTICE b/paimon-core/src/main/resources/NOTICE new file mode 100644 index 000000000000..dd2479b1d6e7 --- /dev/null +++ b/paimon-core/src/main/resources/NOTICE @@ -0,0 +1,8 @@ +paimon-core +Copyright 2023-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) +- com.squareup.okhttp3:okhttp:4.12.0 \ No newline at end of file diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index e66b82a6223e..ec5d276da417 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -27,10 +27,10 @@ import org.junit.Test; import java.io.IOException; -import java.net.URI; import java.time.Duration; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -52,13 +52,13 @@ public class HttpClientTest { public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); - URI baseUrl = mockWebServer.url("").uri(); + String baseUrl = mockWebServer.url("").toString(); errorHandler = mock(ErrorHandler.class); HttpClientOptions httpClientOptions = new HttpClientOptions( baseUrl, - Duration.ofSeconds(3), - Duration.ofSeconds(3), + Optional.of(Duration.ofSeconds(3)), + Optional.of(Duration.ofSeconds(3)), objectMapper, 1, 10, 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 bf84f9e64acd..a9e661bf4bb8 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 @@ -27,7 +27,6 @@ import org.junit.Test; import java.io.IOException; -import java.net.URI; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -42,11 +41,12 @@ public class RESTCatalogTest { public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); - URI baseUrl = mockWebServer.url("").uri(); + String baseUrl = mockWebServer.url("").toString(); Options options = new Options(); - options.set(RESTCatalogOptions.ENDPOINT, baseUrl); + options.set(RESTCatalogOptions.URI, baseUrl); options.set(RESTCatalogOptions.TOKEN, initToken); options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); + mockOptions(RESTCatalogInternalOptions.PREFIX.key(), "prefix"); restCatalog = new RESTCatalog(options); } @@ -57,13 +57,19 @@ public void tearDown() throws IOException { @Test public void testGetConfig() { - String mockResponse = "{\"options\": {\"a\": \"b\"}}"; + String key = "a"; + String value = "b"; + mockOptions(key, value); + Map response = restCatalog.optionsInner(); + assertEquals(value, response.get(key)); + } + + private void mockOptions(String key, String value) { + String mockResponse = String.format("{\"options\": {\"%s\": \"%s\"}}", key, value); MockResponse mockResponseObj = new MockResponse() .setBody(mockResponse) .addHeader("Content-Type", "application/json"); mockWebServer.enqueue(mockResponseObj); - Map response = restCatalog.options(); - assertEquals("b", response.get("a")); } } From 72b7abae7e32b78878503ecee3dcd4a6988883a4 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 15:11:21 +0800 Subject: [PATCH 42/60] [core] Add basic implementation to support REST Catalog --- .../java/org/apache/paimon/options/OptionsUtils.java | 9 --------- .../apache/paimon/rest/RESTCatalogInternalOptions.java | 1 + paimon-open-api/rest-catalog-open-api.yaml | 6 ------ .../apache/paimon/open/api/RESTCatalogController.java | 4 +--- 4 files changed, 2 insertions(+), 18 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java b/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java index 1ea7bf37796e..a625454f3996 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/OptionsUtils.java @@ -20,7 +20,6 @@ import org.apache.paimon.utils.TimeUtils; -import java.net.URI; import java.time.Duration; import java.util.Arrays; import java.util.HashMap; @@ -80,14 +79,6 @@ public static T convertValue(Object rawValue, Class clazz) { throw new IllegalArgumentException("Unsupported type: " + clazz); } - static URI convertToUri(Object o) { - if (o.getClass() == URI.class) { - return (URI) o; - } - - return URI.create(o.toString()); - } - @SuppressWarnings("unchecked") static Map convertToProperties(Object o) { if (o instanceof Map) { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java index 6011aa562106..c245b6110d15 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java @@ -21,6 +21,7 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; +/** Internal options for REST Catalog. */ public class RESTCatalogInternalOptions { public static final ConfigOption PREFIX = ConfigOptions.key("prefix") diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index fe987fc35c02..6ce154886bde 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -34,12 +34,6 @@ paths: - config summary: Get Config operationId: getConfig - parameters: - - name: prefix - in: path - required: true - schema: - type: string requestBody: content: application/json: diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java index 47845bd04a90..1a90c51d44de 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -30,7 +30,6 @@ import org.springframework.http.HttpStatus; import org.springframework.http.ResponseEntity; import org.springframework.web.bind.annotation.CrossOrigin; -import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.PostMapping; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RestController; @@ -59,8 +58,7 @@ public class RESTCatalogController { content = {@Content(schema = @Schema())}) }) @PostMapping(ResourcePaths.V1_CONFIG) - public ResponseEntity getConfig( - @PathVariable String prefix, @RequestBody ConfigRequest request) { + public ResponseEntity getConfig(@RequestBody ConfigRequest request) { try { Map defaults = new HashMap<>(); ConfigResponse response = new ConfigResponse(defaults); From 599a1cac6275cc9528f72f7c18676bf985ead522 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 15:16:00 +0800 Subject: [PATCH 43/60] [core] Add basic implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTCatalog.java | 2 +- .../main/java/org/apache/paimon/rest/RESTCatalogOptions.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 6117dc2f18d9..7e97c67722ed 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 @@ -56,7 +56,7 @@ public RESTCatalog(Options options) { String uri = options.get(RESTCatalogOptions.URI); token = options.get(RESTCatalogOptions.TOKEN); Optional connectTimeout = options.getOptional(RESTCatalogOptions.CONNECT_TIMEOUT); - Optional readTimeout = options.getOptional(RESTCatalogOptions.CONNECT_TIMEOUT); + Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); int queueSize = options.get(RESTCatalogOptions.THREAD_POOL_QUEUE_SIZE); HttpClientOptions httpClientOptions = diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 79a35981632f..ab25e3e69792 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -31,12 +31,12 @@ public class RESTCatalogOptions { .noDefaultValue() .withDescription("REST Catalog server's uri."); public static final ConfigOption TOKEN = - ConfigOptions.key("rest.catalog.auth.token") + ConfigOptions.key("token") .stringType() .noDefaultValue() .withDescription("REST Catalog server's auth token."); public static final ConfigOption CONNECT_TIMEOUT = - ConfigOptions.key("rest.client.connect-timeout") + ConfigOptions.key("rest.client.connection-timeout") .durationType() .noDefaultValue() .withDescription("REST Catalog http client connect timeout."); From 2484e48f3405498ad54b2d4ad13770afda3ef9e7 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 15:18:41 +0800 Subject: [PATCH 44/60] [core] Add basic implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7e97c67722ed..db52d70d130d 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 @@ -187,7 +187,7 @@ Map optionsInner() { return response.options(); } - public Map mergeOptions( + private Map mergeOptions( Map propertiesFromServer, Map clientProperties) { Map merged = propertiesFromServer != null From 4e8598c2e780fc6b6a46519edb673acb6f061047 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 15:24:05 +0800 Subject: [PATCH 45/60] [core] Add basic implementation to support REST Catalog --- .../org/apache/paimon/rest/RESTCatalogInternalOptions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java index c245b6110d15..cf61caa20e88 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java @@ -26,6 +26,6 @@ public class RESTCatalogInternalOptions { public static final ConfigOption PREFIX = ConfigOptions.key("prefix") .stringType() - .defaultValue("default") - .withDescription("REST Catalog server's endpoint prefix."); + .noDefaultValue() + .withDescription("REST Catalog uri's prefix."); } From 516782620b65b32689aa24d1b33b5daa54e48017 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 15:53:15 +0800 Subject: [PATCH 46/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 08a6a177b544..0e6350b9c2d6 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -254,8 +254,10 @@ under the License. - * + *:* + okhttp3/internal/publicsuffix/NOTICE + okhttp/LICENSE NOTICE From a65aebd7ac4f44eae7fd329748c991ded7e3283d Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 16:03:48 +0800 Subject: [PATCH 47/60] [core] Add basic implementation to support REST Catalog --- paimon-core/src/main/resources/{ => META-INF}/NOTICE | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename paimon-core/src/main/resources/{ => META-INF}/NOTICE (100%) diff --git a/paimon-core/src/main/resources/NOTICE b/paimon-core/src/main/resources/META-INF/NOTICE similarity index 100% rename from paimon-core/src/main/resources/NOTICE rename to paimon-core/src/main/resources/META-INF/NOTICE From 22ebee53107810300d68490316c1f7f3a7af82a8 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 16:51:26 +0800 Subject: [PATCH 48/60] [core] Add basic implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTCatalog.java | 3 ++- .../main/java/org/apache/paimon/rest/RESTCatalogOptions.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) 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 db52d70d130d..d9d446899abe 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 @@ -55,7 +55,8 @@ public class RESTCatalog implements Catalog { public RESTCatalog(Options options) { String uri = options.get(RESTCatalogOptions.URI); token = options.get(RESTCatalogOptions.TOKEN); - Optional connectTimeout = options.getOptional(RESTCatalogOptions.CONNECT_TIMEOUT); + Optional connectTimeout = + options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); int queueSize = options.get(RESTCatalogOptions.THREAD_POOL_QUEUE_SIZE); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index ab25e3e69792..962cf115f26e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -35,7 +35,7 @@ public class RESTCatalogOptions { .stringType() .noDefaultValue() .withDescription("REST Catalog server's auth token."); - public static final ConfigOption CONNECT_TIMEOUT = + public static final ConfigOption CONNECTION_TIMEOUT = ConfigOptions.key("rest.client.connection-timeout") .durationType() .noDefaultValue() From 6c7c96cbbc07d6becfa0f1cda69b8adfc389b2f9 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 18:09:26 +0800 Subject: [PATCH 49/60] [core] Add basic implementation to support REST Catalog --- .../org/apache/paimon/rest/RESTCatalog.java | 35 +++++------ .../paimon/rest/RESTCatalogFactory.java | 8 ++- .../java/org/apache/paimon/rest/RESTUtil.java | 58 +++++++++++++++++++ .../paimon/rest/responses/ConfigResponse.java | 47 ++++++++++++--- .../apache/paimon/rest/RESTCatalogTest.java | 6 +- .../paimon/rest/RESTObjectMapperTest.java | 4 +- .../open/api/RESTCatalogController.java | 3 +- 7 files changed, 123 insertions(+), 38 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java 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 d9d446899abe..7cc2819daf0a 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 @@ -32,15 +32,12 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; -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.databind.ObjectMapper; import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; /** A catalog implementation for REST. */ @@ -49,6 +46,7 @@ public class RESTCatalog implements Catalog { private String token; private ResourcePaths resourcePaths; private Map options; + private Map baseHeader; private static final ObjectMapper objectMapper = RESTObjectMapper.create(); @@ -70,7 +68,10 @@ public RESTCatalog(Options options) { queueSize, DefaultErrorHandler.getInstance()); this.client = new HttpClient(httpClientOptions); - this.options = mergeOptions(optionsInner(), options.toMap()); + Map initHeaders = + RESTUtil.merge(configHeaders(options.toMap()), authHeaders(token)); + this.options = optionsInner(initHeaders, options.toMap()); + this.baseHeader = configHeaders(this.options()); this.resourcePaths = ResourcePaths.forCatalogProperties( this.options.get(RESTCatalogInternalOptions.PREFIX)); @@ -178,29 +179,21 @@ public boolean allowUpperCase() { public void close() throws Exception {} @VisibleForTesting - Map optionsInner() { + Map optionsInner( + Map headers, Map clientProperties) { ConfigResponse response = client.post( - ResourcePaths.config(), - new ConfigRequest(), - ConfigResponse.class, - headers()); - return response.options(); + ResourcePaths.config(), new ConfigRequest(), ConfigResponse.class, headers); + return response.merge(clientProperties); } - private Map mergeOptions( - Map propertiesFromServer, Map clientProperties) { - Map merged = - propertiesFromServer != null - ? Maps.newHashMap(propertiesFromServer) - : Maps.newHashMap(); - merged.putAll(clientProperties); - return ImmutableMap.copyOf(Maps.filterValues(merged, Objects::nonNull)); - } - - private Map headers() { + private Map authHeaders(String token) { Map header = new HashMap<>(); header.put("Authorization", "Bearer " + token); return header; } + + private static Map configHeaders(Map properties) { + return RESTUtil.extractPrefixMap(properties, "header."); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index ca0f3c0ddf9c..b956895e4c12 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -21,8 +21,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; +import org.apache.paimon.options.CatalogOptions; /** Factory to create {@link RESTCatalog}. */ public class RESTCatalogFactory implements CatalogFactory { @@ -34,7 +33,10 @@ public String identifier() { } @Override - public Catalog create(FileIO fileIO, Path warehouse, CatalogContext context) { + public Catalog create(CatalogContext context) { + if (context.options().getOptional(CatalogOptions.WAREHOUSE).isPresent()) { + throw new IllegalArgumentException("warehouse is not supported in config."); + } return new RESTCatalog(context.options()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java new file mode 100644 index 000000000000..f251a4a9fe9e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java @@ -0,0 +1,58 @@ +/* + * 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.utils.Preconditions; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + +import java.util.Map; + +public class RESTUtil { + public static Map extractPrefixMap( + Map properties, String prefix) { + Preconditions.checkNotNull(properties, "Invalid properties map: null"); + Map result = Maps.newHashMap(); + properties.forEach( + (key, value) -> { + if (key != null && key.startsWith(prefix)) { + result.put(key.substring(prefix.length()), value); + } + }); + + return result; + } + + public static Map merge( + Map target, Map updates) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + + target.forEach( + (key, value) -> { + if (!updates.containsKey(key)) { + builder.put(key, value); + } + }); + + updates.forEach(builder::put); + + return builder.build(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index 666b7288102c..e6bc93470364 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -19,29 +19,58 @@ package org.apache.paimon.rest.responses; import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.utils.Preconditions; import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import java.beans.ConstructorProperties; import java.util.Map; +import java.util.Objects; /** Response for getting config. */ @JsonIgnoreProperties(ignoreUnknown = true) public class ConfigResponse implements RESTResponse { - private static final String FIELD_OPTIONS = "options"; + private static final String FIELD_DEFAULTS = "defaults"; + private static final String FIELD_OVERRIDES = "overrides"; - @JsonProperty(FIELD_OPTIONS) - private Map options; + @JsonProperty(FIELD_DEFAULTS) + private Map defaults; - @ConstructorProperties({FIELD_OPTIONS}) - public ConfigResponse(Map options) { - this.options = options; + @JsonProperty(FIELD_OVERRIDES) + private Map overrides; + + @ConstructorProperties({FIELD_DEFAULTS, FIELD_OVERRIDES}) + public ConfigResponse(Map defaults, Map overrides) { + this.defaults = defaults; + this.overrides = overrides; + } + + public Map merge(Map clientProperties) { + Preconditions.checkNotNull( + clientProperties, + "Cannot merge client properties with server-provided properties. Invalid client configuration: null"); + Map merged = + defaults != null ? Maps.newHashMap(defaults) : Maps.newHashMap(); + merged.putAll(clientProperties); + + if (overrides != null) { + merged.putAll(overrides); + } + + return ImmutableMap.copyOf(Maps.filterValues(merged, Objects::nonNull)); + } + + @JsonGetter(FIELD_DEFAULTS) + public Map defaults() { + return defaults; } - @JsonGetter(FIELD_OPTIONS) - public Map options() { - return options; + @JsonGetter(FIELD_OVERRIDES) + public Map overrides() { + return overrides; } } 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 a9e661bf4bb8..a791065d4e05 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 @@ -27,6 +27,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.HashMap; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -60,12 +61,13 @@ public void testGetConfig() { String key = "a"; String value = "b"; mockOptions(key, value); - Map response = restCatalog.optionsInner(); + Map header = new HashMap<>(); + Map response = restCatalog.optionsInner(header, new HashMap<>()); assertEquals(value, response.get(key)); } private void mockOptions(String key, String value) { - String mockResponse = String.format("{\"options\": {\"%s\": \"%s\"}}", key, value); + String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); MockResponse mockResponseObj = new MockResponse() .setBody(mockResponse) diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java index 8a1705f2e226..83a8805d29a0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -40,10 +40,10 @@ public void configResponseParseTest() throws Exception { String confKey = "a"; Map conf = new HashMap<>(); conf.put(confKey, "b"); - ConfigResponse response = new ConfigResponse(conf); + ConfigResponse response = new ConfigResponse(conf, conf); String responseStr = mapper.writeValueAsString(response); ConfigResponse parseData = mapper.readValue(responseStr, ConfigResponse.class); - assertEquals(conf.get(confKey), parseData.options().get(confKey)); + assertEquals(conf.get(confKey), parseData.defaults().get(confKey)); } @Test diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java index 1a90c51d44de..ec0128932656 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -61,7 +61,8 @@ public class RESTCatalogController { public ResponseEntity getConfig(@RequestBody ConfigRequest request) { try { Map defaults = new HashMap<>(); - ConfigResponse response = new ConfigResponse(defaults); + Map overrides = new HashMap<>(); + ConfigResponse response = new ConfigResponse(defaults, overrides); return new ResponseEntity<>(response, HttpStatus.CREATED); } catch (Exception e) { return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); From 41dd0aae9ab352d3156b9fbfe7f67cfeb0e5c55e Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 18:17:30 +0800 Subject: [PATCH 50/60] [core] Add basic implementation to support REST Catalog --- .../java/org/apache/paimon/rest/RESTUtil.java | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java index f251a4a9fe9e..dc1f7fd303d7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java @@ -30,27 +30,23 @@ public static Map extractPrefixMap( Map properties, String prefix) { Preconditions.checkNotNull(properties, "Invalid properties map: null"); Map result = Maps.newHashMap(); - properties.forEach( - (key, value) -> { - if (key != null && key.startsWith(prefix)) { - result.put(key.substring(prefix.length()), value); - } - }); - + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey() != null && entry.getKey().startsWith(prefix)) { + result.put( + entry.getKey().substring(prefix.length()), properties.get(entry.getKey())); + } + } return result; } public static Map merge( Map target, Map updates) { ImmutableMap.Builder builder = ImmutableMap.builder(); - - target.forEach( - (key, value) -> { - if (!updates.containsKey(key)) { - builder.put(key, value); - } - }); - + for (Map.Entry entry : target.entrySet()) { + if (!updates.containsKey(entry.getKey())) { + builder.put(entry.getKey(), entry.getValue()); + } + } updates.forEach(builder::put); return builder.build(); From ad576cdd199b6d4f9a4ad5567fa3ee430a1ae9d3 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 27 Nov 2024 18:25:18 +0800 Subject: [PATCH 51/60] [core] Add basic implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/rest/RESTUtil.java | 1 + paimon-open-api/rest-catalog-open-api.yaml | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java index dc1f7fd303d7..3d42e99fa6d5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java @@ -25,6 +25,7 @@ import java.util.Map; +/** Util for REST. */ public class RESTUtil { public static Map extractPrefixMap( Map properties, String prefix) { diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 6ce154886bde..7c22b9bcbe2b 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -56,7 +56,12 @@ components: ConfigResponse: type: object properties: - options: + defaults: + type: object + additionalProperties: + type: string + writeOnly: true + overrides: type: object additionalProperties: type: string From 8ad1624c7c6cfd105e76c93cce14e165c360321f Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 11:11:17 +0800 Subject: [PATCH 52/60] [core] Add basic implementation to support REST Catalog --- .../org/apache/paimon/rest/HttpClient.java | 21 ++++++++-- .../apache/paimon/rest/HttpClientOptions.java | 7 ---- .../org/apache/paimon/rest/RESTCatalog.java | 29 ++++++------- .../paimon/rest/RESTCatalogFactory.java | 4 -- .../paimon/rest/RESTCatalogOptions.java | 5 --- .../org/apache/paimon/rest/RESTClient.java | 2 + .../paimon/rest/requests/ConfigRequest.java | 31 -------------- .../apache/paimon/rest/HttpClientTest.java | 41 +++++++++++++++---- .../apache/paimon/rest/RESTCatalogTest.java | 11 ++++- paimon-open-api/generate.sh | 3 ++ paimon-open-api/rest-catalog-open-api.yaml | 12 +----- .../open/api/RESTCatalogController.java | 8 ++-- 12 files changed, 83 insertions(+), 91 deletions(-) delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index 3ce1875f6348..c4609361b6a9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -37,7 +37,7 @@ import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; import static okhttp3.ConnectionSpec.CLEARTEXT; import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; @@ -62,6 +62,22 @@ public HttpClient(HttpClientOptions httpClientOptions) { this.errorHandler = httpClientOptions.errorHandler(); } + @Override + public T get( + String path, Class responseType, Map headers) { + try { + Request request = + new Request.Builder() + .url(uri + path) + .get() + .headers(Headers.of(headers)) + .build(); + return exec(request, responseType); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + @Override public T post( String path, RESTRequest body, Class responseType, Map headers) { @@ -109,8 +125,7 @@ private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingExce } private static OkHttpClient createHttpClient(HttpClientOptions httpClientOptions) { - BlockingQueue workQueue = - new LinkedBlockingQueue<>(httpClientOptions.queueSize()); + BlockingQueue workQueue = new SynchronousQueue<>(); ExecutorService executorService = createCachedThreadPool(httpClientOptions.threadPoolSize(), thread_name, workQueue); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java index 1c620b8e68c8..6c4bfeeb4401 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java @@ -32,7 +32,6 @@ public class HttpClientOptions { private final ObjectMapper mapper; private final int threadPoolSize; private final ErrorHandler errorHandler; - private final int queueSize; public HttpClientOptions( String uri, @@ -40,7 +39,6 @@ public HttpClientOptions( Optional readTimeout, ObjectMapper mapper, int threadPoolSize, - int queueSize, ErrorHandler errorHandler) { this.uri = uri; this.connectTimeout = connectTimeout; @@ -48,7 +46,6 @@ public HttpClientOptions( this.mapper = mapper; this.threadPoolSize = threadPoolSize; this.errorHandler = errorHandler; - this.queueSize = queueSize; } public String uri() { @@ -71,10 +68,6 @@ public int threadPoolSize() { return threadPoolSize; } - public int queueSize() { - return queueSize; - } - public ErrorHandler errorHandler() { return errorHandler; } 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 7cc2819daf0a..d4fd172b54c1 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 @@ -25,17 +25,17 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; -import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.time.Duration; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -49,15 +49,19 @@ public class RESTCatalog implements Catalog { private Map baseHeader; private static final ObjectMapper objectMapper = RESTObjectMapper.create(); + static final String AUTH_HEADER = "Authorization"; + static final String AUTH_HEADER_VALUE_FORMAT = "Bearer %s"; public RESTCatalog(Options options) { + if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { + throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); + } String uri = options.get(RESTCatalogOptions.URI); token = options.get(RESTCatalogOptions.TOKEN); Optional connectTimeout = options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); - int queueSize = options.get(RESTCatalogOptions.THREAD_POOL_QUEUE_SIZE); HttpClientOptions httpClientOptions = new HttpClientOptions( uri, @@ -65,12 +69,13 @@ public RESTCatalog(Options options) { readTimeout, objectMapper, threadPoolSize, - queueSize, DefaultErrorHandler.getInstance()); this.client = new HttpClient(httpClientOptions); + Map authHeaders = + ImmutableMap.of(AUTH_HEADER, String.format(AUTH_HEADER_VALUE_FORMAT, token)); Map initHeaders = - RESTUtil.merge(configHeaders(options.toMap()), authHeaders(token)); - this.options = optionsInner(initHeaders, options.toMap()); + RESTUtil.merge(configHeaders(options.toMap()), authHeaders); + this.options = fetchOptionsFromServer(initHeaders, options.toMap()); this.baseHeader = configHeaders(this.options()); this.resourcePaths = ResourcePaths.forCatalogProperties( @@ -179,20 +184,12 @@ public boolean allowUpperCase() { public void close() throws Exception {} @VisibleForTesting - Map optionsInner( + Map fetchOptionsFromServer( Map headers, Map clientProperties) { - ConfigResponse response = - client.post( - ResourcePaths.config(), new ConfigRequest(), ConfigResponse.class, headers); + ConfigResponse response = client.get(ResourcePaths.config(), ConfigResponse.class, headers); return response.merge(clientProperties); } - private Map authHeaders(String token) { - Map header = new HashMap<>(); - header.put("Authorization", "Bearer " + token); - return header; - } - private static Map configHeaders(Map properties) { return RESTUtil.extractPrefixMap(properties, "header."); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index b956895e4c12..a5c773cb4bd5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -21,7 +21,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; -import org.apache.paimon.options.CatalogOptions; /** Factory to create {@link RESTCatalog}. */ public class RESTCatalogFactory implements CatalogFactory { @@ -34,9 +33,6 @@ public String identifier() { @Override public Catalog create(CatalogContext context) { - if (context.options().getOptional(CatalogOptions.WAREHOUSE).isPresent()) { - throw new IllegalArgumentException("warehouse is not supported in config."); - } return new RESTCatalog(context.options()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java index 962cf115f26e..6155b893751b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -50,9 +50,4 @@ public class RESTCatalogOptions { .intType() .defaultValue(1) .withDescription("REST Catalog http client thread num."); - public static final ConfigOption THREAD_POOL_QUEUE_SIZE = - ConfigOptions.key("rest.client.num-queue") - .intType() - .defaultValue(500) - .withDescription("REST Catalog http client thread queue size."); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java index bc1e7ccdb39d..feeed06a417a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -24,6 +24,8 @@ /** Interface for a basic HTTP Client for interfacing with the REST catalog. */ public interface RESTClient extends Closeable { + T get(String path, Class responseType, Map headers); + T post( String path, RESTRequest body, Class responseType, Map headers); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java deleted file mode 100644 index c671a944301c..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest.requests; - -import org.apache.paimon.rest.RESTRequest; - -import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; - -/** Request to get config. */ -@JsonIgnoreProperties(ignoreUnknown = true) -public class ConfigRequest implements RESTRequest { - @JsonCreator - public ConfigRequest() {} -} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index ec5d276da417..258771f420c9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import okhttp3.mockwebserver.MockResponse; @@ -32,6 +33,8 @@ import java.util.Map; import java.util.Optional; +import static org.apache.paimon.rest.RESTCatalog.AUTH_HEADER; +import static org.apache.paimon.rest.RESTCatalog.AUTH_HEADER_VALUE_FORMAT; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -47,6 +50,8 @@ public class HttpClientTest { private MockRESTData mockResponseData; private String mockResponseDataStr; private Map headers; + private static final String MOCK_PATH = "/v1/api/mock"; + private static final String TOKEN = "token"; @Before public void setUp() throws IOException { @@ -61,12 +66,12 @@ public void setUp() throws IOException { Optional.of(Duration.ofSeconds(3)), objectMapper, 1, - 10, errorHandler); - mockResponseData = new MockRESTData("test"); + mockResponseData = new MockRESTData(MOCK_PATH); mockResponseDataStr = objectMapper.writeValueAsString(mockResponseData); httpClient = new HttpClient(httpClientOptions); - headers = headers("token"); + headers = ImmutableMap.of(AUTH_HEADER, String.format(AUTH_HEADER_VALUE_FORMAT, TOKEN)); + ; } @After @@ -74,21 +79,34 @@ public void tearDown() throws IOException { mockWebServer.shutdown(); } + @Test + public void testGetSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = httpClient.get(MOCK_PATH, MockRESTData.class, headers); + verify(errorHandler, times(0)).accept(any()); + assertEquals(mockResponseData.data(), response.data()); + } + + @Test + public void testGetFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.get(MOCK_PATH, MockRESTData.class, headers); + verify(errorHandler, times(1)).accept(any()); + } + @Test public void testPostSuccess() { - MockResponse mockResponseObj = generateMockResponse(mockResponseDataStr, 200); - mockWebServer.enqueue(mockResponseObj); + mockHttpCallWithCode(mockResponseDataStr, 200); MockRESTData response = - httpClient.post("test", mockResponseData, MockRESTData.class, headers); + httpClient.post(MOCK_PATH, mockResponseData, MockRESTData.class, headers); verify(errorHandler, times(0)).accept(any()); assertEquals(mockResponseData.data(), response.data()); } @Test public void testPostFail() { - MockResponse mockResponseObj = generateMockResponse(mockResponseDataStr, 400); - mockWebServer.enqueue(mockResponseObj); - httpClient.post("test", mockResponseData, MockRESTData.class, headers); + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.post(MOCK_PATH, mockResponseData, MockRESTData.class, headers); verify(errorHandler, times(1)).accept(any()); } @@ -98,6 +116,11 @@ private Map headers(String token) { return header; } + private void mockHttpCallWithCode(String body, Integer code) { + MockResponse mockResponseObj = generateMockResponse(body, code); + mockWebServer.enqueue(mockResponseObj); + } + private MockResponse generateMockResponse(String data, Integer code) { return new MockResponse() .setResponseCode(code) 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 a791065d4e05..3ed8730862ee 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 @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import okhttp3.mockwebserver.MockResponse; @@ -31,6 +32,7 @@ import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; /** Test for REST Catalog. */ public class RESTCatalogTest { @@ -56,13 +58,20 @@ public void tearDown() throws IOException { mockWebServer.shutdown(); } + @Test + public void testInitFailWhenDefineWarehouse() { + Options options = new Options(); + options.set(CatalogOptions.WAREHOUSE, "/a/b/c"); + assertThrows(IllegalArgumentException.class, () -> new RESTCatalog(options)); + } + @Test public void testGetConfig() { String key = "a"; String value = "b"; mockOptions(key, value); Map header = new HashMap<>(); - Map response = restCatalog.optionsInner(header, new HashMap<>()); + Map response = restCatalog.fetchOptionsFromServer(header, new HashMap<>()); assertEquals(value, response.get(key)); } diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh index cde8eb69a2c0..b63aa538abc4 100755 --- a/paimon-open-api/generate.sh +++ b/paimon-open-api/generate.sh @@ -16,6 +16,9 @@ # limitations under the License. # Start the application +cd .. +mvn clean install -DskipTests +cd ./paimon-open-api mvn spring-boot:run & SPRING_PID=$! # Wait for the application to be ready diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 7c22b9bcbe2b..432ee123b8d4 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -28,18 +28,12 @@ servers: - url: http://localhost:8080 description: Server URL in Development environment paths: - /api/v1/{prefix}/config: - post: + /api/v1/config: + get: tags: - config summary: Get Config operationId: getConfig - requestBody: - content: - application/json: - schema: - $ref: '#/components/schemas/ConfigRequest' - required: true responses: "500": description: Internal Server Error @@ -51,8 +45,6 @@ paths: $ref: '#/components/schemas/ConfigResponse' components: schemas: - ConfigRequest: - type: object ConfigResponse: type: object properties: diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java index ec0128932656..b47554057105 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -19,7 +19,6 @@ package org.apache.paimon.open.api; import org.apache.paimon.rest.ResourcePaths; -import org.apache.paimon.rest.requests.ConfigRequest; import org.apache.paimon.rest.responses.ConfigResponse; import io.swagger.v3.oas.annotations.Operation; @@ -30,8 +29,7 @@ import org.springframework.http.HttpStatus; import org.springframework.http.ResponseEntity; import org.springframework.web.bind.annotation.CrossOrigin; -import org.springframework.web.bind.annotation.PostMapping; -import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.GetMapping; import org.springframework.web.bind.annotation.RestController; import java.util.HashMap; @@ -57,8 +55,8 @@ public class RESTCatalogController { responseCode = "500", content = {@Content(schema = @Schema())}) }) - @PostMapping(ResourcePaths.V1_CONFIG) - public ResponseEntity getConfig(@RequestBody ConfigRequest request) { + @GetMapping(ResourcePaths.V1_CONFIG) + public ResponseEntity getConfig() { try { Map defaults = new HashMap<>(); Map overrides = new HashMap<>(); From 3b865ce087c7c27dc26ffb2fb29e5295c43d9148 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 11:15:41 +0800 Subject: [PATCH 53/60] [core] Add basic implementation to support REST Catalog --- .../src/test/java/org/apache/paimon/rest/HttpClientTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java index 258771f420c9..1140e399824c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -71,7 +71,6 @@ public void setUp() throws IOException { mockResponseDataStr = objectMapper.writeValueAsString(mockResponseData); httpClient = new HttpClient(httpClientOptions); headers = ImmutableMap.of(AUTH_HEADER, String.format(AUTH_HEADER_VALUE_FORMAT, TOKEN)); - ; } @After From ef004a2ffcdca32ff751a7842ee7c82a7b596784 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 11:40:52 +0800 Subject: [PATCH 54/60] [core] Add basic implementation to support REST Catalog --- .../main/java/org/apache/paimon/rest/HttpClient.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java index c4609361b6a9..e092711e5f97 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -52,7 +52,7 @@ public class HttpClient implements RESTClient { private final ObjectMapper mapper; private final ErrorHandler errorHandler; - private static final String thread_name = "REST-CATALOG-HTTP-CLIENT-THREAD-POOL"; + private static final String THREAD_NAME = "REST-CATALOG-HTTP-CLIENT-THREAD-POOL"; private static final MediaType MEDIA_TYPE = MediaType.parse("application/json"); public HttpClient(HttpClientOptions httpClientOptions) { @@ -127,19 +127,15 @@ private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingExce private static OkHttpClient createHttpClient(HttpClientOptions httpClientOptions) { BlockingQueue workQueue = new SynchronousQueue<>(); ExecutorService executorService = - createCachedThreadPool(httpClientOptions.threadPoolSize(), thread_name, workQueue); + createCachedThreadPool(httpClientOptions.threadPoolSize(), THREAD_NAME, workQueue); OkHttpClient.Builder builder = new OkHttpClient.Builder() .dispatcher(new Dispatcher(executorService)) .retryOnConnectionFailure(true) .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); - if (httpClientOptions.connectTimeout().isPresent()) { - builder.connectTimeout(httpClientOptions.connectTimeout().get()); - } - if (httpClientOptions.readTimeout().isPresent()) { - builder.readTimeout(httpClientOptions.readTimeout().get()); - } + httpClientOptions.connectTimeout().ifPresent(builder::connectTimeout); + httpClientOptions.readTimeout().ifPresent(builder::readTimeout); return builder.build(); } From 2b8811c495d464f28cfb06d5ce867addd9216f81 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 13:51:01 +0800 Subject: [PATCH 55/60] [core] Add basic implementation to support REST Catalog --- .../org/apache/paimon/utils/ThreadPoolUtils.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java index f768f6b26a6b..67be05617115 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java @@ -47,7 +47,13 @@ /** Utils for thread pool. */ public class ThreadPoolUtils { - /** Create a thread pool with max thread number and default queue. */ + /** + * Create a thread pool with max thread number. Inactive threads will automatically exit. + * + *

The {@link Executors#newCachedThreadPool} cannot limit max thread number. Non-core threads + * must be used with {@link SynchronousQueue}, but synchronous queue will be blocked when there + * is max thread number. + */ public static ThreadPoolExecutor createCachedThreadPool(int threadNum, String namePrefix) { return createCachedThreadPool(threadNum, namePrefix, new LinkedBlockingQueue<>()); } @@ -55,10 +61,6 @@ public static ThreadPoolExecutor createCachedThreadPool(int threadNum, String na /** * Create a thread pool with max thread number and user define queue. Inactive threads will * automatically exit. - * - *

The {@link Executors#newCachedThreadPool} cannot limit max thread number. Non-core threads - * must be used with {@link SynchronousQueue}, but synchronous queue will be blocked when there - * is max thread number. */ public static ThreadPoolExecutor createCachedThreadPool( int threadNum, String namePrefix, BlockingQueue workQueue) { From ea28b06ec12813c79abef0d0cb78bd7f5e7a6b02 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 13:52:19 +0800 Subject: [PATCH 56/60] [core] Add basic implementation to support REST Catalog --- .../src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java index 67be05617115..dd9fc668f4ee 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java @@ -59,7 +59,7 @@ public static ThreadPoolExecutor createCachedThreadPool(int threadNum, String na } /** - * Create a thread pool with max thread number and user define queue. Inactive threads will + * Create a thread pool with max thread number and define queue. Inactive threads will * automatically exit. */ public static ThreadPoolExecutor createCachedThreadPool( From 72fc7dda36200edd3294ebb12e621c8de8b89614 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 13:58:22 +0800 Subject: [PATCH 57/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 0e6350b9c2d6..64ea566abd1b 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -33,6 +33,7 @@ under the License. 6.20.3-ververica-2.0 + 4.12.0 @@ -68,7 +69,7 @@ under the License. com.squareup.okhttp3 okhttp - 4.12.0 + ${okhttp.version} @@ -215,7 +216,7 @@ under the License. com.squareup.okhttp3 mockwebserver - 4.12.0 + ${okhttp.version} test From da96c828d144833e9036265c1765147ca87849aa Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 14:10:21 +0800 Subject: [PATCH 58/60] [core] Add basic implementation to support REST Catalog --- .../apache/paimon/rest/HttpClientOptions.java | 2 +- .../org/apache/paimon/rest/RESTCatalog.java | 3 +- .../org/apache/paimon/rest/ResourcePaths.java | 7 ---- .../apache/paimon/rest/ResourcePathsTest.java | 40 ------------------- 4 files changed, 3 insertions(+), 49 deletions(-) delete mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java index 6c4bfeeb4401..694779cfdb86 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java @@ -23,7 +23,7 @@ import java.time.Duration; import java.util.Optional; -/** HTTP client build parameter. */ +/** Options for Http Client. */ public class HttpClientOptions { private final String uri; 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 d4fd172b54c1..c96400831370 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 @@ -186,7 +186,8 @@ public void close() throws Exception {} @VisibleForTesting Map fetchOptionsFromServer( Map headers, Map clientProperties) { - ConfigResponse response = client.get(ResourcePaths.config(), ConfigResponse.class, headers); + ConfigResponse response = + client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers); return response.merge(clientProperties); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index fb54bea3b7c9..1fad87588a33 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -18,11 +18,8 @@ package org.apache.paimon.rest; -import java.util.StringJoiner; - /** Resource paths for REST catalog. */ public class ResourcePaths { - private static final StringJoiner SLASH = new StringJoiner("/"); public static final String V1_CONFIG = "/api/v1/config"; public static ResourcePaths forCatalogProperties(String prefix) { @@ -34,8 +31,4 @@ public static ResourcePaths forCatalogProperties(String prefix) { public ResourcePaths(String prefix) { this.prefix = prefix; } - - public static String config() { - return SLASH.add("api").add("v1").add("config").toString(); - } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java deleted file mode 100644 index d87f16a0e9e2..000000000000 --- a/paimon-core/src/test/java/org/apache/paimon/rest/ResourcePathsTest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -/** Test for {@link ResourcePaths}. */ -public class ResourcePathsTest { - private final String prefix = "test"; - private ResourcePaths resourcePaths = ResourcePaths.forCatalogProperties(prefix); - - @Test - public void configPathTest() { - ResourcePaths resourcePaths = ResourcePaths.forCatalogProperties(prefix); - String expected = replacePrefix(ResourcePaths.V1_CONFIG, prefix); - assertEquals(expected, "/" + resourcePaths.config()); - } - - private String replacePrefix(String path, String prefix) { - return path.replace("{prefix}", prefix); - } -} From 32c3a4c3d933cdb60bbdc3ed984fb6676c4b61fb Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 14:17:03 +0800 Subject: [PATCH 59/60] [core] Add basic implementation to support REST Catalog --- paimon-core/pom.xml | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 64ea566abd1b..cee7ad693548 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -253,16 +253,6 @@ under the License. shade - - - *:* - - okhttp3/internal/publicsuffix/NOTICE - okhttp/LICENSE - NOTICE - - - com.squareup.okhttp3:okhttp From b3db19646346e6c1f94007a8ee74aaa8031a62cd Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 28 Nov 2024 14:38:02 +0800 Subject: [PATCH 60/60] update paimon core pom exclude NOTICE --- paimon-core/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index cee7ad693548..e137d57a6db1 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -253,6 +253,14 @@ under the License. shade + + + * + + okhttp3/internal/publicsuffix/NOTICE + + + com.squareup.okhttp3:okhttp