From eb8b32f35136b8a07ebc09dc9115d81887313b2b Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Dec 2024 10:17:53 +0800 Subject: [PATCH 01/19] close resource in RESTCatalog and add test for getTimeToWaitByExpiresInMills in AuthSession --- .../apache/paimon/rest/auth/AuthSession.java | 54 ++++++++++--------- .../paimon/rest/auth/AuthSessionTest.java | 18 +++++++ 2 files changed, 48 insertions(+), 24 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java index 74efb8508a06..3ca7590e5f96 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java @@ -33,9 +33,10 @@ public class AuthSession { static final int TOKEN_REFRESH_NUM_RETRIES = 5; + static final long MIN_REFRESH_WAIT_MILLIS = 10; + static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes + private static final Logger log = LoggerFactory.getLogger(AuthSession.class); - private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes - private static final long MIN_REFRESH_WAIT_MILLIS = 10; private final CredentialsProvider credentialsProvider; private volatile Map headers; @@ -76,12 +77,38 @@ public Map getHeaders() { return headers; } + public Boolean refresh() { + if (this.credentialsProvider.supportRefresh() + && this.credentialsProvider.keepRefreshed() + && this.credentialsProvider.expiresInMills().isPresent()) { + boolean isSuccessful = this.credentialsProvider.refresh(); + if (isSuccessful) { + Map currentHeaders = this.headers; + this.headers = + RESTUtil.merge(currentHeaders, this.credentialsProvider.authHeader()); + } + return isSuccessful; + } + + return false; + } + @VisibleForTesting static void scheduleTokenRefresh( ScheduledExecutorService executor, AuthSession session, long expiresAtMillis) { scheduleTokenRefresh(executor, session, expiresAtMillis, 0); } + @VisibleForTesting + static long getTimeToWaitByExpiresInMills(long expiresInMillis) { + // how much ahead of time to start the refresh to allow it to complete + long refreshWindowMillis = Math.min(expiresInMillis, MAX_REFRESH_WINDOW_MILLIS); + // how much time to wait before expiration + long waitIntervalMillis = expiresInMillis - refreshWindowMillis; + // how much time to actually wait + return Math.max(waitIntervalMillis, MIN_REFRESH_WAIT_MILLIS); + } + private static void scheduleTokenRefresh( ScheduledExecutorService executor, AuthSession session, @@ -89,12 +116,7 @@ private static void scheduleTokenRefresh( int retryTimes) { if (retryTimes < TOKEN_REFRESH_NUM_RETRIES) { long expiresInMillis = expiresAtMillis - System.currentTimeMillis(); - // how much ahead of time to start the refresh to allow it to complete - long refreshWindowMillis = Math.min(expiresInMillis, MAX_REFRESH_WINDOW_MILLIS); - // how much time to wait before expiration - long waitIntervalMillis = expiresInMillis - refreshWindowMillis; - // how much time to actually wait - long timeToWait = Math.max(waitIntervalMillis, MIN_REFRESH_WAIT_MILLIS); + long timeToWait = getTimeToWaitByExpiresInMills(expiresInMillis); executor.schedule( () -> { @@ -118,20 +140,4 @@ private static void scheduleTokenRefresh( log.warn("Failed to refresh token after {} retries.", TOKEN_REFRESH_NUM_RETRIES); } } - - public Boolean refresh() { - if (this.credentialsProvider.supportRefresh() - && this.credentialsProvider.keepRefreshed() - && this.credentialsProvider.expiresInMills().isPresent()) { - boolean isSuccessful = this.credentialsProvider.refresh(); - if (isSuccessful) { - Map currentHeaders = this.headers; - this.headers = - RESTUtil.merge(currentHeaders, this.credentialsProvider.authHeader()); - } - return isSuccessful; - } - - return false; - } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java index 81b3ea57b703..1f4a48fd5e8c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java @@ -35,6 +35,8 @@ import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; +import static org.apache.paimon.rest.auth.AuthSession.MAX_REFRESH_WINDOW_MILLIS; +import static org.apache.paimon.rest.auth.AuthSession.MIN_REFRESH_WAIT_MILLIS; import static org.apache.paimon.rest.auth.AuthSession.TOKEN_REFRESH_NUM_RETRIES; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.verify; @@ -121,6 +123,22 @@ public void testRetryWhenRefreshFail() throws Exception { verify(credentialsProvider, Mockito.times(TOKEN_REFRESH_NUM_RETRIES + 1)).refresh(); } + @Test + public void testGetTimeToWaitByExpiresInMills() { + long expiresInMillis = -100L; + long timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(MIN_REFRESH_WAIT_MILLIS, timeToWait); + expiresInMillis = (long) (MAX_REFRESH_WINDOW_MILLIS * 0.5); + timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(MIN_REFRESH_WAIT_MILLIS, timeToWait); + expiresInMillis = MAX_REFRESH_WINDOW_MILLIS; + timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(timeToWait, MIN_REFRESH_WAIT_MILLIS); + expiresInMillis = MAX_REFRESH_WINDOW_MILLIS * 2L; + timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(timeToWait, MAX_REFRESH_WINDOW_MILLIS); + } + private Pair generateTokenAndWriteToFile(String fileName) throws IOException { File tokenFile = folder.newFile(fileName); String token = UUID.randomUUID().toString(); From d3f725995370d7b9652eaa3e6c484e7d0b83ace8 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Dec 2024 11:48:41 +0800 Subject: [PATCH 02/19] add list databases support in RESTCatalog --- .../org/apache/paimon/rest/HttpClient.java | 26 ++++++++++ .../org/apache/paimon/rest/RESTCatalog.java | 13 ++++- .../org/apache/paimon/rest/RESTClient.java | 6 +++ .../org/apache/paimon/rest/ResourcePaths.java | 7 +++ .../paimon/rest/responses/DatabaseName.java | 44 +++++++++++++++++ .../rest/responses/ListDatabasesResponse.java | 47 +++++++++++++++++++ paimon-open-api/generate.sh | 2 +- paimon-open-api/rest-catalog-open-api.yaml | 35 ++++++++++++++ .../open/api/RESTCatalogController.java | 30 ++++++++++++ 9 files changed, 208 insertions(+), 2 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.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 e092711e5f97..c1e250e20b7d 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 @@ -26,6 +26,7 @@ import okhttp3.Dispatcher; import okhttp3.Headers; +import okhttp3.HttpUrl; import okhttp3.MediaType; import okhttp3.OkHttpClient; import okhttp3.Request; @@ -78,6 +79,31 @@ public T get( } } + @Override + public T get( + String path, + Map queryParams, + Class responseType, + Map headers) { + try { + HttpUrl.Builder httpUrlBuilder = HttpUrl.parse(uri + path).newBuilder(); + if (queryParams != null) { + for (Map.Entry param : queryParams.entrySet()) { + httpUrlBuilder.addQueryParameter(param.getKey(), param.getValue()); + } + } + Request request = + new Request.Builder() + .url(httpUrlBuilder.build()) + .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) { 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 f3007bf4bf02..b8a6659215ad 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 @@ -30,11 +30,14 @@ import org.apache.paimon.rest.auth.CredentialsProvider; import org.apache.paimon.rest.auth.CredentialsProviderFactory; import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.ListDatabasesResponse; 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.annotations.VisibleForTesting; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.time.Duration; @@ -42,6 +45,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; @@ -113,7 +117,14 @@ public FileIO fileIO() { @Override public List listDatabases() { - throw new UnsupportedOperationException(); + ListDatabasesResponse response = + client.get(resourcePaths.databases(), ListDatabasesResponse.class, headers()); + if (response.databases() != null) { + return response.databases().stream() + .map(DatabaseName::name) + .collect(Collectors.toList()); + } + return ImmutableList.of(); } @Override 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 feeed06a417a..66354aa8321f 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 @@ -26,6 +26,12 @@ public interface RESTClient extends Closeable { T get(String path, Class responseType, Map headers); + T get( + String path, + Map queryParams, + 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/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index aaca6193802d..e2a8a079dd03 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,10 +18,13 @@ package org.apache.paimon.rest; +import java.util.StringJoiner; + /** Resource paths for REST catalog. */ public class ResourcePaths { public static final String V1_CONFIG = "/api/v1/config"; + private static final StringJoiner SLASH = new StringJoiner("/"); public static ResourcePaths forCatalogProperties(String prefix) { return new ResourcePaths(prefix); @@ -32,4 +35,8 @@ public static ResourcePaths forCatalogProperties(String prefix) { public ResourcePaths(String prefix) { this.prefix = prefix; } + + public String databases() { + return SLASH.add("api").add("v1").add(prefix).add("databases").toString(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java new file mode 100644 index 000000000000..c2656bd5dca4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java @@ -0,0 +1,44 @@ +/* + * 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.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.beans.ConstructorProperties; + +/** Class for Database entity. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class DatabaseName { + private static final String FIELD_NAME = "name"; + + @JsonProperty(FIELD_NAME) + private String name; + + @ConstructorProperties({FIELD_NAME}) + public DatabaseName(String name) { + this.name = name; + } + + @JsonGetter(FIELD_NAME) + public String name() { + return this.name; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java new file mode 100644 index 000000000000..a8db97572c11 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java @@ -0,0 +1,47 @@ +/* + * 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 org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.beans.ConstructorProperties; +import java.util.List; + +/** Response for listing databases. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ListDatabasesResponse implements RESTResponse { + private static final String FIELD_DATABASES = "databases"; + + @JsonProperty(FIELD_DATABASES) + private List databases; + + @ConstructorProperties({FIELD_DATABASES}) + public ListDatabasesResponse(List databases) { + this.databases = databases; + } + + @JsonGetter(FIELD_DATABASES) + public List databases() { + return this.databases; + } +} diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh index b63aa538abc4..c6e0a7d9972c 100755 --- a/paimon-open-api/generate.sh +++ b/paimon-open-api/generate.sh @@ -17,7 +17,7 @@ # Start the application cd .. -mvn clean install -DskipTests +#mvn clean install -DskipTests cd ./paimon-open-api mvn spring-boot:run & SPRING_PID=$! diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 432ee123b8d4..476ca55a3aae 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -28,6 +28,27 @@ servers: - url: http://localhost:8080 description: Server URL in Development environment paths: + /api/v1/{prefix}/databases: + get: + tags: + - database + summary: List Databases + operationId: listDatabases + parameters: + - name: prefix + in: query + required: true + schema: + type: string + responses: + "201": + description: Created + content: + application/json: + schema: + $ref: '#/components/schemas/ListDatabasesResponse' + "500": + description: Internal Server Error /api/v1/config: get: tags: @@ -45,6 +66,20 @@ paths: $ref: '#/components/schemas/ConfigResponse' components: schemas: + DatabaseName: + type: object + properties: + name: + type: string + writeOnly: true + ListDatabasesResponse: + type: object + properties: + databases: + type: array + writeOnly: true + items: + $ref: '#/components/schemas/DatabaseName' 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 b47554057105..20bdc01c8294 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 @@ -20,6 +20,10 @@ import org.apache.paimon.rest.ResourcePaths; import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import io.swagger.v3.oas.annotations.Operation; import io.swagger.v3.oas.annotations.media.Content; @@ -66,4 +70,30 @@ public ResponseEntity getConfig() { return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); } } + + @Operation( + summary = "List Databases", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "201", + content = { + @Content( + schema = @Schema(implementation = ListDatabasesResponse.class), + mediaType = "application/json") + }), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping("/api/v1/{prefix}/databases") + public ResponseEntity listDatabases(String prefix) { + try { + ListDatabasesResponse response = + new ListDatabasesResponse(ImmutableList.of(new DatabaseName("account"))); + return new ResponseEntity<>(response, HttpStatus.CREATED); + } catch (Exception e) { + return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); + } + } } From 553f2037ec9e0760e19518268b0e3eb4f1293915 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Dec 2024 13:59:24 +0800 Subject: [PATCH 03/19] support create database api --- .../paimon/rest/DefaultErrorHandler.java | 6 ++ .../org/apache/paimon/rest/RESTCatalog.java | 11 ++- .../exceptions/AlreadyExistsException.java | 25 +++++++ .../exceptions/NoSuchResourceException.java | 25 +++++++ .../rest/requests/CreateDatabaseRequest.java | 68 +++++++++++++++++++ .../responses/CreateDatabaseResponse.java | 57 ++++++++++++++++ .../open/api/RESTCatalogController.java | 34 +++++++++- 7 files changed, 224 insertions(+), 2 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.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 1a8618c1c603..568f0b7f11bb 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 @@ -18,8 +18,10 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.exceptions.AlreadyExistsException; import org.apache.paimon.rest.exceptions.BadRequestException; import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NoSuchResourceException; import org.apache.paimon.rest.exceptions.NotAuthorizedException; import org.apache.paimon.rest.exceptions.RESTException; import org.apache.paimon.rest.exceptions.ServiceFailureException; @@ -45,6 +47,10 @@ public void accept(ErrorResponse error) { throw new NotAuthorizedException("Not authorized: %s", error.message()); case 403: throw new ForbiddenException("Forbidden: %s", error.message()); + case 404: + throw new NoSuchResourceException("%s", error.message()); + case 409: + throw new AlreadyExistsException("%s", error.message()); case 405: case 406: break; 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 b8a6659215ad..f305d5666410 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 @@ -29,7 +29,10 @@ import org.apache.paimon.rest.auth.AuthSession; import org.apache.paimon.rest.auth.CredentialsProvider; import org.apache.paimon.rest.auth.CredentialsProviderFactory; +import org.apache.paimon.rest.exceptions.AlreadyExistsException; +import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.schema.Schema; @@ -130,7 +133,13 @@ public List listDatabases() { @Override public void createDatabase(String name, boolean ignoreIfExists, Map properties) throws DatabaseAlreadyExistException { - throw new UnsupportedOperationException(); + CreateDatabaseRequest request = new CreateDatabaseRequest(name, ignoreIfExists, properties); + try { + client.post( + resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); + } catch (AlreadyExistsException e) { + throw new DatabaseAlreadyExistException(name); + } } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java new file mode 100644 index 000000000000..6dcef999b760 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.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.exceptions; + +public class AlreadyExistsException extends RESTException { + public AlreadyExistsException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java new file mode 100644 index 000000000000..9a3634ea39c1 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.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.exceptions; + +public class NoSuchResourceException extends RESTException { + public NoSuchResourceException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java new file mode 100644 index 000000000000..ea00c975e62e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -0,0 +1,68 @@ +/* + * 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.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.beans.ConstructorProperties; +import java.util.Map; + +/** Request for creating database. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class CreateDatabaseRequest implements RESTRequest { + private static final String FIELD_NAME = "name"; + private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; + private static final String FIELD_PROPERTIES = "properties"; + + @JsonProperty(FIELD_NAME) + private String name; + + @JsonProperty(FIELD_IGNORE_IF_EXISTS) + private boolean ignoreIfExists; + + @JsonProperty(FIELD_PROPERTIES) + private Map properties; + + @ConstructorProperties({FIELD_NAME, FIELD_IGNORE_IF_EXISTS, FIELD_PROPERTIES}) + public CreateDatabaseRequest( + String name, boolean ignoreIfExists, Map properties) { + this.name = name; + this.ignoreIfExists = ignoreIfExists; + this.properties = properties; + } + + @JsonGetter(FIELD_NAME) + public String name() { + return name; + } + + @JsonGetter(FIELD_IGNORE_IF_EXISTS) + public boolean ignoreIfExists() { + return ignoreIfExists; + } + + @JsonGetter(FIELD_PROPERTIES) + public Map properties() { + return properties; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java new file mode 100644 index 000000000000..b03cf7d1a88c --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -0,0 +1,57 @@ +/* + * 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 org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.beans.ConstructorProperties; +import java.util.Map; + +/** Response for creating database. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class CreateDatabaseResponse implements RESTResponse { + private static final String FIELD_NAME = "name"; + private static final String FIELD_PROPERTIES = "properties"; + + @JsonProperty(FIELD_NAME) + private String name; + + @JsonProperty(FIELD_PROPERTIES) + private Map properties; + + @ConstructorProperties({FIELD_NAME, FIELD_PROPERTIES}) + public CreateDatabaseResponse(String name, Map properties) { + this.name = name; + this.properties = properties; + } + + @JsonGetter(FIELD_NAME) + public String name() { + return name; + } + + @JsonGetter(FIELD_PROPERTIES) + public Map properties() { + return 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 20bdc01c8294..3ed453027a39 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,9 @@ package org.apache.paimon.open.api; import org.apache.paimon.rest.ResourcePaths; +import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; import org.apache.paimon.rest.responses.ListDatabasesResponse; @@ -34,6 +36,9 @@ import org.springframework.http.ResponseEntity; import org.springframework.web.bind.annotation.CrossOrigin; import org.springframework.web.bind.annotation.GetMapping; +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; import java.util.HashMap; @@ -87,7 +92,7 @@ public ResponseEntity getConfig() { content = {@Content(schema = @Schema())}) }) @GetMapping("/api/v1/{prefix}/databases") - public ResponseEntity listDatabases(String prefix) { + public ResponseEntity listDatabases(@PathVariable String prefix) { try { ListDatabasesResponse response = new ListDatabasesResponse(ImmutableList.of(new DatabaseName("account"))); @@ -96,4 +101,31 @@ public ResponseEntity listDatabases(String prefix) { return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); } } + + @Operation( + summary = "Create Databases", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "201", + content = { + @Content( + schema = @Schema(implementation = CreateDatabaseResponse.class), + mediaType = "application/json") + }), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/api/v1/{prefix}/databases") + public ResponseEntity createDatabases( + @PathVariable String prefix, @RequestBody CreateDatabaseRequest request) { + try { + Map properties = new HashMap<>(); + CreateDatabaseResponse response = new CreateDatabaseResponse("name", properties); + return new ResponseEntity<>(response, HttpStatus.CREATED); + } catch (Exception e) { + return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); + } + } } From fdbdf8c1fc0be36922048e9ea625ed9fdf665fa5 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Dec 2024 14:38:07 +0800 Subject: [PATCH 04/19] support create database api --- .../org/apache/paimon/rest/HttpClient.java | 26 ------- .../org/apache/paimon/rest/RESTCatalog.java | 11 ++- .../org/apache/paimon/rest/RESTClient.java | 6 -- .../org/apache/paimon/rest/ResourcePaths.java | 4 + .../exceptions/AlreadyExistsException.java | 1 + .../exceptions/NoSuchResourceException.java | 1 + .../rest/responses/GetDatabaseResponse.java | 75 +++++++++++++++++++ paimon-open-api/generate.sh | 2 +- .../open/api/RESTCatalogController.java | 37 +++++++++ 9 files changed, 129 insertions(+), 34 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.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 c1e250e20b7d..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 @@ -26,7 +26,6 @@ import okhttp3.Dispatcher; import okhttp3.Headers; -import okhttp3.HttpUrl; import okhttp3.MediaType; import okhttp3.OkHttpClient; import okhttp3.Request; @@ -79,31 +78,6 @@ public T get( } } - @Override - public T get( - String path, - Map queryParams, - Class responseType, - Map headers) { - try { - HttpUrl.Builder httpUrlBuilder = HttpUrl.parse(uri + path).newBuilder(); - if (queryParams != null) { - for (Map.Entry param : queryParams.entrySet()) { - httpUrlBuilder.addQueryParameter(param.getKey(), param.getValue()); - } - } - Request request = - new Request.Builder() - .url(httpUrlBuilder.build()) - .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) { 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 f305d5666410..922fd1de341f 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 @@ -30,10 +30,12 @@ import org.apache.paimon.rest.auth.CredentialsProvider; import org.apache.paimon.rest.auth.CredentialsProviderFactory; import org.apache.paimon.rest.exceptions.AlreadyExistsException; +import org.apache.paimon.rest.exceptions.NoSuchResourceException; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.GetDatabaseResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -144,7 +146,14 @@ public void createDatabase(String name, boolean ignoreIfExists, Map null)); + } catch (NoSuchResourceException e) { + throw new DatabaseNotExistException(name); + } } @Override 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 66354aa8321f..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 @@ -26,12 +26,6 @@ public interface RESTClient extends Closeable { T get(String path, Class responseType, Map headers); - T get( - String path, - Map queryParams, - 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/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index e2a8a079dd03..a6d0000a225b 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 @@ -39,4 +39,8 @@ public ResourcePaths(String prefix) { public String databases() { return SLASH.add("api").add("v1").add(prefix).add("databases").toString(); } + + public String database(String databaseName) { + return SLASH.add("api").add("v1").add(prefix).add("databases").add(databaseName).toString(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java index 6dcef999b760..0a7263f9ce1c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest.exceptions; +/** Exception thrown on HTTP 409 means a resource already exists. */ public class AlreadyExistsException extends RESTException { public AlreadyExistsException(String message, Object... args) { super(message, args); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java index 9a3634ea39c1..f8c3b05b951c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest.exceptions; +/** Exception thrown on HTTP 404 means a resource not exists. */ public class NoSuchResourceException extends RESTException { public NoSuchResourceException(String message, Object... args) { super(message, args); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java new file mode 100644 index 000000000000..9c0aac6c1b3e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -0,0 +1,75 @@ +/* + * 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.catalog.Database; +import org.apache.paimon.rest.RESTResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +import java.beans.ConstructorProperties; +import java.util.Map; +import java.util.Optional; + +/** Response for getting database. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class GetDatabaseResponse implements RESTResponse, Database { + private static final String FIELD_NAME = "name"; + private static final String FIELD_OPTIONS = "options"; + private static final String FIELD_COMMENT = "comment"; + + @JsonProperty(FIELD_NAME) + private final String name; + + @JsonProperty(FIELD_OPTIONS) + private final Map options; + + @JsonProperty(FIELD_COMMENT) + @Nullable + private final String comment; + + @ConstructorProperties({FIELD_NAME, FIELD_OPTIONS, FIELD_COMMENT}) + public GetDatabaseResponse(String name, Map options, @Nullable String comment) { + this.name = name; + this.options = options; + this.comment = comment; + } + + @Override + @JsonGetter(FIELD_NAME) + public String name() { + return name; + } + + @Override + @JsonGetter(FIELD_OPTIONS) + public Map options() { + return options; + } + + @Override + @JsonGetter(FIELD_COMMENT) + public Optional comment() { + return Optional.ofNullable(comment); + } +} diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh index c6e0a7d9972c..b63aa538abc4 100755 --- a/paimon-open-api/generate.sh +++ b/paimon-open-api/generate.sh @@ -17,7 +17,7 @@ # Start the application cd .. -#mvn clean install -DskipTests +mvn clean install -DskipTests cd ./paimon-open-api mvn spring-boot:run & SPRING_PID=$! 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 3ed453027a39..62d2d3ea20c3 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 @@ -23,6 +23,8 @@ import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.ErrorResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; @@ -113,6 +115,10 @@ public ResponseEntity listDatabases(@PathVariable String schema = @Schema(implementation = CreateDatabaseResponse.class), mediaType = "application/json") }), + @ApiResponse( + responseCode = "409", + description = "Resource has exist", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), @ApiResponse( responseCode = "500", content = {@Content(schema = @Schema())}) @@ -128,4 +134,35 @@ public ResponseEntity createDatabases( return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); } } + + @Operation( + summary = "Get Database", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "201", + content = { + @Content( + schema = @Schema(implementation = GetDatabaseResponse.class), + mediaType = "application/json") + }), + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping("/api/v1/{prefix}/databases/{database}") + public ResponseEntity getDatabases( + @PathVariable String prefix, @PathVariable String database) { + try { + Map options = new HashMap<>(); + GetDatabaseResponse response = new GetDatabaseResponse("name", options, "comment"); + return new ResponseEntity<>(response, HttpStatus.CREATED); + } catch (Exception e) { + return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); + } + } } From fe6fcd27e32c07c53742e257e439e621319445a1 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Dec 2024 15:26:25 +0800 Subject: [PATCH 05/19] generate database swagger api --- paimon-open-api/rest-catalog-open-api.yaml | 124 ++++++++++++++++++++- 1 file changed, 121 insertions(+), 3 deletions(-) diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 476ca55a3aae..d3e3dd8c2e7d 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -36,19 +36,83 @@ paths: operationId: listDatabases parameters: - name: prefix - in: query + in: path required: true schema: type: string responses: + "500": + description: Internal Server Error "201": description: Created content: application/json: schema: $ref: '#/components/schemas/ListDatabasesResponse' + post: + tags: + - database + summary: Create Databases + operationId: createDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/CreateDatabaseRequest' + required: true + responses: + "409": + description: Resource has exist + content: + '*/*': + schema: + $ref: '#/components/schemas/ErrorResponse' "500": description: Internal Server Error + "201": + description: Created + content: + application/json: + schema: + $ref: '#/components/schemas/CreateDatabaseResponse' + /api/v1/{prefix}/databases/{database}: + get: + tags: + - database + summary: Get Database + operationId: getDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + responses: + "404": + description: Resource not found + content: + '*/*': + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error + "201": + description: Created + content: + application/json: + schema: + $ref: '#/components/schemas/GetDatabaseResponse' /api/v1/config: get: tags: @@ -56,16 +120,56 @@ paths: summary: Get Config operationId: getConfig responses: - "500": - description: Internal Server Error "201": description: Created content: application/json: schema: $ref: '#/components/schemas/ConfigResponse' + "500": + description: Internal Server Error components: schemas: + CreateDatabaseRequest: + type: object + properties: + name: + type: string + writeOnly: true + ignoreIfExists: + type: boolean + writeOnly: true + properties: + type: object + additionalProperties: + type: string + writeOnly: true + ErrorResponse: + type: object + properties: + message: + type: string + writeOnly: true + code: + type: integer + format: int32 + writeOnly: true + stack: + type: array + writeOnly: true + items: + type: string + CreateDatabaseResponse: + type: object + properties: + name: + type: string + writeOnly: true + properties: + type: object + additionalProperties: + type: string + writeOnly: true DatabaseName: type: object properties: @@ -80,6 +184,20 @@ components: writeOnly: true items: $ref: '#/components/schemas/DatabaseName' + GetDatabaseResponse: + type: object + properties: + name: + type: string + writeOnly: true + options: + type: object + additionalProperties: + type: string + writeOnly: true + comment: + type: string + writeOnly: true ConfigResponse: type: object properties: From 5196caa290715f19d5c98746a5a16c21f7ce6f1d Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Dec 2024 15:58:39 +0800 Subject: [PATCH 06/19] add example to config api in open api define --- .../paimon/rest/DefaultErrorHandler.java | 7 ++- paimon-open-api/rest-catalog-open-api.yaml | 61 ++++++++++++------- .../open/api/RESTCatalogController.java | 17 ++++++ 3 files changed, 60 insertions(+), 25 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 568f0b7f11bb..73a08b2d12ab 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 @@ -30,6 +30,7 @@ /** Default error handler. */ public class DefaultErrorHandler extends ErrorHandler { + private static final ErrorHandler INSTANCE = new DefaultErrorHandler(); public static ErrorHandler getInstance() { @@ -49,17 +50,19 @@ public void accept(ErrorResponse error) { throw new ForbiddenException("Forbidden: %s", error.message()); case 404: throw new NoSuchResourceException("%s", error.message()); - case 409: - throw new AlreadyExistsException("%s", error.message()); case 405: case 406: break; + case 409: + throw new AlreadyExistsException("%s", error.message()); case 500: throw new ServiceFailureException("Server error: %s", error.message()); case 501: throw new UnsupportedOperationException(error.message()); case 503: throw new ServiceUnavailableException("Service unavailable: %s", error.message()); + default: + break; } throw new RESTException("Unable to process: %s", error.message()); diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index d3e3dd8c2e7d..b993ba9cd127 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -41,14 +41,14 @@ paths: schema: type: string responses: - "500": - description: Internal Server Error "201": description: Created content: application/json: schema: $ref: '#/components/schemas/ListDatabasesResponse' + "500": + description: Internal Server Error post: tags: - database @@ -67,20 +67,20 @@ paths: $ref: '#/components/schemas/CreateDatabaseRequest' required: true responses: - "409": - description: Resource has exist - content: - '*/*': - schema: - $ref: '#/components/schemas/ErrorResponse' - "500": - description: Internal Server Error "201": description: Created content: application/json: schema: $ref: '#/components/schemas/CreateDatabaseResponse' + "500": + description: Internal Server Error + "409": + description: Resource has exist + content: + '*/*': + schema: + $ref: '#/components/schemas/ErrorResponse' /api/v1/{prefix}/databases/{database}: get: tags: @@ -105,14 +105,14 @@ paths: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' - "500": - description: Internal Server Error "201": description: Created content: application/json: schema: $ref: '#/components/schemas/GetDatabaseResponse' + "500": + description: Internal Server Error /api/v1/config: get: tags: @@ -126,6 +126,21 @@ paths: application/json: schema: $ref: '#/components/schemas/ConfigResponse' + examples: + defaults: + description: defaults + value: |- + { + "k1": "v1", + "k2": "v2", + } + overrides: + description: overrides + value: |- + { + "k3": "v1", + "k4": "v2", + } "500": description: Internal Server Error components: @@ -144,6 +159,17 @@ components: additionalProperties: type: string writeOnly: true + CreateDatabaseResponse: + type: object + properties: + name: + type: string + writeOnly: true + properties: + type: object + additionalProperties: + type: string + writeOnly: true ErrorResponse: type: object properties: @@ -159,17 +185,6 @@ components: writeOnly: true items: type: string - CreateDatabaseResponse: - type: object - properties: - name: - type: string - writeOnly: true - properties: - type: object - additionalProperties: - type: string - writeOnly: true DatabaseName: 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 62d2d3ea20c3..e5398b34b6a0 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 @@ -31,6 +31,7 @@ import io.swagger.v3.oas.annotations.Operation; import io.swagger.v3.oas.annotations.media.Content; +import io.swagger.v3.oas.annotations.media.ExampleObject; import io.swagger.v3.oas.annotations.media.Schema; import io.swagger.v3.oas.annotations.responses.ApiResponse; import io.swagger.v3.oas.annotations.responses.ApiResponses; @@ -60,6 +61,22 @@ public class RESTCatalogController { content = { @Content( schema = @Schema(implementation = ConfigResponse.class), + examples = { + @ExampleObject( + name = "defaults", + value = + "{\n" + + " \"k1\": \"v1\",\n" + + " \"k2\": \"v2\",\n" + + "}"), + @ExampleObject( + name = "overrides", + value = + "{\n" + + " \"k3\": \"v1\",\n" + + " \"k4\": \"v2\",\n" + + "}"), + }, mediaType = "application/json") }), @ApiResponse( From 41d2660fcc54ba719c2c2b097667d2125446182a Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 11:32:32 +0800 Subject: [PATCH 07/19] fix open api swagger yaml not schema bug --- .../paimon/rest/DefaultErrorHandler.java | 22 +++--- .../org/apache/paimon/rest/RESTCatalog.java | 6 +- .../paimon/rest/responses/ConfigResponse.java | 4 +- .../responses/CreateDatabaseResponse.java | 4 +- .../paimon/rest/responses/DatabaseName.java | 2 +- .../paimon/rest/responses/ErrorResponse.java | 6 +- .../rest/responses/GetDatabaseResponse.java | 24 ++++-- .../rest/responses/ListDatabasesResponse.java | 2 +- .../apache/paimon/rest/RESTCatalogTest.java | 2 +- .../paimon/rest/RESTObjectMapperTest.java | 6 +- paimon-open-api/generate.sh | 1 + paimon-open-api/rest-catalog-open-api.yaml | 20 +---- .../open/api/RESTCatalogController.java | 75 +++++-------------- .../service/network/NettyBufferPool.java | 2 +- 14 files changed, 69 insertions(+), 107 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 73a08b2d12ab..ce2cbb56ae24 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 @@ -39,32 +39,32 @@ public static ErrorHandler getInstance() { @Override public void accept(ErrorResponse error) { - int code = error.code(); + int code = error.getCode(); + String message = error.getMessage(); switch (code) { case 400: - throw new BadRequestException( - String.format("Malformed request: %s", error.message())); + throw new BadRequestException(String.format("Malformed request: %s", message)); case 401: - throw new NotAuthorizedException("Not authorized: %s", error.message()); + throw new NotAuthorizedException("Not authorized: %s", message); case 403: - throw new ForbiddenException("Forbidden: %s", error.message()); + throw new ForbiddenException("Forbidden: %s", message); case 404: - throw new NoSuchResourceException("%s", error.message()); + throw new NoSuchResourceException("%s", message); case 405: case 406: break; case 409: - throw new AlreadyExistsException("%s", error.message()); + throw new AlreadyExistsException("%s", message); case 500: - throw new ServiceFailureException("Server error: %s", error.message()); + throw new ServiceFailureException("Server error: %s", message); case 501: - throw new UnsupportedOperationException(error.message()); + throw new UnsupportedOperationException(message); case 503: - throw new ServiceUnavailableException("Service unavailable: %s", error.message()); + throw new ServiceUnavailableException("Service unavailable: %s", message); default: break; } - throw new RESTException("Unable to process: %s", error.message()); + throw new RESTException("Unable to process: %s", message); } } 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 922fd1de341f..9e9508de356d 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 @@ -124,9 +124,9 @@ public FileIO fileIO() { public List listDatabases() { ListDatabasesResponse response = client.get(resourcePaths.databases(), ListDatabasesResponse.class, headers()); - if (response.databases() != null) { - return response.databases().stream() - .map(DatabaseName::name) + if (response.getDatabases() != null) { + return response.getDatabases().stream() + .map(DatabaseName::getName) .collect(Collectors.toList()); } return ImmutableList.of(); 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 903cfc84b46d..3660edc391d8 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 @@ -65,12 +65,12 @@ public Map merge(Map clientProperties) { } @JsonGetter(FIELD_DEFAULTS) - public Map defaults() { + public Map getDefaults() { return defaults; } @JsonGetter(FIELD_OVERRIDES) - public Map overrides() { + public Map getOverrides() { return overrides; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java index b03cf7d1a88c..d08a38404e3e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -46,12 +46,12 @@ public CreateDatabaseResponse(String name, Map properties) { } @JsonGetter(FIELD_NAME) - public String name() { + public String getName() { return name; } @JsonGetter(FIELD_PROPERTIES) - public Map properties() { + public Map getProperties() { return properties; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java index c2656bd5dca4..58a6ad77394a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java @@ -38,7 +38,7 @@ public DatabaseName(String name) { } @JsonGetter(FIELD_NAME) - public String name() { + public String getName() { return this.name; } } 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 685fe53071b6..c30cd3833427 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 @@ -63,17 +63,17 @@ public ErrorResponse(String message, int code, Throwable throwable) { } @JsonGetter(FIELD_MESSAGE) - public String message() { + public String getMessage() { return message; } @JsonGetter(FIELD_CODE) - public Integer code() { + public Integer getCode() { return code; } @JsonGetter(FIELD_STACK) - public List stack() { + public List getStack() { return stack; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java index 9c0aac6c1b3e..211dce6187d7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -55,21 +55,33 @@ public GetDatabaseResponse(String name, Map options, @Nullable S this.comment = comment; } - @Override @JsonGetter(FIELD_NAME) - public String name() { + public String getName() { return name; } - @Override @JsonGetter(FIELD_OPTIONS) - public Map options() { + public Map getOptions() { return options; } - @Override @JsonGetter(FIELD_COMMENT) - public Optional comment() { + public Optional getComment() { return Optional.ofNullable(comment); } + + @Override + public String name() { + return this.getName(); + } + + @Override + public Map options() { + return this.getOptions(); + } + + @Override + public Optional comment() { + return this.getComment(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java index a8db97572c11..fe01261e0a62 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java @@ -41,7 +41,7 @@ public ListDatabasesResponse(List databases) { } @JsonGetter(FIELD_DATABASES) - public List databases() { + public List getDatabases() { return this.databases; } } 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 f3f56e97215f..ff51e355f19f 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 @@ -77,7 +77,7 @@ public void testGetConfig() { } private void mockOptions(String key, String value) { - String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); + String mockResponse = String.format("{\"getDefaults\": {\"%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 83a8805d29a0..65859a9dd283 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, 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.getDefaults().get(confKey)); } @Test @@ -53,7 +53,7 @@ public void errorResponseParseTest() throws Exception { 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()); + assertEquals(message, parseData.getMessage()); + assertEquals(code, parseData.getCode()); } } diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh index b63aa538abc4..619b642ab760 100755 --- a/paimon-open-api/generate.sh +++ b/paimon-open-api/generate.sh @@ -17,6 +17,7 @@ # Start the application cd .. +mvn spotless:apply mvn clean install -DskipTests cd ./paimon-open-api mvn spring-boot:run & diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index b993ba9cd127..5b6b1ad0408b 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -127,15 +127,15 @@ paths: schema: $ref: '#/components/schemas/ConfigResponse' examples: - defaults: - description: defaults + getDefaults: + description: getDefaults value: |- { "k1": "v1", "k2": "v2", } - overrides: - description: overrides + getOverrides: + description: getOverrides value: |- { "k3": "v1", @@ -164,25 +164,20 @@ components: properties: name: type: string - writeOnly: true properties: type: object additionalProperties: type: string - writeOnly: true ErrorResponse: type: object properties: message: type: string - writeOnly: true code: type: integer format: int32 - writeOnly: true stack: type: array - writeOnly: true items: type: string DatabaseName: @@ -190,13 +185,11 @@ components: properties: name: type: string - writeOnly: true ListDatabasesResponse: type: object properties: databases: type: array - writeOnly: true items: $ref: '#/components/schemas/DatabaseName' GetDatabaseResponse: @@ -204,15 +197,12 @@ components: properties: name: type: string - writeOnly: true options: type: object additionalProperties: type: string - writeOnly: true comment: type: string - writeOnly: true ConfigResponse: type: object properties: @@ -220,9 +210,7 @@ components: type: object additionalProperties: type: string - writeOnly: true overrides: type: object additionalProperties: type: string - writeOnly: true 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 e5398b34b6a0..b10dc7c0c707 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 @@ -31,12 +31,9 @@ import io.swagger.v3.oas.annotations.Operation; import io.swagger.v3.oas.annotations.media.Content; -import io.swagger.v3.oas.annotations.media.ExampleObject; 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.GetMapping; import org.springframework.web.bind.annotation.PathVariable; @@ -57,42 +54,22 @@ public class RESTCatalogController { tags = {"config"}) @ApiResponses({ @ApiResponse( - responseCode = "201", + responseCode = "200", content = { @Content( schema = @Schema(implementation = ConfigResponse.class), - examples = { - @ExampleObject( - name = "defaults", - value = - "{\n" - + " \"k1\": \"v1\",\n" - + " \"k2\": \"v2\",\n" - + "}"), - @ExampleObject( - name = "overrides", - value = - "{\n" - + " \"k3\": \"v1\",\n" - + " \"k4\": \"v2\",\n" - + "}"), - }, - mediaType = "application/json") + mediaType = "application/json" + ) }), @ApiResponse( responseCode = "500", content = {@Content(schema = @Schema())}) }) @GetMapping(ResourcePaths.V1_CONFIG) - public ResponseEntity getConfig() { - try { - Map defaults = new HashMap<>(); - 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); - } + public ConfigResponse getConfig() { + Map defaults = new HashMap<>(); + Map overrides = new HashMap<>(); + return new ConfigResponse(defaults, overrides); } @Operation( @@ -100,7 +77,7 @@ public ResponseEntity getConfig() { tags = {"database"}) @ApiResponses({ @ApiResponse( - responseCode = "201", + responseCode = "200", content = { @Content( schema = @Schema(implementation = ListDatabasesResponse.class), @@ -111,14 +88,8 @@ public ResponseEntity getConfig() { content = {@Content(schema = @Schema())}) }) @GetMapping("/api/v1/{prefix}/databases") - public ResponseEntity listDatabases(@PathVariable String prefix) { - try { - ListDatabasesResponse response = - new ListDatabasesResponse(ImmutableList.of(new DatabaseName("account"))); - return new ResponseEntity<>(response, HttpStatus.CREATED); - } catch (Exception e) { - return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); - } + public ListDatabasesResponse listDatabases(@PathVariable String prefix) { + return new ListDatabasesResponse(ImmutableList.of(new DatabaseName("account"))); } @Operation( @@ -126,7 +97,7 @@ public ResponseEntity listDatabases(@PathVariable String tags = {"database"}) @ApiResponses({ @ApiResponse( - responseCode = "201", + responseCode = "200", content = { @Content( schema = @Schema(implementation = CreateDatabaseResponse.class), @@ -141,15 +112,10 @@ public ResponseEntity listDatabases(@PathVariable String content = {@Content(schema = @Schema())}) }) @PostMapping("/api/v1/{prefix}/databases") - public ResponseEntity createDatabases( + public CreateDatabaseResponse createDatabases( @PathVariable String prefix, @RequestBody CreateDatabaseRequest request) { - try { - Map properties = new HashMap<>(); - CreateDatabaseResponse response = new CreateDatabaseResponse("name", properties); - return new ResponseEntity<>(response, HttpStatus.CREATED); - } catch (Exception e) { - return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); - } + Map properties = new HashMap<>(); + return new CreateDatabaseResponse("name", properties); } @Operation( @@ -157,7 +123,7 @@ public ResponseEntity createDatabases( tags = {"database"}) @ApiResponses({ @ApiResponse( - responseCode = "201", + responseCode = "200", content = { @Content( schema = @Schema(implementation = GetDatabaseResponse.class), @@ -172,14 +138,9 @@ public ResponseEntity createDatabases( content = {@Content(schema = @Schema())}) }) @GetMapping("/api/v1/{prefix}/databases/{database}") - public ResponseEntity getDatabases( + public GetDatabaseResponse getDatabases( @PathVariable String prefix, @PathVariable String database) { - try { - Map options = new HashMap<>(); - GetDatabaseResponse response = new GetDatabaseResponse("name", options, "comment"); - return new ResponseEntity<>(response, HttpStatus.CREATED); - } catch (Exception e) { - return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); - } + Map options = new HashMap<>(); + return new GetDatabaseResponse("name", options, "comment"); } } diff --git a/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java b/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java index 151308d0edc5..72db65188fe3 100644 --- a/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java +++ b/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java @@ -46,7 +46,7 @@ public NettyBufferPool(int numberOfArenas) { boolean preferDirect = true; // Arenas allocate chunks of pageSize << maxOrder bytes. With these - // defaults, this results in chunks of 16 MB. + // getDefaults, this results in chunks of 16 MB. int pageSize = 8192; int maxOrder = 11; From 59347382a83d1ddb4a672dc3bb72dfe87c2dd41e Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 11:39:36 +0800 Subject: [PATCH 08/19] change properties to options for database in rest --- .../org/apache/paimon/rest/RESTCatalog.java | 2 +- .../rest/requests/CreateDatabaseRequest.java | 23 +++++++++---------- .../responses/CreateDatabaseResponse.java | 18 +++++++-------- .../open/api/RESTCatalogController.java | 3 +-- 4 files changed, 22 insertions(+), 24 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 9e9508de356d..26be4b82be7c 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 @@ -237,7 +237,7 @@ public void close() throws Exception { Map fetchOptionsFromServer( Map headers, Map clientProperties) { ConfigResponse response = - client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers()); + client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class,headers); return response.merge(clientProperties); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java index ea00c975e62e..1a277ef8faf4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -32,7 +32,7 @@ public class CreateDatabaseRequest implements RESTRequest { private static final String FIELD_NAME = "name"; private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; - private static final String FIELD_PROPERTIES = "properties"; + private static final String FIELD_OPTIONS = "options"; @JsonProperty(FIELD_NAME) private String name; @@ -40,29 +40,28 @@ public class CreateDatabaseRequest implements RESTRequest { @JsonProperty(FIELD_IGNORE_IF_EXISTS) private boolean ignoreIfExists; - @JsonProperty(FIELD_PROPERTIES) - private Map properties; + @JsonProperty(FIELD_OPTIONS) + private Map options; - @ConstructorProperties({FIELD_NAME, FIELD_IGNORE_IF_EXISTS, FIELD_PROPERTIES}) - public CreateDatabaseRequest( - String name, boolean ignoreIfExists, Map properties) { + @ConstructorProperties({FIELD_NAME, FIELD_IGNORE_IF_EXISTS, FIELD_OPTIONS}) + public CreateDatabaseRequest(String name, boolean ignoreIfExists, Map options) { this.name = name; this.ignoreIfExists = ignoreIfExists; - this.properties = properties; + this.options = options; } @JsonGetter(FIELD_NAME) - public String name() { + public String getName() { return name; } @JsonGetter(FIELD_IGNORE_IF_EXISTS) - public boolean ignoreIfExists() { + public boolean getIgnoreIfExists() { return ignoreIfExists; } - @JsonGetter(FIELD_PROPERTIES) - public Map properties() { - return properties; + @JsonGetter(FIELD_OPTIONS) + public Map getOptions() { + return options; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java index d08a38404e3e..5ed92c9c2374 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -31,18 +31,18 @@ @JsonIgnoreProperties(ignoreUnknown = true) public class CreateDatabaseResponse implements RESTResponse { private static final String FIELD_NAME = "name"; - private static final String FIELD_PROPERTIES = "properties"; + private static final String FIELD_OPTIONS = "options"; @JsonProperty(FIELD_NAME) private String name; - @JsonProperty(FIELD_PROPERTIES) - private Map properties; + @JsonProperty(FIELD_OPTIONS) + private Map options; - @ConstructorProperties({FIELD_NAME, FIELD_PROPERTIES}) - public CreateDatabaseResponse(String name, Map properties) { + @ConstructorProperties({FIELD_NAME, FIELD_OPTIONS}) + public CreateDatabaseResponse(String name, Map options) { this.name = name; - this.properties = properties; + this.options = options; } @JsonGetter(FIELD_NAME) @@ -50,8 +50,8 @@ public String getName() { return name; } - @JsonGetter(FIELD_PROPERTIES) - public Map getProperties() { - return properties; + @JsonGetter(FIELD_OPTIONS) + public Map getOptions() { + return options; } } 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 b10dc7c0c707..7d7a80d29ecd 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 @@ -58,8 +58,7 @@ public class RESTCatalogController { content = { @Content( schema = @Schema(implementation = ConfigResponse.class), - mediaType = "application/json" - ) + mediaType = "application/json") }), @ApiResponse( responseCode = "500", From 28585787fe0cdd429e75e3e381e9c66ac439d07b Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 11:51:33 +0800 Subject: [PATCH 09/19] generate new swagger api yaml for response change --- .../org/apache/paimon/rest/RESTCatalog.java | 2 +- paimon-open-api/rest-catalog-open-api.yaml | 68 +++++-------------- 2 files changed, 17 insertions(+), 53 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 26be4b82be7c..730eaef964cd 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 @@ -237,7 +237,7 @@ public void close() throws Exception { Map fetchOptionsFromServer( Map headers, Map clientProperties) { ConfigResponse response = - client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class,headers); + client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers); return response.merge(clientProperties); } diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 5b6b1ad0408b..df649670b936 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -1,21 +1,3 @@ -# -# 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 @@ -41,8 +23,8 @@ paths: schema: type: string responses: - "201": - description: Created + "200": + description: OK content: application/json: schema: @@ -67,20 +49,20 @@ paths: $ref: '#/components/schemas/CreateDatabaseRequest' required: true responses: - "201": - description: Created + "200": + description: OK content: application/json: schema: $ref: '#/components/schemas/CreateDatabaseResponse' - "500": - description: Internal Server Error "409": description: Resource has exist content: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error /api/v1/{prefix}/databases/{database}: get: tags: @@ -99,18 +81,18 @@ paths: schema: type: string responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/GetDatabaseResponse' "404": description: Resource not found content: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' - "201": - description: Created - content: - application/json: - schema: - $ref: '#/components/schemas/GetDatabaseResponse' "500": description: Internal Server Error /api/v1/config: @@ -120,27 +102,12 @@ paths: summary: Get Config operationId: getConfig responses: - "201": - description: Created + "200": + description: OK content: application/json: schema: $ref: '#/components/schemas/ConfigResponse' - examples: - getDefaults: - description: getDefaults - value: |- - { - "k1": "v1", - "k2": "v2", - } - getOverrides: - description: getOverrides - value: |- - { - "k3": "v1", - "k4": "v2", - } "500": description: Internal Server Error components: @@ -150,21 +117,18 @@ components: properties: name: type: string - writeOnly: true ignoreIfExists: type: boolean - writeOnly: true - properties: + options: type: object additionalProperties: type: string - writeOnly: true CreateDatabaseResponse: type: object properties: name: type: string - properties: + options: type: object additionalProperties: type: string From d6a4f3d924e30c744f17e35cde2830deb5294890 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 12:45:18 +0800 Subject: [PATCH 10/19] add licence to swagger api yaml --- paimon-open-api/rest-catalog-open-api.yaml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index df649670b936..feb603344943 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -1,3 +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. +# + +--- openapi: 3.0.1 info: title: RESTCatalog API From 946513e17df5339d7f01041d187878cced71b96e Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 12:47:09 +0800 Subject: [PATCH 11/19] fix comment in NettyBufferPool --- .../java/org/apache/paimon/service/network/NettyBufferPool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java b/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java index 72db65188fe3..151308d0edc5 100644 --- a/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java +++ b/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NettyBufferPool.java @@ -46,7 +46,7 @@ public NettyBufferPool(int numberOfArenas) { boolean preferDirect = true; // Arenas allocate chunks of pageSize << maxOrder bytes. With these - // getDefaults, this results in chunks of 16 MB. + // defaults, this results in chunks of 16 MB. int pageSize = 8192; int maxOrder = 11; From d0326da914fa957e86f2ffd422437d2e43b06b53 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 13:55:00 +0800 Subject: [PATCH 12/19] add drop database --- .../org/apache/paimon/rest/HttpClient.java | 24 +++++++++- .../org/apache/paimon/rest/RESTCatalog.java | 8 +++- .../org/apache/paimon/rest/RESTClient.java | 2 + .../org/apache/paimon/rest/RESTMessage.java | 3 ++ .../rest/requests/CreateDatabaseRequest.java | 2 - .../rest/requests/DropDatabaseRequest.java | 46 +++++++++++++++++++ .../paimon/rest/responses/ConfigResponse.java | 2 - .../responses/CreateDatabaseResponse.java | 2 - .../paimon/rest/responses/DatabaseName.java | 6 +-- .../paimon/rest/responses/ErrorResponse.java | 4 +- .../rest/responses/GetDatabaseResponse.java | 2 - .../rest/responses/ListDatabasesResponse.java | 2 - 12 files changed, 86 insertions(+), 17 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.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 e092711e5f97..97696aef09ed 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 @@ -95,6 +95,23 @@ public T post( } } + @Override + public T delete( + String path, RESTRequest body, Map headers) { + try { + RequestBody requestBody = buildRequestBody(body); + Request request = + new Request.Builder() + .url(uri + path) + .delete(requestBody) + .headers(Headers.of(headers)) + .build(); + return exec(request, null); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + @Override public void close() throws IOException { okHttpClient.dispatcher().cancelAll(); @@ -111,10 +128,13 @@ private T exec(Request request, Class responseType) response.code()); errorHandler.accept(error); } - if (responseBodyStr == null) { + if (responseType != null && responseBodyStr != null) { + return mapper.readValue(responseBodyStr, responseType); + } else if (responseType == null) { + return null; + } else { throw new RESTException("response body is null."); } - return mapper.readValue(responseBodyStr, responseType); } catch (Exception 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 730eaef964cd..01c46e20eeca 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,6 +32,7 @@ import org.apache.paimon.rest.exceptions.AlreadyExistsException; import org.apache.paimon.rest.exceptions.NoSuchResourceException; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.DropDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; @@ -159,7 +160,12 @@ public Database getDatabase(String name) throws DatabaseNotExistException { @Override public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException { - throw new UnsupportedOperationException(); + DropDatabaseRequest request = new DropDatabaseRequest(name, ignoreIfNotExists, cascade); + try { + client.delete(resourcePaths.database(name), request, headers()); + } catch (NoSuchResourceException e) { + throw new DatabaseNotExistException(name); + } } @Override 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 feeed06a417a..d0244f309ef4 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 @@ -28,4 +28,6 @@ public interface RESTClient extends Closeable { T post( String path, RESTRequest body, Class responseType, Map headers); + + T delete(String path, RESTRequest body, Map headers); } 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 6cb0b6fa6573..31d46df7ef0f 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,8 @@ package org.apache.paimon.rest; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + /** Interface to mark both REST requests and responses. */ +@JsonIgnoreProperties(ignoreUnknown = true) public interface RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java index 1a277ef8faf4..117fe9d7b859 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -21,14 +21,12 @@ import org.apache.paimon.rest.RESTRequest; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; import java.util.Map; /** Request for creating database. */ -@JsonIgnoreProperties(ignoreUnknown = true) public class CreateDatabaseRequest implements RESTRequest { private static final String FIELD_NAME = "name"; private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java new file mode 100644 index 000000000000..5a4a8b3b0078 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java @@ -0,0 +1,46 @@ +/* + * 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; + +/** Request for DropDatabase. */ +public class DropDatabaseRequest implements RESTRequest { + private final String name; + private final boolean ignoreIfNotExists; + private final boolean cascade; + + public DropDatabaseRequest(String name, boolean ignoreIfNotExists, boolean cascade) { + this.name = name; + this.ignoreIfNotExists = ignoreIfNotExists; + this.cascade = cascade; + } + + public String getName() { + return name; + } + + public boolean getIgnoreIfNotExists() { + return ignoreIfNotExists; + } + + public boolean getCascade() { + return cascade; + } +} 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 3660edc391d8..2a8da6b0a4aa 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 @@ -24,7 +24,6 @@ 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 org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; @@ -32,7 +31,6 @@ import java.util.Objects; /** Response for getting config. */ -@JsonIgnoreProperties(ignoreUnknown = true) public class ConfigResponse implements RESTResponse { private static final String FIELD_DEFAULTS = "defaults"; private static final String FIELD_OVERRIDES = "overrides"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java index 5ed92c9c2374..2c25c8668a63 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -21,14 +21,12 @@ import org.apache.paimon.rest.RESTResponse; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; import java.util.Map; /** Response for creating database. */ -@JsonIgnoreProperties(ignoreUnknown = true) public class CreateDatabaseResponse implements RESTResponse { private static final String FIELD_NAME = "name"; private static final String FIELD_OPTIONS = "options"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java index 58a6ad77394a..4a641a8e3ee5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java @@ -18,15 +18,15 @@ package org.apache.paimon.rest.responses; +import org.apache.paimon.rest.RESTMessage; + import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; /** Class for Database entity. */ -@JsonIgnoreProperties(ignoreUnknown = true) -public class DatabaseName { +public class DatabaseName implements RESTMessage { private static final String FIELD_NAME = "name"; @JsonProperty(FIELD_NAME) 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 c30cd3833427..b187165d7161 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,8 @@ package org.apache.paimon.rest.responses; +import org.apache.paimon.rest.RESTResponse; + import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; @@ -29,7 +31,7 @@ import java.util.List; /** Response for error. */ -public class ErrorResponse { +public class ErrorResponse implements RESTResponse { private static final String FIELD_MESSAGE = "message"; private static final String FIELD_CODE = "code"; private static final String FIELD_STACK = "stack"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java index 211dce6187d7..f9f077bdf21e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -22,7 +22,6 @@ import org.apache.paimon.rest.RESTResponse; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; @@ -32,7 +31,6 @@ import java.util.Optional; /** Response for getting database. */ -@JsonIgnoreProperties(ignoreUnknown = true) public class GetDatabaseResponse implements RESTResponse, Database { private static final String FIELD_NAME = "name"; private static final String FIELD_OPTIONS = "options"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java index fe01261e0a62..2a899b72f48d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java @@ -21,14 +21,12 @@ import org.apache.paimon.rest.RESTResponse; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; import java.util.List; /** Response for listing databases. */ -@JsonIgnoreProperties(ignoreUnknown = true) public class ListDatabasesResponse implements RESTResponse { private static final String FIELD_DATABASES = "databases"; From 54ffe906bbd2d4e663159b5342779a88359953d7 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 15:21:22 +0800 Subject: [PATCH 13/19] add drop database api define in swagger yaml --- .../org/apache/paimon/rest/RESTCatalog.java | 2 +- .../apache/paimon/rest/RESTObjectMapper.java | 1 + .../rest/requests/CreateDatabaseRequest.java | 9 ++- .../rest/requests/DropDatabaseRequest.java | 24 ++++-- .../paimon/rest/responses/ConfigResponse.java | 8 +- .../responses/CreateDatabaseResponse.java | 8 +- .../paimon/rest/responses/DatabaseName.java | 7 +- .../paimon/rest/responses/ErrorResponse.java | 9 ++- .../rest/responses/GetDatabaseResponse.java | 15 ++-- .../rest/responses/ListDatabasesResponse.java | 6 +- .../paimon/rest/RESTObjectMapperTest.java | 73 +++++++++++++++++ paimon-open-api/rest-catalog-open-api.yaml | 80 ++++++++++++++----- .../open/api/RESTCatalogController.java | 22 ++++- ...{OpenAPIConfig.java => SwaggerConfig.java} | 5 +- 14 files changed, 210 insertions(+), 59 deletions(-) rename paimon-open-api/src/main/java/org/apache/paimon/open/api/config/{OpenAPIConfig.java => SwaggerConfig.java} (97%) 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 01c46e20eeca..3c2538df0ca2 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 @@ -160,7 +160,7 @@ public Database getDatabase(String name) throws DatabaseNotExistException { @Override public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException { - DropDatabaseRequest request = new DropDatabaseRequest(name, ignoreIfNotExists, cascade); + DropDatabaseRequest request = new DropDatabaseRequest(ignoreIfNotExists, cascade); try { client.delete(resourcePaths.database(name), request, headers()); } catch (NoSuchResourceException e) { 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 b1c83e90224a..2a014fe87b7c 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 @@ -29,6 +29,7 @@ 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.configure(SerializationFeature.WRITE_SELF_REFERENCES_AS_NULL, true); mapper.registerModule(new JavaTimeModule()); return mapper; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java index 117fe9d7b859..d746a193c959 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -20,10 +20,10 @@ import org.apache.paimon.rest.RESTRequest; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.beans.ConstructorProperties; import java.util.Map; /** Request for creating database. */ @@ -41,8 +41,11 @@ public class CreateDatabaseRequest implements RESTRequest { @JsonProperty(FIELD_OPTIONS) private Map options; - @ConstructorProperties({FIELD_NAME, FIELD_IGNORE_IF_EXISTS, FIELD_OPTIONS}) - public CreateDatabaseRequest(String name, boolean ignoreIfExists, Map options) { + @JsonCreator + public CreateDatabaseRequest( + @JsonProperty(FIELD_NAME) String name, + @JsonProperty(FIELD_IGNORE_IF_EXISTS) boolean ignoreIfExists, + @JsonProperty(FIELD_OPTIONS) Map options) { this.name = name; this.ignoreIfExists = ignoreIfExists; this.options = options; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java index 5a4a8b3b0078..d97f211c1caa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/DropDatabaseRequest.java @@ -20,26 +20,36 @@ import org.apache.paimon.rest.RESTRequest; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + /** Request for DropDatabase. */ public class DropDatabaseRequest implements RESTRequest { - private final String name; + + private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; + private static final String FIELD_CASCADE = "cascade"; + + @JsonProperty(FIELD_IGNORE_IF_EXISTS) private final boolean ignoreIfNotExists; + + @JsonProperty(FIELD_CASCADE) private final boolean cascade; - public DropDatabaseRequest(String name, boolean ignoreIfNotExists, boolean cascade) { - this.name = name; + @JsonCreator + public DropDatabaseRequest( + @JsonProperty(FIELD_IGNORE_IF_EXISTS) boolean ignoreIfNotExists, + @JsonProperty(FIELD_CASCADE) boolean cascade) { this.ignoreIfNotExists = ignoreIfNotExists; this.cascade = cascade; } - public String getName() { - return name; - } - + @JsonGetter(FIELD_IGNORE_IF_EXISTS) public boolean getIgnoreIfNotExists() { return ignoreIfNotExists; } + @JsonGetter(FIELD_CASCADE) public boolean getCascade() { return cascade; } 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 2a8da6b0a4aa..c86923bdf7c9 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 @@ -23,10 +23,10 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.beans.ConstructorProperties; import java.util.Map; import java.util.Objects; @@ -41,8 +41,10 @@ public class ConfigResponse implements RESTResponse { @JsonProperty(FIELD_OVERRIDES) private Map overrides; - @ConstructorProperties({FIELD_DEFAULTS, FIELD_OVERRIDES}) - public ConfigResponse(Map defaults, Map overrides) { + @JsonCreator + public ConfigResponse( + @JsonProperty(FIELD_DEFAULTS) Map defaults, + @JsonProperty(FIELD_OVERRIDES) Map overrides) { this.defaults = defaults; this.overrides = overrides; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java index 2c25c8668a63..f81e53622b0d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -20,10 +20,10 @@ import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.beans.ConstructorProperties; import java.util.Map; /** Response for creating database. */ @@ -37,8 +37,10 @@ public class CreateDatabaseResponse implements RESTResponse { @JsonProperty(FIELD_OPTIONS) private Map options; - @ConstructorProperties({FIELD_NAME, FIELD_OPTIONS}) - public CreateDatabaseResponse(String name, Map options) { + @JsonCreator + public CreateDatabaseResponse( + @JsonProperty(FIELD_NAME) String name, + @JsonProperty(FIELD_OPTIONS) Map options) { this.name = name; this.options = options; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java index 4a641a8e3ee5..02847e0b1595 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java @@ -20,11 +20,10 @@ import org.apache.paimon.rest.RESTMessage; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.beans.ConstructorProperties; - /** Class for Database entity. */ public class DatabaseName implements RESTMessage { private static final String FIELD_NAME = "name"; @@ -32,8 +31,8 @@ public class DatabaseName implements RESTMessage { @JsonProperty(FIELD_NAME) private String name; - @ConstructorProperties({FIELD_NAME}) - public DatabaseName(String name) { + @JsonCreator + public DatabaseName(@JsonProperty(FIELD_NAME) String name) { this.name = name; } 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 b187165d7161..c3ab2beb42de 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,10 +20,10 @@ import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.beans.ConstructorProperties; import java.io.PrintWriter; import java.io.StringWriter; import java.util.ArrayList; @@ -51,8 +51,11 @@ public ErrorResponse(String message, Integer code) { this.stack = new ArrayList(); } - @ConstructorProperties({FIELD_MESSAGE, FIELD_CODE, FIELD_STACK}) - public ErrorResponse(String message, int code, List stack) { + @JsonCreator + public ErrorResponse( + @JsonProperty(FIELD_MESSAGE) String message, + @JsonProperty(FIELD_CODE) int code, + @JsonProperty(FIELD_STACK) List stack) { this.message = message; this.code = code; this.stack = stack; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java index f9f077bdf21e..662cb492217c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -21,12 +21,12 @@ import org.apache.paimon.catalog.Database; import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; -import java.beans.ConstructorProperties; import java.util.Map; import java.util.Optional; @@ -46,8 +46,11 @@ public class GetDatabaseResponse implements RESTResponse, Database { @Nullable private final String comment; - @ConstructorProperties({FIELD_NAME, FIELD_OPTIONS, FIELD_COMMENT}) - public GetDatabaseResponse(String name, Map options, @Nullable String comment) { + @JsonCreator + public GetDatabaseResponse( + @JsonProperty(FIELD_NAME) String name, + @JsonProperty(FIELD_OPTIONS) Map options, + @JsonProperty(FIELD_COMMENT) @Nullable String comment) { this.name = name; this.options = options; this.comment = comment; @@ -64,8 +67,8 @@ public Map getOptions() { } @JsonGetter(FIELD_COMMENT) - public Optional getComment() { - return Optional.ofNullable(comment); + public String getComment() { + return comment; } @Override @@ -80,6 +83,6 @@ public Map options() { @Override public Optional comment() { - return this.getComment(); + return Optional.ofNullable(this.getComment()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java index 2a899b72f48d..38773f354b77 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java @@ -20,10 +20,10 @@ import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.beans.ConstructorProperties; import java.util.List; /** Response for listing databases. */ @@ -33,8 +33,8 @@ public class ListDatabasesResponse implements RESTResponse { @JsonProperty(FIELD_DATABASES) private List databases; - @ConstructorProperties({FIELD_DATABASES}) - public ListDatabasesResponse(List databases) { + @JsonCreator + public ListDatabasesResponse(@JsonProperty(FIELD_DATABASES) List databases) { this.databases = databases; } 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 65859a9dd283..c8fbd62213df 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 @@ -18,8 +18,14 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.DropDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.DatabaseName; import org.apache.paimon.rest.responses.ErrorResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -27,6 +33,7 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -56,4 +63,70 @@ public void errorResponseParseTest() throws Exception { assertEquals(message, parseData.getMessage()); assertEquals(code, parseData.getCode()); } + + @Test + public void createDatabaseRequestParseTest() throws Exception { + String name = "name"; + boolean ignoreIfExists = true; + Map options = new HashMap<>(); + options.put("a", "b"); + CreateDatabaseRequest request = new CreateDatabaseRequest(name, ignoreIfExists, options); + String requestStr = mapper.writeValueAsString(request); + CreateDatabaseRequest parseData = mapper.readValue(requestStr, CreateDatabaseRequest.class); + assertEquals(name, parseData.getName()); + assertEquals(ignoreIfExists, parseData.getIgnoreIfExists()); + assertEquals(options, parseData.getOptions()); + } + + @Test + public void dropDatabaseRequestParseTest() throws Exception { + boolean ignoreIfNotExists = true; + boolean cascade = true; + DropDatabaseRequest request = new DropDatabaseRequest(ignoreIfNotExists, cascade); + String requestStr = mapper.writeValueAsString(request); + DropDatabaseRequest parseData = mapper.readValue(requestStr, DropDatabaseRequest.class); + assertEquals(ignoreIfNotExists, parseData.getIgnoreIfNotExists()); + assertEquals(cascade, parseData.getCascade()); + } + + @Test + public void createDatabaseResponseParseTest() throws Exception { + String name = "name"; + Map options = new HashMap<>(); + options.put("a", "b"); + CreateDatabaseResponse response = new CreateDatabaseResponse(name, options); + String responseStr = mapper.writeValueAsString(response); + CreateDatabaseResponse parseData = + mapper.readValue(responseStr, CreateDatabaseResponse.class); + assertEquals(name, parseData.getName()); + assertEquals(options, parseData.getOptions()); + } + + @Test + public void getDatabaseResponseParseTest() throws Exception { + String name = "name"; + Map options = new HashMap<>(); + options.put("a", "b"); + String comment = "comment"; + GetDatabaseResponse response = new GetDatabaseResponse(name, options, comment); + String responseStr = mapper.writeValueAsString(response); + GetDatabaseResponse parseData = mapper.readValue(responseStr, GetDatabaseResponse.class); + assertEquals(name, parseData.getName()); + assertEquals(options, parseData.getOptions()); + assertEquals(comment, parseData.getComment()); + } + + @Test + public void listDatabaseResponseParseTest() throws Exception { + String name = "name"; + DatabaseName databaseName = new DatabaseName(name); + List databaseNameList = new ArrayList<>(); + databaseNameList.add(databaseName); + ListDatabasesResponse response = new ListDatabasesResponse(databaseNameList); + String responseStr = mapper.writeValueAsString(response); + ListDatabasesResponse parseData = + mapper.readValue(responseStr, ListDatabasesResponse.class); + assertEquals(databaseNameList.size(), parseData.getDatabases().size()); + assertEquals(name, parseData.getDatabases().get(0).getName()); + } } diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index feb603344943..91c460201607 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -41,14 +41,14 @@ paths: schema: type: string responses: + "500": + description: Internal Server Error "200": description: OK content: application/json: schema: $ref: '#/components/schemas/ListDatabasesResponse' - "500": - description: Internal Server Error post: tags: - database @@ -65,22 +65,21 @@ paths: application/json: schema: $ref: '#/components/schemas/CreateDatabaseRequest' - required: true responses: - "200": - description: OK - content: - application/json: - schema: - $ref: '#/components/schemas/CreateDatabaseResponse' + "500": + description: Internal Server Error "409": description: Resource has exist content: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' - "500": - description: Internal Server Error + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/CreateDatabaseResponse' /api/v1/{prefix}/databases/{database}: get: tags: @@ -105,14 +104,44 @@ paths: application/json: schema: $ref: '#/components/schemas/GetDatabaseResponse' + "500": + description: Internal Server Error "404": description: Resource not found content: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' + delete: + tags: + - database + summary: Drop Database + operationId: dropDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/DropDatabaseRequest' + responses: "500": description: Internal Server Error + "404": + description: Resource not found + content: + '*/*': + schema: + $ref: '#/components/schemas/ErrorResponse' /api/v1/config: get: tags: @@ -120,14 +149,14 @@ paths: summary: Get Config operationId: getConfig responses: + "500": + description: Internal Server Error "200": description: OK content: application/json: schema: $ref: '#/components/schemas/ConfigResponse' - "500": - description: Internal Server Error components: schemas: CreateDatabaseRequest: @@ -141,15 +170,6 @@ components: type: object additionalProperties: type: string - CreateDatabaseResponse: - type: object - properties: - name: - type: string - options: - type: object - additionalProperties: - type: string ErrorResponse: type: object properties: @@ -162,6 +182,15 @@ components: type: array items: type: string + CreateDatabaseResponse: + type: object + properties: + name: + type: string + options: + type: object + additionalProperties: + type: string DatabaseName: type: object properties: @@ -196,3 +225,10 @@ components: type: object additionalProperties: type: string + DropDatabaseRequest: + type: object + properties: + ignoreIfNotExists: + type: boolean + cascade: + type: boolean 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 7d7a80d29ecd..fdde60d670ca 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 @@ -20,6 +20,7 @@ import org.apache.paimon.rest.ResourcePaths; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.DropDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; @@ -32,13 +33,14 @@ 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.parameters.RequestBody; import io.swagger.v3.oas.annotations.responses.ApiResponse; import io.swagger.v3.oas.annotations.responses.ApiResponses; import org.springframework.web.bind.annotation.CrossOrigin; +import org.springframework.web.bind.annotation.DeleteMapping; import org.springframework.web.bind.annotation.GetMapping; 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; import java.util.HashMap; @@ -142,4 +144,22 @@ public GetDatabaseResponse getDatabases( Map options = new HashMap<>(); return new GetDatabaseResponse("name", options, "comment"); } + + @Operation( + summary = "Drop Database", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @DeleteMapping("/api/v1/{prefix}/databases/{database}") + public void dropDatabases( + @PathVariable String prefix, + @PathVariable String database, + @RequestBody DropDatabaseRequest request) {} } 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/SwaggerConfig.java similarity index 97% rename from paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java rename to paimon-open-api/src/main/java/org/apache/paimon/open/api/config/SwaggerConfig.java index 01234c41bbff..c64aca05c50b 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/SwaggerConfig.java @@ -29,10 +29,9 @@ import java.util.ArrayList; import java.util.List; -/** Config for OpenAPI. */ +/** Configuration for Swagger. */ @Configuration -public class OpenAPIConfig { - +public class SwaggerConfig { @Value("${openapi.url}") private String devUrl; From 4c20018a98e80ebc53327d7b62058b643c61d79d Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 15:25:57 +0800 Subject: [PATCH 14/19] delete no need conf in RESTObjectMapper --- .../src/main/java/org/apache/paimon/rest/RESTObjectMapper.java | 1 - 1 file changed, 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 2a014fe87b7c..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 @@ -29,7 +29,6 @@ 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.configure(SerializationFeature.WRITE_SELF_REFERENCES_AS_NULL, true); mapper.registerModule(new JavaTimeModule()); return mapper; } From e64306f0a38cd8988ac26d9e3f2b7a96557ab557 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 16:12:47 +0800 Subject: [PATCH 15/19] add test for listDatabases, createDatabase, getDatabase, dropDatabase in RESTCatalog --- .../paimon/rest/DefaultErrorHandlerTest.java | 8 +++ .../apache/paimon/rest/HttpClientTest.java | 14 ++++ .../apache/paimon/rest/MockRESTMessage.java | 72 +++++++++++++++++++ .../apache/paimon/rest/RESTCatalogTest.java | 58 +++++++++++++-- .../paimon/rest/RESTObjectMapperTest.java | 51 +++++-------- paimon-open-api/rest-catalog-open-api.yaml | 26 ++----- 6 files changed, 170 insertions(+), 59 deletions(-) create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java 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 1f1b9c01aace..340e38f6a7f8 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 @@ -18,8 +18,10 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.exceptions.AlreadyExistsException; import org.apache.paimon.rest.exceptions.BadRequestException; import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NoSuchResourceException; import org.apache.paimon.rest.exceptions.NotAuthorizedException; import org.apache.paimon.rest.exceptions.RESTException; import org.apache.paimon.rest.exceptions.ServiceFailureException; @@ -54,10 +56,16 @@ public void testHandleErrorResponse() { assertThrows( ForbiddenException.class, () -> defaultErrorHandler.accept(generateErrorResponse(403))); + assertThrows( + NoSuchResourceException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(404))); assertThrows( RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(405))); assertThrows( RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(406))); + assertThrows( + AlreadyExistsException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(409))); assertThrows( ServiceFailureException.class, () -> defaultErrorHandler.accept(generateErrorResponse(500))); 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 17c13b932fd2..f12af12a9d35 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 @@ -110,6 +110,20 @@ public void testPostFail() { verify(errorHandler, times(1)).accept(any()); } + @Test + public void testDeleteSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = httpClient.delete(MOCK_PATH, mockResponseData, headers); + verify(errorHandler, times(0)).accept(any()); + } + + @Test + public void testDeleteFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.delete(MOCK_PATH, mockResponseData, headers); + verify(errorHandler, times(1)).accept(any()); + } + private Map headers(String token) { Map header = new HashMap<>(); header.put("Authorization", "Bearer " + token); diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java new file mode 100644 index 000000000000..5d1906c5064c --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.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.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.DropDatabaseRequest; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** Mock REST message. */ +public class MockRESTMessage { + + public static String databaseName() { + return "database"; + } + + public static CreateDatabaseRequest createDatabaseRequest(String name) { + boolean ignoreIfExists = true; + Map options = new HashMap<>(); + options.put("a", "b"); + return new CreateDatabaseRequest(name, ignoreIfExists, options); + } + + public static DropDatabaseRequest dropDatabaseRequest() { + boolean ignoreIfNotExists = true; + boolean cascade = true; + return new DropDatabaseRequest(ignoreIfNotExists, cascade); + } + + public static CreateDatabaseResponse createDatabaseResponse(String name) { + Map options = new HashMap<>(); + options.put("a", "b"); + return new CreateDatabaseResponse(name, options); + } + + public static GetDatabaseResponse getDatabaseResponse(String name) { + Map options = new HashMap<>(); + options.put("a", "b"); + String comment = "comment"; + return new GetDatabaseResponse(name, options, comment); + } + + public static ListDatabasesResponse listDatabasesResponse(String name) { + DatabaseName databaseName = new DatabaseName(name); + List databaseNameList = new ArrayList<>(); + databaseNameList.add(databaseName); + return new ListDatabasesResponse(databaseNameList); + } +} 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 ff51e355f19f..5fc80c952bd6 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,8 +18,15 @@ package org.apache.paimon.rest; +import org.apache.paimon.catalog.Database; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import okhttp3.mockwebserver.MockResponse; import okhttp3.mockwebserver.MockWebServer; @@ -29,14 +36,17 @@ import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; /** Test for REST Catalog. */ public class RESTCatalogTest { + private ObjectMapper mapper = RESTObjectMapper.create(); private MockWebServer mockWebServer; private RESTCatalog restCatalog; @@ -50,7 +60,11 @@ public void setUp() throws IOException { String initToken = "init_token"; options.set(RESTCatalogOptions.TOKEN, initToken); options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); - mockOptions(RESTCatalogInternalOptions.PREFIX.key(), "prefix"); + String mockResponse = + String.format( + "{\"defaults\": {\"%s\": \"%s\"}}", + RESTCatalogInternalOptions.PREFIX.key(), "prefix"); + mockResponse(mockResponse); restCatalog = new RESTCatalog(options); } @@ -70,14 +84,50 @@ public void testInitFailWhenDefineWarehouse() { public void testGetConfig() { String key = "a"; String value = "b"; - mockOptions(key, value); + String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); + mockResponse(mockResponse); Map header = new HashMap<>(); Map response = restCatalog.fetchOptionsFromServer(header, new HashMap<>()); assertEquals(value, response.get(key)); } - private void mockOptions(String key, String value) { - String mockResponse = String.format("{\"getDefaults\": {\"%s\": \"%s\"}}", key, value); + @Test + public void testListDatabases() throws JsonProcessingException { + String name = MockRESTMessage.databaseName(); + ListDatabasesResponse response = MockRESTMessage.listDatabasesResponse(name); + mockResponse(mapper.writeValueAsString(response)); + List result = restCatalog.listDatabases(); + assertEquals(response.getDatabases().size(), result.size()); + assertEquals(name, result.get(0)); + } + + @Test + public void testCreateDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + CreateDatabaseResponse response = MockRESTMessage.createDatabaseResponse(name); + mockResponse(mapper.writeValueAsString(response)); + assertDoesNotThrow(() -> restCatalog.createDatabase(name, false, response.getOptions())); + } + + @Test + public void testGetDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + GetDatabaseResponse response = MockRESTMessage.getDatabaseResponse(name); + mockResponse(mapper.writeValueAsString(response)); + Database result = restCatalog.getDatabase(name); + assertEquals(name, result.name()); + assertEquals(response.getOptions().size(), result.options().size()); + assertEquals(response.getComment(), result.comment().get()); + } + + @Test + public void testDropDatabase() { + String name = "name"; + mockResponse(""); + assertDoesNotThrow(() -> restCatalog.dropDatabase(name, false, false)); + } + + private void mockResponse(String mockResponse) { 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 c8fbd62213df..152d6f3e505c 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 @@ -22,7 +22,6 @@ import org.apache.paimon.rest.requests.DropDatabaseRequest; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; -import org.apache.paimon.rest.responses.DatabaseName; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; @@ -33,7 +32,6 @@ import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -66,67 +64,54 @@ public void errorResponseParseTest() throws Exception { @Test public void createDatabaseRequestParseTest() throws Exception { - String name = "name"; - boolean ignoreIfExists = true; - Map options = new HashMap<>(); - options.put("a", "b"); - CreateDatabaseRequest request = new CreateDatabaseRequest(name, ignoreIfExists, options); + String name = MockRESTMessage.databaseName(); + CreateDatabaseRequest request = MockRESTMessage.createDatabaseRequest(name); String requestStr = mapper.writeValueAsString(request); CreateDatabaseRequest parseData = mapper.readValue(requestStr, CreateDatabaseRequest.class); - assertEquals(name, parseData.getName()); - assertEquals(ignoreIfExists, parseData.getIgnoreIfExists()); - assertEquals(options, parseData.getOptions()); + assertEquals(request.getName(), parseData.getName()); + assertEquals(request.getIgnoreIfExists(), parseData.getIgnoreIfExists()); + assertEquals(request.getOptions().size(), parseData.getOptions().size()); } @Test public void dropDatabaseRequestParseTest() throws Exception { - boolean ignoreIfNotExists = true; - boolean cascade = true; - DropDatabaseRequest request = new DropDatabaseRequest(ignoreIfNotExists, cascade); + DropDatabaseRequest request = MockRESTMessage.dropDatabaseRequest(); String requestStr = mapper.writeValueAsString(request); DropDatabaseRequest parseData = mapper.readValue(requestStr, DropDatabaseRequest.class); - assertEquals(ignoreIfNotExists, parseData.getIgnoreIfNotExists()); - assertEquals(cascade, parseData.getCascade()); + assertEquals(request.getIgnoreIfNotExists(), parseData.getIgnoreIfNotExists()); + assertEquals(request.getCascade(), parseData.getCascade()); } @Test public void createDatabaseResponseParseTest() throws Exception { - String name = "name"; - Map options = new HashMap<>(); - options.put("a", "b"); - CreateDatabaseResponse response = new CreateDatabaseResponse(name, options); + String name = MockRESTMessage.databaseName(); + CreateDatabaseResponse response = MockRESTMessage.createDatabaseResponse(name); String responseStr = mapper.writeValueAsString(response); CreateDatabaseResponse parseData = mapper.readValue(responseStr, CreateDatabaseResponse.class); assertEquals(name, parseData.getName()); - assertEquals(options, parseData.getOptions()); + assertEquals(response.getOptions().size(), parseData.getOptions().size()); } @Test public void getDatabaseResponseParseTest() throws Exception { - String name = "name"; - Map options = new HashMap<>(); - options.put("a", "b"); - String comment = "comment"; - GetDatabaseResponse response = new GetDatabaseResponse(name, options, comment); + String name = MockRESTMessage.databaseName(); + GetDatabaseResponse response = MockRESTMessage.getDatabaseResponse(name); String responseStr = mapper.writeValueAsString(response); GetDatabaseResponse parseData = mapper.readValue(responseStr, GetDatabaseResponse.class); assertEquals(name, parseData.getName()); - assertEquals(options, parseData.getOptions()); - assertEquals(comment, parseData.getComment()); + assertEquals(response.getOptions().size(), parseData.getOptions().size()); + assertEquals(response.getComment(), parseData.getComment()); } @Test public void listDatabaseResponseParseTest() throws Exception { - String name = "name"; - DatabaseName databaseName = new DatabaseName(name); - List databaseNameList = new ArrayList<>(); - databaseNameList.add(databaseName); - ListDatabasesResponse response = new ListDatabasesResponse(databaseNameList); + String name = MockRESTMessage.databaseName(); + ListDatabasesResponse response = MockRESTMessage.listDatabasesResponse(name); String responseStr = mapper.writeValueAsString(response); ListDatabasesResponse parseData = mapper.readValue(responseStr, ListDatabasesResponse.class); - assertEquals(databaseNameList.size(), parseData.getDatabases().size()); + assertEquals(response.getDatabases().size(), parseData.getDatabases().size()); assertEquals(name, parseData.getDatabases().get(0).getName()); } } diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 91c460201607..ccc020b32f64 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -1,21 +1,3 @@ -# -# 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 @@ -104,14 +86,14 @@ paths: application/json: schema: $ref: '#/components/schemas/GetDatabaseResponse' - "500": - description: Internal Server Error "404": description: Resource not found content: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error delete: tags: - database @@ -134,14 +116,14 @@ paths: schema: $ref: '#/components/schemas/DropDatabaseRequest' responses: - "500": - description: Internal Server Error "404": description: Resource not found content: '*/*': schema: $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error /api/v1/config: get: tags: From 69de916fa9c2bfa9c30d5f5ad0973e1e3d18427f Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 17:15:51 +0800 Subject: [PATCH 16/19] format --- .../apache/paimon/rest/exceptions/AlreadyExistsException.java | 1 + .../apache/paimon/rest/exceptions/NoSuchResourceException.java | 1 + .../org/apache/paimon/rest/requests/CreateDatabaseRequest.java | 1 + .../java/org/apache/paimon/rest/responses/ConfigResponse.java | 1 + .../org/apache/paimon/rest/responses/CreateDatabaseResponse.java | 1 + .../main/java/org/apache/paimon/rest/responses/DatabaseName.java | 1 + .../java/org/apache/paimon/rest/responses/ErrorResponse.java | 1 + .../org/apache/paimon/rest/responses/GetDatabaseResponse.java | 1 + 8 files changed, 8 insertions(+) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java index 0a7263f9ce1c..8e30c8375bf9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java @@ -20,6 +20,7 @@ /** Exception thrown on HTTP 409 means a resource already exists. */ public class AlreadyExistsException extends RESTException { + public AlreadyExistsException(String message, Object... args) { super(message, args); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java index f8c3b05b951c..cc4c7881f465 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java @@ -20,6 +20,7 @@ /** Exception thrown on HTTP 404 means a resource not exists. */ public class NoSuchResourceException extends RESTException { + public NoSuchResourceException(String message, Object... args) { super(message, args); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java index d746a193c959..6067bf544b87 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -28,6 +28,7 @@ /** Request for creating database. */ public class CreateDatabaseRequest implements RESTRequest { + private static final String FIELD_NAME = "name"; private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; private static final String FIELD_OPTIONS = "options"; 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 c86923bdf7c9..e8fff88b09c2 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 @@ -32,6 +32,7 @@ /** Response for getting config. */ public class ConfigResponse implements RESTResponse { + private static final String FIELD_DEFAULTS = "defaults"; private static final String FIELD_OVERRIDES = "overrides"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java index f81e53622b0d..43c99254f399 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -28,6 +28,7 @@ /** Response for creating database. */ public class CreateDatabaseResponse implements RESTResponse { + private static final String FIELD_NAME = "name"; private static final String FIELD_OPTIONS = "options"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java index 02847e0b1595..9a93b2fd1e3d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java @@ -26,6 +26,7 @@ /** Class for Database entity. */ public class DatabaseName implements RESTMessage { + private static final String FIELD_NAME = "name"; @JsonProperty(FIELD_NAME) 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 c3ab2beb42de..d24c8f0f9936 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 @@ -32,6 +32,7 @@ /** Response for error. */ public class ErrorResponse implements RESTResponse { + private static final String FIELD_MESSAGE = "message"; private static final String FIELD_CODE = "code"; private static final String FIELD_STACK = "stack"; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java index 662cb492217c..5f002fbf2883 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -32,6 +32,7 @@ /** Response for getting database. */ public class GetDatabaseResponse implements RESTResponse, Database { + private static final String FIELD_NAME = "name"; private static final String FIELD_OPTIONS = "options"; private static final String FIELD_COMMENT = "comment"; From 2ba96fe02f47ef595bf6e3686c1354622d8208cb Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 17:22:58 +0800 Subject: [PATCH 17/19] conf reset to OpenAPIConfig --- .../api/config/{SwaggerConfig.java => OpenAPIConfig.java} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename paimon-open-api/src/main/java/org/apache/paimon/open/api/config/{SwaggerConfig.java => OpenAPIConfig.java} (97%) diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/SwaggerConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java similarity index 97% rename from paimon-open-api/src/main/java/org/apache/paimon/open/api/config/SwaggerConfig.java rename to paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java index c64aca05c50b..0e28cd95f9d2 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/SwaggerConfig.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -29,9 +29,9 @@ import java.util.ArrayList; import java.util.List; -/** Configuration for Swagger. */ +/** Config for OpenAPI. */ @Configuration -public class SwaggerConfig { +public class OpenAPIConfig { @Value("${openapi.url}") private String devUrl; From 47303a16f9b144e63535cef1c1289dc4c30f940f Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 17:37:27 +0800 Subject: [PATCH 18/19] move comment to options in GetDatabaseResponse --- .../rest/RESTCatalogInternalOptions.java | 5 +++++ .../rest/responses/GetDatabaseResponse.java | 21 +++++-------------- .../apache/paimon/rest/MockRESTMessage.java | 6 ++++-- .../apache/paimon/rest/RESTCatalogTest.java | 2 +- .../paimon/rest/RESTObjectMapperTest.java | 2 +- paimon-open-api/rest-catalog-open-api.yaml | 2 -- .../open/api/RESTCatalogController.java | 2 +- 7 files changed, 17 insertions(+), 23 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 62a8bf134ae5..722010923c46 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 @@ -33,4 +33,9 @@ public class RESTCatalogInternalOptions { .stringType() .noDefaultValue() .withDescription("REST Catalog auth credentials provider."); + public static final ConfigOption DATABASE_COMMENT = + ConfigOptions.key("comment") + .stringType() + .defaultValue(null) + .withDescription("REST Catalog database comment."); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java index 5f002fbf2883..f8f7c8794b7b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -25,17 +25,16 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import javax.annotation.Nullable; - import java.util.Map; import java.util.Optional; +import static org.apache.paimon.rest.RESTCatalogInternalOptions.DATABASE_COMMENT; + /** Response for getting database. */ public class GetDatabaseResponse implements RESTResponse, Database { private static final String FIELD_NAME = "name"; private static final String FIELD_OPTIONS = "options"; - private static final String FIELD_COMMENT = "comment"; @JsonProperty(FIELD_NAME) private final String name; @@ -43,18 +42,12 @@ public class GetDatabaseResponse implements RESTResponse, Database { @JsonProperty(FIELD_OPTIONS) private final Map options; - @JsonProperty(FIELD_COMMENT) - @Nullable - private final String comment; - @JsonCreator public GetDatabaseResponse( @JsonProperty(FIELD_NAME) String name, - @JsonProperty(FIELD_OPTIONS) Map options, - @JsonProperty(FIELD_COMMENT) @Nullable String comment) { + @JsonProperty(FIELD_OPTIONS) Map options) { this.name = name; this.options = options; - this.comment = comment; } @JsonGetter(FIELD_NAME) @@ -67,11 +60,6 @@ public Map getOptions() { return options; } - @JsonGetter(FIELD_COMMENT) - public String getComment() { - return comment; - } - @Override public String name() { return this.getName(); @@ -84,6 +72,7 @@ public Map options() { @Override public Optional comment() { - return Optional.ofNullable(this.getComment()); + return Optional.ofNullable( + this.options.getOrDefault(DATABASE_COMMENT.key(), DATABASE_COMMENT.defaultValue())); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java index 5d1906c5064c..f111c41f6ada 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java @@ -30,6 +30,8 @@ import java.util.List; import java.util.Map; +import static org.apache.paimon.rest.RESTCatalogInternalOptions.DATABASE_COMMENT; + /** Mock REST message. */ public class MockRESTMessage { @@ -59,8 +61,8 @@ public static CreateDatabaseResponse createDatabaseResponse(String name) { public static GetDatabaseResponse getDatabaseResponse(String name) { Map options = new HashMap<>(); options.put("a", "b"); - String comment = "comment"; - return new GetDatabaseResponse(name, options, comment); + options.put(DATABASE_COMMENT.key(), "comment"); + return new GetDatabaseResponse(name, options); } public static ListDatabasesResponse listDatabasesResponse(String name) { 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 5fc80c952bd6..cffac6046623 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 @@ -117,7 +117,7 @@ public void testGetDatabase() throws Exception { Database result = restCatalog.getDatabase(name); assertEquals(name, result.name()); assertEquals(response.getOptions().size(), result.options().size()); - assertEquals(response.getComment(), result.comment().get()); + assertEquals(response.comment().get(), result.comment().get()); } @Test 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 152d6f3e505c..622a98993692 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 @@ -101,7 +101,7 @@ public void getDatabaseResponseParseTest() throws Exception { GetDatabaseResponse parseData = mapper.readValue(responseStr, GetDatabaseResponse.class); assertEquals(name, parseData.getName()); assertEquals(response.getOptions().size(), parseData.getOptions().size()); - assertEquals(response.getComment(), parseData.getComment()); + assertEquals(response.comment().get(), parseData.comment().get()); } @Test diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index ccc020b32f64..fc46c81d31a6 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -194,8 +194,6 @@ components: type: object additionalProperties: type: string - comment: - type: string 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 fdde60d670ca..364cc5adbb2c 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 @@ -142,7 +142,7 @@ public CreateDatabaseResponse createDatabases( public GetDatabaseResponse getDatabases( @PathVariable String prefix, @PathVariable String database) { Map options = new HashMap<>(); - return new GetDatabaseResponse("name", options, "comment"); + return new GetDatabaseResponse("name", options); } @Operation( From bfee5baf7674c983f50992e7c7921854da26224b Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Dec 2024 17:53:14 +0800 Subject: [PATCH 19/19] add license to open api yaml --- paimon-open-api/rest-catalog-open-api.yaml | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index fc46c81d31a6..2a5d1dc58418 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -1,3 +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. +# + +--- openapi: 3.0.1 info: title: RESTCatalog API @@ -23,14 +41,14 @@ paths: schema: type: string responses: - "500": - description: Internal Server Error "200": description: OK content: application/json: schema: $ref: '#/components/schemas/ListDatabasesResponse' + "500": + description: Internal Server Error post: tags: - database