Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi merged PR #4553: URL: https://github.com/apache/paimon/pull/4553 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553: URL: https://github.com/apache/paimon/pull/4553#discussion_r1861554737 ## paimon-core/pom.xml: ## @@ -219,6 +242,42 @@ under the License. + +org.apache.maven.plugins +maven-shade-plugin + + +shade-paimon +package + +shade + + + + +*:* + Review Comment: Why need to exclude NOTICE and LICENSE? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
jerry-024 commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1861455611
##
paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.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.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 {
Review Comment:
As get config use HTTP get is enough, i deleted this class.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
jerry-024 commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1861447333
##
paimon-core/src/main/java/org/apache/paimon/rest/requests/ConfigRequest.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.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 {
Review Comment:
The parameter should disable the warehouse because catalogs and warehouses
are one-to-one for defined catalogs, so the client should not be able to define
a warehouse.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
jerry-024 commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1861417026
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,199 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.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.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;
+
+/** A catalog implementation for REST. */
+public class RESTCatalog implements Catalog {
+private RESTClient client;
+private String token;
+private ResourcePaths resourcePaths;
+private Map options;
+private Map baseHeader;
+
+private static final ObjectMapper objectMapper = RESTObjectMapper.create();
+
+public RESTCatalog(Options options) {
+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,
+connectTimeout,
+readTimeout,
+objectMapper,
+threadPoolSize,
+queueSize,
+DefaultErrorHandler.getInstance());
+this.client = new HttpClient(httpClientOptions);
+Map initHeaders =
+RESTUtil.merge(configHeaders(options.toMap()),
authHeaders(token));
+this.options = optionsInner(initHeaders, options.toMap());
+this.baseHeader = configHeaders(this.options());
Review Comment:
When set up, we need to merge the auth header. Later, we will support using
header() to refresh the token.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860489680
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java:
##
@@ -0,0 +1,42 @@
+/*
+ * 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.options.CatalogOptions;
+
+/** Factory to create {@link RESTCatalog}. */
+public class RESTCatalogFactory implements CatalogFactory {
+public static final String IDENTIFIER = "rest";
+
+@Override
+public String identifier() {
+return IDENTIFIER;
+}
+
+@Override
+public Catalog create(CatalogContext context) {
+if
(context.options().getOptional(CatalogOptions.WAREHOUSE).isPresent()) {
Review Comment:
Maybe check this in RestCatalog class?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860487867
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,199 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.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.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;
+
+/** A catalog implementation for REST. */
+public class RESTCatalog implements Catalog {
+private RESTClient client;
+private String token;
+private ResourcePaths resourcePaths;
+private Map options;
+private Map baseHeader;
+
+private static final ObjectMapper objectMapper = RESTObjectMapper.create();
+
+public RESTCatalog(Options options) {
+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,
+connectTimeout,
+readTimeout,
+objectMapper,
+threadPoolSize,
+queueSize,
+DefaultErrorHandler.getInstance());
+this.client = new HttpClient(httpClientOptions);
+Map initHeaders =
+RESTUtil.merge(configHeaders(options.toMap()),
authHeaders(token));
+this.options = optionsInner(initHeaders, options.toMap());
+this.baseHeader = configHeaders(this.options());
Review Comment:
We need to merge authHeader too?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860488315
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,199 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.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.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;
+
+/** A catalog implementation for REST. */
+public class RESTCatalog implements Catalog {
+private RESTClient client;
+private String token;
+private ResourcePaths resourcePaths;
+private Map options;
+private Map baseHeader;
+
+private static final ObjectMapper objectMapper = RESTObjectMapper.create();
+
+public RESTCatalog(Options options) {
+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,
+connectTimeout,
+readTimeout,
+objectMapper,
+threadPoolSize,
+queueSize,
+DefaultErrorHandler.getInstance());
+this.client = new HttpClient(httpClientOptions);
+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));
+}
+
+@Override
+public String warehouse() {
+throw new UnsupportedOperationException();
+}
+
+@Override
+public Map options() {
+return this.options;
+}
+
+@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
DatabaseNotExistExc
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860485518
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,199 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.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.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;
+
+/** A catalog implementation for REST. */
+public class RESTCatalog implements Catalog {
+private RESTClient client;
+private String token;
+private ResourcePaths resourcePaths;
+private Map options;
+private Map baseHeader;
+
+private static final ObjectMapper objectMapper = RESTObjectMapper.create();
+
+public RESTCatalog(Options options) {
+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,
+connectTimeout,
+readTimeout,
+objectMapper,
+threadPoolSize,
+queueSize,
+DefaultErrorHandler.getInstance());
+this.client = new HttpClient(httpClientOptions);
+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));
+}
+
+@Override
+public String warehouse() {
+throw new UnsupportedOperationException();
+}
+
+@Override
+public Map options() {
+return this.options;
+}
+
+@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
DatabaseNotExistExc
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860482622
##
paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java:
##
@@ -0,0 +1,131 @@
+/*
+ * 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.RESTException;
+import org.apache.paimon.rest.responses.ErrorResponse;
+
+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.Headers;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+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.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+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 uri;
+private final ObjectMapper mapper;
+private final ErrorHandler 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.uri = httpClientOptions.uri();
+this.mapper = httpClientOptions.mapper();
+this.okHttpClient = createHttpClient(httpClientOptions);
+this.errorHandler = httpClientOptions.errorHandler();
+}
+
+@Override
+public T post(
+String path, RESTRequest body, Class responseType, Map headers) {
+try {
+RequestBody requestBody = buildRequestBody(body);
+Request request =
+new Request.Builder()
+.url(uri + path)
+.post(requestBody)
+.headers(Headers.of(headers))
+.build();
+return exec(request, responseType);
+} catch (Exception e) {
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void close() throws IOException {
+okHttpClient.dispatcher().cancelAll();
+okHttpClient.connectionPool().evictAll();
+}
+
+private T exec(Request request, Class
responseType) {
+try (Response response = okHttpClient.newCall(request).execute()) {
+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());
+errorHandler.accept(error);
+}
+if (responseBodyStr == null) {
+throw new RESTException("response body is null.");
+}
+return mapper.readValue(responseBodyStr, responseType);
+} catch (Exception e) {
+throw new RESTException(e, "rest exception");
+}
+}
+
+private RequestBody buildRequestBody(RESTRequest body) throws
JsonProcessingException {
+return RequestBody.create(mapper.writeValueAsBytes(body), MEDIA_TYPE);
+}
+
+private static OkHttpClient createHttpClient(HttpClientOptions
httpClientOptions) {
+BlockingQueue workQueue =
+new LinkedBlockingQueue<>(httpClientOptions.queueSize());
Review Comment:
Just use `SynchronousQueue`, remove `queueSize` option.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use th
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860280619
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,206 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.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.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. */
+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) {
+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,
+connectTimeout,
+readTimeout,
+objectMapper,
+threadPoolSize,
+queueSize,
+DefaultErrorHandler.getInstance());
+this.client = new HttpClient(httpClientOptions);
+this.options = mergeOptions(optionsInner(), options.toMap());
+this.resourcePaths =
+ResourcePaths.forCatalogProperties(
+this.options.get(RESTCatalogInternalOptions.PREFIX));
+}
+
+@Override
+public String warehouse() {
+throw new UnsupportedOperationException();
+}
+
+@Override
+public Map options() {
+return this.options;
+}
+
+@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 {
+
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860003187
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.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()
+.noDefaultValue()
+.withDescription("REST Catalog server's auth token.");
+public static final ConfigOption CONNECT_TIMEOUT_MILLIS =
+ConfigOptions.key("rest.catalog.connect.timeout.millis")
Review Comment:
Keep `rest.client.connection-timeout`, the default value is none.
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.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()
+.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")
Review Comment:
the default value is none.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860007140
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,179 @@
+/*
+ * 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.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.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** 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;
+
+public RESTCatalog(Options options) {
+String 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);
+Integer threadPoolSize =
options.get(RESTCatalogOptions.THREAD_POOL_SIZE);
+HttpClientBuildParameter httpClientBuildParameter =
+new HttpClientBuildParameter(
+endpoint,
+connectTimeoutMillis,
+readTimeoutMillis,
+objectMapper,
+threadPoolSize,
+DefaultErrorHandler.getInstance());
+this.client = new HttpClient(httpClientBuildParameter);
+this.resourcePaths =
+
ResourcePaths.forCatalogProperties(options.get(RESTCatalogOptions.ENDPOINT_PREFIX));
+}
+
+@Override
+public String warehouse() {
+throw new UnsupportedOperationException();
+}
+
+@Override
+public Map options() {
+ConfigResponse response =
+client.post(
+resourcePaths.config(),
+new ConfigRequest(),
+ConfigResponse.class,
+headers());
+return response.defaults();
Review Comment:
Just return class field options. You should construct it in the constructor.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1860001281
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.stringType()
+.noDefaultValue()
+.withDescription("REST Catalog server's endpoint.");
+public static final ConfigOption ENDPOINT_PREFIX =
+ConfigOptions.key("rest.catalog.endpoint.prefix")
Review Comment:
If it is an internal API, it should be in `RestCatalogInternalOptions`.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r186463
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.stringType()
+.noDefaultValue()
+.withDescription("REST Catalog server's endpoint.");
+public static final ConfigOption ENDPOINT_PREFIX =
+ConfigOptions.key("rest.catalog.endpoint.prefix")
Review Comment:
I feel like this prefix is internal API, it is generated by server.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1859997969
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.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")
Review Comment:
key is token
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
Review Comment:
key is uri, just use CatalogOptions.URI
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1859998347
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.stringType()
+.noDefaultValue()
+.withDescription("REST Catalog server's endpoint.");
+public static final ConfigOption ENDPOINT_PREFIX =
+ConfigOptions.key("rest.catalog.endpoint.prefix")
Review Comment:
key is prefix
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
jerry-024 commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1859785722
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.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()
+.noDefaultValue()
+.withDescription("REST Catalog server's auth token.");
+public static final ConfigOption CONNECT_TIMEOUT_MILLIS =
+ConfigOptions.key("rest.catalog.connect.timeout.millis")
Review Comment:
As this conf name is `connect timeout` in HTTP client, how about
`rest.client.connec-timeout`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
jerry-024 commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1859715268
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.stringType()
+.noDefaultValue()
+.withDescription("REST Catalog server's endpoint.");
+public static final ConfigOption ENDPOINT_PREFIX =
+ConfigOptions.key("rest.catalog.endpoint.prefix")
Review Comment:
For isolation. Prefixes can isolate different namespaces and prevent naming
conflicts between different teams or applications.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Re: [PR] [core] Add basic implementation to support REST Catalog [paimon]
JingsongLi commented on code in PR #4553:
URL: https://github.com/apache/paimon/pull/4553#discussion_r1855449135
##
paimon-core/pom.xml:
##
@@ -219,6 +241,27 @@ under the License.
+
+org.apache.maven.plugins
+maven-shade-plugin
+
+
+shade-paimon
+package
+
+shade
+
+
Review Comment:
Please add NOTICE file too.
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/** Options for REST Catalog. */
+public class RESTCatalogOptions {
+public static final ConfigOption ENDPOINT =
+ConfigOptions.key("rest.catalog.endpoint")
+.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()
+.noDefaultValue()
+.withDescription("REST Catalog server's auth token.");
+public static final ConfigOption CONNECT_TIMEOUT_MILLIS =
+ConfigOptions.key("rest.catalog.connect.timeout.millis")
Review Comment:
`rest.client.connection-timeout`, the type can be a durationType
##
paimon-core/pom.xml:
##
@@ -219,6 +241,27 @@ under the License.
+
+org.apache.maven.plugins
+maven-shade-plugin
+
+
+shade-paimon
+package
+
+shade
+
+
Review Comment:
Do you need to configure `includes`?
##
paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java:
##
@@ -0,0 +1,179 @@
+/*
+ * 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.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.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** A catalog implementation for REST. */
+publ
