From 3b83dd58e97e71aa1a42d4db83927114d09f5f93 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 17 Dec 2024 18:07:39 +0800 Subject: [PATCH 01/23] change RESTCatalog extends AbstractCatalog --- .../org/apache/paimon/rest/RESTCatalog.java | 88 ++++++------------- .../apache/paimon/rest/RESTCatalogTest.java | 75 ++-------------- 2 files changed, 34 insertions(+), 129 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 8b53bef8486b..efc0074a1ce9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -18,18 +18,16 @@ package org.apache.paimon.rest; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; 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.exceptions.NoSuchResourceException; import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; @@ -41,7 +39,7 @@ 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.schema.TableSchema; import org.apache.paimon.utils.Pair; import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; @@ -50,6 +48,7 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -61,7 +60,7 @@ import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; /** A catalog implementation for REST. */ -public class RESTCatalog implements Catalog { +public class RESTCatalog extends AbstractCatalog { private static final ObjectMapper OBJECT_MAPPER = RESTObjectMapper.create(); @@ -74,6 +73,11 @@ public class RESTCatalog implements Catalog { private volatile ScheduledExecutorService refreshExecutor = null; public RESTCatalog(Options options) { + this(null, options); + } + + public RESTCatalog(FileIO fileIO, Options options) { + super(fileIO, options); if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); } @@ -139,21 +143,13 @@ public List listDatabases() { } @Override - public void createDatabase(String name, boolean ignoreIfExists, Map properties) - throws DatabaseAlreadyExistException { + protected void createDatabaseImpl(String name, Map properties) { CreateDatabaseRequest request = new CreateDatabaseRequest(name, properties); - try { - client.post( - resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new DatabaseAlreadyExistException(name); - } - } + client.post(resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); } @Override - public Database getDatabase(String name) throws DatabaseNotExistException { + protected Database getDatabaseImpl(String name) throws DatabaseNotExistException { try { GetDatabaseResponse response = client.get(resourcePaths.database(name), GetDatabaseResponse.class, headers()); @@ -165,22 +161,12 @@ public Database getDatabase(String name) throws DatabaseNotExistException { } @Override - public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) - throws DatabaseNotExistException, DatabaseNotEmptyException { - try { - if (!cascade && !this.listTables(name).isEmpty()) { - throw new DatabaseNotEmptyException(name); - } - client.delete(resourcePaths.database(name), headers()); - } catch (NoSuchResourceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(name); - } - } + protected void dropDatabaseImpl(String name) { + client.delete(resourcePaths.database(name), headers()); } @Override - public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + protected void alterDatabaseImpl(String name, List changes) throws DatabaseNotExistException { try { Pair, Set> setPropertiesToRemoveKeys = @@ -199,33 +185,25 @@ public void alterDatabase(String name, List changes, boolean ign throw new IllegalStateException("Failed to update properties"); } } catch (NoSuchResourceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(name); - } + throw new DatabaseNotExistException(name); } } @Override - public Table getTable(Identifier identifier) throws TableNotExistException { - throw new UnsupportedOperationException(); + protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { + return null; } @Override - public List listTables(String databaseName) throws DatabaseNotExistException { - return new ArrayList(); + protected List listTablesImpl(String databaseName) { + return Collections.emptyList(); } @Override - public void dropTable(Identifier identifier, boolean ignoreIfNotExists) - throws TableNotExistException { - throw new UnsupportedOperationException(); - } + protected void dropTableImpl(Identifier identifier) {} @Override - public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) - throws TableAlreadyExistException, DatabaseNotExistException { - throw new UnsupportedOperationException(); - } + protected void createTableImpl(Identifier identifier, Schema schema) {} @Override public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) @@ -234,27 +212,11 @@ public void renameTable(Identifier fromTable, Identifier toTable, boolean ignore } @Override - public void alterTable( - Identifier identifier, List changes, boolean ignoreIfNotExists) - throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { - throw new UnsupportedOperationException(); - } + protected void renameTableImpl(Identifier fromTable, Identifier toTable) {} @Override - public void createPartition(Identifier identifier, Map partitionSpec) - throws TableNotExistException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropPartition(Identifier identifier, Map partitions) - throws TableNotExistException, PartitionNotExistException {} - - @Override - public List listPartitions(Identifier identifier) - throws TableNotExistException { - throw new UnsupportedOperationException(); - } + protected void alterTableImpl(Identifier identifier, List changes) + throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException {} @Override public boolean caseSensitive() { 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 9b1582929560..b2d108c31c7b 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 @@ -50,7 +50,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; /** Test for REST Catalog. */ public class RESTCatalogTest { @@ -113,11 +112,11 @@ public void testListDatabases() throws JsonProcessingException { } @Test - public void testCreateDatabase() throws Exception { + public void testCreateDatabaseImpl() throws Exception { String name = MockRESTMessage.databaseName(); CreateDatabaseResponse response = MockRESTMessage.createDatabaseResponse(name); mockResponse(mapper.writeValueAsString(response), 200); - assertDoesNotThrow(() -> restCatalog.createDatabase(name, false, response.getOptions())); + assertDoesNotThrow(() -> restCatalog.createDatabaseImpl(name, response.getOptions())); } @Test @@ -132,85 +131,29 @@ public void testGetDatabase() throws Exception { } @Test - public void testDropDatabase() throws Exception { + public void testDropDatabaseImpl() throws Exception { String name = MockRESTMessage.databaseName(); mockResponse("", 200); - assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, true)); - verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(true)); - verify(mockRestCatalog, times(0)).listTables(eq(name)); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabaseImpl(name)); + verify(mockRestCatalog, times(1)).dropDatabaseImpl(eq(name)); } @Test - public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsFalse() throws Exception { - String name = MockRESTMessage.databaseName(); - ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); - mockResponse(mapper.writeValueAsString(response), 404); - assertThrows( - Catalog.DatabaseNotExistException.class, - () -> mockRestCatalog.dropDatabase(name, false, true)); - } - - @Test - public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsTrue() throws Exception { - String name = MockRESTMessage.databaseName(); - ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); - mockResponse(mapper.writeValueAsString(response), 404); - assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, true, true)); - verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(true), eq(true)); - verify(mockRestCatalog, times(0)).listTables(eq(name)); - } - - @Test - public void testDropDatabaseWhenCascadeIsFalseAndNoTables() throws Exception { - String name = MockRESTMessage.databaseName(); - boolean cascade = false; - mockResponse("", 200); - when(mockRestCatalog.listTables(name)).thenReturn(new ArrayList<>()); - assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, cascade)); - verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); - verify(mockRestCatalog, times(1)).listTables(eq(name)); - } - - @Test - public void testDropDatabaseWhenCascadeIsFalseAndTablesExist() throws Exception { - String name = MockRESTMessage.databaseName(); - boolean cascade = false; - mockResponse("", 200); - List tables = new ArrayList<>(); - tables.add("t1"); - when(mockRestCatalog.listTables(name)).thenReturn(tables); - assertThrows( - Catalog.DatabaseNotEmptyException.class, - () -> mockRestCatalog.dropDatabase(name, false, cascade)); - verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); - verify(mockRestCatalog, times(1)).listTables(eq(name)); - } - - @Test - public void testAlterDatabase() throws Exception { + public void testAlterDatabaseImpl() throws Exception { String name = MockRESTMessage.databaseName(); AlterDatabaseResponse response = MockRESTMessage.alterDatabaseResponse(); mockResponse(mapper.writeValueAsString(response), 200); - assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); + assertDoesNotThrow(() -> mockRestCatalog.alterDatabaseImpl(name, new ArrayList<>())); } @Test - public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsFalse() - throws Exception { + public void testAlterDatabaseImplWhenDatabaseNotExist() throws Exception { String name = MockRESTMessage.databaseName(); ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); mockResponse(mapper.writeValueAsString(response), 404); assertThrows( Catalog.DatabaseNotExistException.class, - () -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), false)); - } - - @Test - public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsTrue() throws Exception { - String name = MockRESTMessage.databaseName(); - ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); - mockResponse(mapper.writeValueAsString(response), 404); - assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); + () -> mockRestCatalog.alterDatabaseImpl(name, new ArrayList<>())); } private void mockResponse(String mockResponse, int httpCode) { From 999d4aabaa0b89a8e09e9f3b717bf8a5ee75a3b8 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 11:34:04 +0800 Subject: [PATCH 02/23] change RESTCatalog extend abstract catalog --- .../org/apache/paimon/rest/RESTCatalog.java | 80 ++++++++++++------- .../paimon/rest/RESTCatalogFactory.java | 8 +- .../apache/paimon/rest/RESTCatalogTest.java | 7 +- 3 files changed, 62 insertions(+), 33 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 efc0074a1ce9..587fc7475c40 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 @@ -23,7 +23,6 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.rest.auth.AuthSession; import org.apache.paimon.rest.auth.CredentialsProvider; @@ -73,32 +72,28 @@ public class RESTCatalog extends AbstractCatalog { private volatile ScheduledExecutorService refreshExecutor = null; public RESTCatalog(Options options) { - this(null, options); + this(options, getClient(options), getCredentialsProvider(options)); } - public RESTCatalog(FileIO fileIO, Options options) { - super(fileIO, options); - if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { - throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); - } - String uri = options.get(RESTCatalogOptions.URI); - Optional connectTimeout = - options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); - Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); - Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); - HttpClientOptions httpClientOptions = - new HttpClientOptions( - uri, - connectTimeout, - readTimeout, - OBJECT_MAPPER, - threadPoolSize, - DefaultErrorHandler.getInstance()); - this.client = new HttpClient(httpClientOptions); - this.baseHeader = configHeaders(options.toMap()); - CredentialsProvider credentialsProvider = - CredentialsProviderFactory.createCredentialsProvider( - options, RESTCatalog.class.getClassLoader()); + public RESTCatalog( + Options options, RESTClient client, CredentialsProvider credentialsProvider) { + this( + client, + credentialsProvider, + new Options( + fetchOptionsFromServer( + client, + RESTUtil.merge( + configHeaders(options.toMap()), + credentialsProvider.authHeader()), + options.toMap()))); + } + + public RESTCatalog( + RESTClient client, CredentialsProvider credentialsProvider, Options optionsWithServer) { + super(getFileIOFromOptions(optionsWithServer), optionsWithServer); + this.client = client; + this.baseHeader = configHeaders(optionsWithServer.toMap()); if (credentialsProvider.keepRefreshed()) { this.catalogAuth = AuthSession.fromRefreshCredentialsProvider( @@ -107,9 +102,7 @@ public RESTCatalog(FileIO fileIO, Options options) { } else { this.catalogAuth = new AuthSession(this.baseHeader, credentialsProvider); } - Map initHeaders = - RESTUtil.merge(configHeaders(options.toMap()), this.catalogAuth.getHeaders()); - this.options = new Options(fetchOptionsFromServer(initHeaders, options.toMap())); + this.options = optionsWithServer; this.resourcePaths = ResourcePaths.forCatalogProperties( this.options.get(RESTCatalogInternalOptions.PREFIX)); @@ -234,13 +227,40 @@ public void close() throws Exception { } @VisibleForTesting - Map fetchOptionsFromServer( - Map headers, Map clientProperties) { + static Map fetchOptionsFromServer( + RESTClient client, Map headers, Map clientProperties) { ConfigResponse response = client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers); return response.merge(clientProperties); } + @VisibleForTesting + static RESTClient getClient(Options options) { + String uri = options.get(RESTCatalogOptions.URI); + Optional connectTimeout = + options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); + Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); + Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + uri, + connectTimeout, + readTimeout, + OBJECT_MAPPER, + threadPoolSize, + DefaultErrorHandler.getInstance()); + return new HttpClient(httpClientOptions); + } + + private static FileIO getFileIOFromOptions(Options options) { + return null; + } + + private static CredentialsProvider getCredentialsProvider(Options options) { + return CredentialsProviderFactory.createCredentialsProvider( + options, RESTCatalog.class.getClassLoader()); + } + private static Map configHeaders(Map properties) { return RESTUtil.extractPrefixMap(properties, "header."); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index a5c773cb4bd5..0d8a0fcd9014 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -21,6 +21,8 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; /** Factory to create {@link RESTCatalog}. */ public class RESTCatalogFactory implements CatalogFactory { @@ -33,6 +35,10 @@ public String identifier() { @Override public Catalog create(CatalogContext context) { - return new RESTCatalog(context.options()); + Options options = context.options(); + if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { + throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); + } + return new RESTCatalog(options); } } 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 b2d108c31c7b..7bacc9346632 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 @@ -58,13 +58,14 @@ public class RESTCatalogTest { private MockWebServer mockWebServer; private RESTCatalog restCatalog; private RESTCatalog mockRestCatalog; + private Options options; @Before public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); String baseUrl = mockWebServer.url("").toString(); - Options options = new Options(); + options = new Options(); options.set(RESTCatalogOptions.URI, baseUrl); String initToken = "init_token"; options.set(RESTCatalogOptions.TOKEN, initToken); @@ -97,7 +98,9 @@ public void testGetConfig() { String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); mockResponse(mockResponse, 200); Map header = new HashMap<>(); - Map response = restCatalog.fetchOptionsFromServer(header, new HashMap<>()); + RESTClient client = RESTCatalog.getClient(options); + Map response = + RESTCatalog.fetchOptionsFromServer(client, header, new HashMap<>()); assertEquals(value, response.get(key)); } From 0293ccb6f9eca3111a412f9aac152ac0814c2c92 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 14:55:42 +0800 Subject: [PATCH 03/23] support get FileIO --- .../org/apache/paimon/rest/RESTCatalog.java | 45 +++++++++++++------ .../paimon/rest/RESTCatalogFactory.java | 2 +- .../apache/paimon/rest/RESTCatalogTest.java | 23 +++++++--- 3 files changed, 51 insertions(+), 19 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index 587fc7475c40..27e1862fc0e0 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 @@ -19,10 +19,13 @@ package org.apache.paimon.rest; import org.apache.paimon.catalog.AbstractCatalog; +import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.rest.auth.AuthSession; import org.apache.paimon.rest.auth.CredentialsProvider; @@ -45,6 +48,8 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.UncheckedIOException; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; @@ -66,41 +71,44 @@ public class RESTCatalog extends AbstractCatalog { private final RESTClient client; private final ResourcePaths resourcePaths; private final Options options; - private final Map baseHeader; private final AuthSession catalogAuth; private volatile ScheduledExecutorService refreshExecutor = null; - public RESTCatalog(Options options) { - this(options, getClient(options), getCredentialsProvider(options)); + public RESTCatalog(CatalogContext context) { + this(context, getClient(context.options()), getCredentialsProvider(context.options())); } public RESTCatalog( - Options options, RESTClient client, CredentialsProvider credentialsProvider) { + CatalogContext context, RESTClient client, CredentialsProvider credentialsProvider) { this( + context, client, credentialsProvider, new Options( fetchOptionsFromServer( client, RESTUtil.merge( - configHeaders(options.toMap()), + configHeaders(context.options().toMap()), credentialsProvider.authHeader()), - options.toMap()))); + context.options().toMap()))); } public RESTCatalog( - RESTClient client, CredentialsProvider credentialsProvider, Options optionsWithServer) { - super(getFileIOFromOptions(optionsWithServer), optionsWithServer); + CatalogContext context, + RESTClient client, + CredentialsProvider credentialsProvider, + Options optionsWithServer) { + super(getFileIOFromOptions(context, optionsWithServer), optionsWithServer); this.client = client; - this.baseHeader = configHeaders(optionsWithServer.toMap()); + Map baseHeader = configHeaders(optionsWithServer.toMap()); if (credentialsProvider.keepRefreshed()) { this.catalogAuth = AuthSession.fromRefreshCredentialsProvider( - tokenRefreshExecutor(), this.baseHeader, credentialsProvider); + tokenRefreshExecutor(), baseHeader, credentialsProvider); } else { - this.catalogAuth = new AuthSession(this.baseHeader, credentialsProvider); + this.catalogAuth = new AuthSession(baseHeader, credentialsProvider); } this.options = optionsWithServer; this.resourcePaths = @@ -252,8 +260,19 @@ static RESTClient getClient(Options options) { return new HttpClient(httpClientOptions); } - private static FileIO getFileIOFromOptions(Options options) { - return null; + // todo: whether it's ok + private static FileIO getFileIOFromOptions(CatalogContext context, Options options) { + String warehouseStr = options.get(CatalogOptions.WAREHOUSE); + Path warehousePath = new Path(warehouseStr); + FileIO fileIO; + CatalogContext contextWithNewOptions = + CatalogContext.create(options, context.preferIO(), context.fallbackIO()); + try { + fileIO = FileIO.get(warehousePath, contextWithNewOptions); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return fileIO; } private static CredentialsProvider getCredentialsProvider(Options options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index 0d8a0fcd9014..b03c9ca4248d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -39,6 +39,6 @@ public Catalog create(CatalogContext context) { if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); } - return new RESTCatalog(options); + return new RESTCatalog(context); } } 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 7bacc9346632..7bb608d5daa3 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 @@ -19,6 +19,7 @@ package org.apache.paimon.rest; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Database; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; @@ -35,7 +36,9 @@ import okhttp3.mockwebserver.MockWebServer; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import java.io.IOException; import java.util.ArrayList; @@ -59,6 +62,9 @@ public class RESTCatalogTest { private RESTCatalog restCatalog; private RESTCatalog mockRestCatalog; private Options options; + private CatalogContext context; + private String warehouseStr; + @Rule public TemporaryFolder folder = new TemporaryFolder(); @Before public void setUp() throws IOException { @@ -70,12 +76,17 @@ public void setUp() throws IOException { String initToken = "init_token"; options.set(RESTCatalogOptions.TOKEN, initToken); options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); + context = CatalogContext.create(options); + warehouseStr = folder.getRoot().getPath(); String mockResponse = String.format( - "{\"defaults\": {\"%s\": \"%s\"}}", - RESTCatalogInternalOptions.PREFIX.key(), "prefix"); + "{\"defaults\": {\"%s\": \"%s\", \"%s\": \"%s\"}}", + RESTCatalogInternalOptions.PREFIX.key(), + "prefix", + CatalogOptions.WAREHOUSE.key(), + warehouseStr); mockResponse(mockResponse, 200); - restCatalog = new RESTCatalog(options); + restCatalog = new RESTCatalog(context); mockRestCatalog = spy(restCatalog); } @@ -87,8 +98,10 @@ public void tearDown() throws IOException { @Test public void testInitFailWhenDefineWarehouse() { Options options = new Options(); - options.set(CatalogOptions.WAREHOUSE, "/a/b/c"); - assertThrows(IllegalArgumentException.class, () -> new RESTCatalog(options)); + options.set(CatalogOptions.WAREHOUSE, warehouseStr); + assertThrows( + IllegalArgumentException.class, + () -> new RESTCatalog(CatalogContext.create(options))); } @Test From 68589a1b94c20cfb52919c1d8063f9c67dd84225 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 15:39:40 +0800 Subject: [PATCH 04/23] add api support list tables and get table --- .../org/apache/paimon/rest/RESTCatalog.java | 57 +++++++++---------- .../org/apache/paimon/rest/ResourcePaths.java | 19 +++++++ .../rest/responses/GetTableResponse.java | 57 +++++++++++++++++++ .../rest/responses/ListDatabasesResponse.java | 6 +- ...abaseName.java => ListTablesResponse.java} | 23 ++++---- .../apache/paimon/rest/MockRESTMessage.java | 6 +- .../open/api/RESTCatalogController.java | 3 +- 7 files changed, 122 insertions(+), 49 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java rename paimon-core/src/main/java/org/apache/paimon/rest/responses/{DatabaseName.java => ListTablesResponse.java} (70%) 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 27e1862fc0e0..d0b0fc41056b 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 @@ -36,9 +36,10 @@ import org.apache.paimon.rest.responses.AlterDatabaseResponse; 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.GetTableResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; +import org.apache.paimon.rest.responses.ListTablesResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.TableSchema; @@ -52,13 +53,11 @@ import java.io.UncheckedIOException; import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; -import java.util.stream.Collectors; import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; @@ -121,24 +120,12 @@ public String warehouse() { throw new UnsupportedOperationException(); } - @Override - public Map options() { - return this.options.toMap(); - } - - @Override - public FileIO fileIO() { - throw new UnsupportedOperationException(); - } - @Override public List listDatabases() { ListDatabasesResponse response = client.get(resourcePaths.databases(), ListDatabasesResponse.class, headers()); if (response.getDatabases() != null) { - return response.getDatabases().stream() - .map(DatabaseName::getName) - .collect(Collectors.toList()); + return response.getDatabases(); } return ImmutableList.of(); } @@ -190,28 +177,37 @@ protected void alterDatabaseImpl(String name, List changes) } } - @Override - protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { - return null; - } - @Override protected List listTablesImpl(String databaseName) { - return Collections.emptyList(); + ListTablesResponse response = + client.get(resourcePaths.tables(databaseName), ListTablesResponse.class, headers()); + if (response.getTables() != null) { + return response.getTables(); + } + return ImmutableList.of(); } @Override - protected void dropTableImpl(Identifier identifier) {} + protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { + try { + GetTableResponse response = + client.get( + resourcePaths.table( + identifier.getDatabaseName(), identifier.getTableName()), + GetTableResponse.class, + headers()); + if (response.getSchema() != null) { + return response.getSchema(); + } + } catch (NoSuchResourceException e) { + throw new TableNotExistException(identifier); + } + throw new TableNotExistException(identifier); + } @Override protected void createTableImpl(Identifier identifier, Schema schema) {} - @Override - public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) - throws TableNotExistException, TableAlreadyExistException { - throw new UnsupportedOperationException(); - } - @Override protected void renameTableImpl(Identifier fromTable, Identifier toTable) {} @@ -219,6 +215,9 @@ protected void renameTableImpl(Identifier fromTable, Identifier toTable) {} protected void alterTableImpl(Identifier identifier, List changes) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException {} + @Override + protected void dropTableImpl(Identifier identifier) {} + @Override public boolean caseSensitive() { return options.getOptional(CASE_SENSITIVE).orElse(true); 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 51277454ffb0..567dfea49046 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 @@ -52,4 +52,23 @@ public String databaseProperties(String databaseName) { .add("properties") .toString(); } + + public String tables(String databaseName) { + return SLASH.add("v1") + .add(prefix) + .add("databases") + .add(databaseName) + .add("tables") + .toString(); + } + + public String table(String databaseName, String tableName) { + return SLASH.add("v1") + .add(prefix) + .add("databases") + .add(databaseName) + .add("tables") + .add(tableName) + .toString(); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.java new file mode 100644 index 000000000000..671c50cac5a1 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetTableResponse.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.schema.TableSchema; + +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; + +/** Response for getting table. */ +public class GetTableResponse implements RESTResponse { + + private static final String FIELD_LOCATION = "location"; + private static final String FIELD_SCHEMA = "schema"; + + @JsonProperty(FIELD_LOCATION) + private final String location; + + @JsonProperty(FIELD_SCHEMA) + private final TableSchema schema; + + @JsonCreator + public GetTableResponse( + @JsonProperty(FIELD_LOCATION) String location, + @JsonProperty(FIELD_SCHEMA) TableSchema schema) { + this.location = location; + this.schema = schema; + } + + @JsonGetter(FIELD_LOCATION) + public String getLocation() { + return this.location; + } + + @JsonGetter(FIELD_SCHEMA) + public TableSchema getSchema() { + return this.schema; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java index 38773f354b77..64a17a6be7e6 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 @@ -31,15 +31,15 @@ public class ListDatabasesResponse implements RESTResponse { private static final String FIELD_DATABASES = "databases"; @JsonProperty(FIELD_DATABASES) - private List databases; + private List databases; @JsonCreator - public ListDatabasesResponse(@JsonProperty(FIELD_DATABASES) List databases) { + public ListDatabasesResponse(@JsonProperty(FIELD_DATABASES) List databases) { this.databases = databases; } @JsonGetter(FIELD_DATABASES) - public List getDatabases() { + public List getDatabases() { return this.databases; } } 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/ListTablesResponse.java similarity index 70% rename from paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java rename to paimon-core/src/main/java/org/apache/paimon/rest/responses/ListTablesResponse.java index 9a93b2fd1e3d..bccaa48438e2 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/ListTablesResponse.java @@ -18,27 +18,28 @@ package org.apache.paimon.rest.responses; -import org.apache.paimon.rest.RESTMessage; +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; -/** Class for Database entity. */ -public class DatabaseName implements RESTMessage { +import java.util.List; - private static final String FIELD_NAME = "name"; +/** Response for listing tables. */ +public class ListTablesResponse implements RESTResponse { + private static final String FIELD_TABLES = "tables"; - @JsonProperty(FIELD_NAME) - private String name; + @JsonProperty(FIELD_TABLES) + private List tables; @JsonCreator - public DatabaseName(@JsonProperty(FIELD_NAME) String name) { - this.name = name; + public ListTablesResponse(@JsonProperty(FIELD_TABLES) List tables) { + this.tables = tables; } - @JsonGetter(FIELD_NAME) - public String getName() { - return this.name; + @JsonGetter(FIELD_TABLES) + public List getTables() { + return this.tables; } } 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 821257a0e10e..5a2a4fb8ac65 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 @@ -22,7 +22,6 @@ import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; 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; @@ -63,9 +62,8 @@ public static GetDatabaseResponse getDatabaseResponse(String name) { } public static ListDatabasesResponse listDatabasesResponse(String name) { - DatabaseName databaseName = new DatabaseName(name); - List databaseNameList = new ArrayList<>(); - databaseNameList.add(databaseName); + List databaseNameList = new ArrayList<>(); + databaseNameList.add(name); return new ListDatabasesResponse(databaseNameList); } 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 5331b65d71b6..ea33f195fef2 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 @@ -24,7 +24,6 @@ import org.apache.paimon.rest.responses.AlterDatabaseResponse; 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; @@ -86,7 +85,7 @@ public ConfigResponse getConfig() { }) @GetMapping("/v1/{prefix}/databases") public ListDatabasesResponse listDatabases(@PathVariable String prefix) { - return new ListDatabasesResponse(ImmutableList.of(new DatabaseName("account"))); + return new ListDatabasesResponse(ImmutableList.of("account")); } @Operation( From c24ee6a0b2a9aa847b6563ce13297f3690718146 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 16:04:56 +0800 Subject: [PATCH 05/23] support create table --- .../paimon/catalog/AbstractCatalog.java | 36 +++++++----- .../org/apache/paimon/rest/RESTCatalog.java | 16 ++++- .../rest/requests/CreateTableRequest.java | 58 +++++++++++++++++++ 3 files changed, 95 insertions(+), 15 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index d7447c37dd79..a6ce5859953c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -297,22 +297,29 @@ public void createTable(Identifier identifier, Schema schema, boolean ignoreIfEx } copyTableDefaultOptions(schema.options()); - - switch (Options.fromMap(schema.options()).get(TYPE)) { - case TABLE: - case MATERIALIZED_TABLE: - createTableImpl(identifier, schema); - break; - case OBJECT_TABLE: - createObjectTable(identifier, schema); - break; - case FORMAT_TABLE: - createFormatTable(identifier, schema); - break; + try { + switch (Options.fromMap(schema.options()).get(TYPE)) { + case TABLE: + case MATERIALIZED_TABLE: + createTableImpl(identifier, schema); + break; + case OBJECT_TABLE: + createObjectTable(identifier, schema); + break; + case FORMAT_TABLE: + createFormatTable(identifier, schema); + break; + } + } catch (TableAlreadyExistException e) { + if (ignoreIfExists) { + return; + } + throw e; } } - private void createObjectTable(Identifier identifier, Schema schema) { + private void createObjectTable(Identifier identifier, Schema schema) + throws TableAlreadyExistException { RowType rowType = schema.rowType(); checkArgument( rowType.getFields().isEmpty() @@ -327,7 +334,8 @@ private void createObjectTable(Identifier identifier, Schema schema) { createTableImpl(identifier, schema.copy(ObjectTable.SCHEMA)); } - protected abstract void createTableImpl(Identifier identifier, Schema schema); + protected abstract void createTableImpl(Identifier identifier, Schema schema) + throws TableAlreadyExistException; @Override public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) 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 d0b0fc41056b..d1ba29779178 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,9 +30,11 @@ 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.exceptions.NoSuchResourceException; import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.CreateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; @@ -206,7 +208,19 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } @Override - protected void createTableImpl(Identifier identifier, Schema schema) {} + protected void createTableImpl(Identifier identifier, Schema schema) + throws TableAlreadyExistException { + try { + CreateTableRequest request = new CreateTableRequest(identifier, schema); + client.post( + resourcePaths.tables(identifier.getDatabaseName()), + request, + GetTableResponse.class, + headers()); + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistException(identifier); + } + } @Override protected void renameTableImpl(Identifier fromTable, Identifier toTable) {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java new file mode 100644 index 000000000000..1e152d7f1f56 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.rest.RESTRequest; +import org.apache.paimon.schema.Schema; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +/** Request for creating table. */ +public class CreateTableRequest implements RESTRequest { + + private static final String FIELD_IDENTIFIER = "identifier"; + private static final String FIELD_SCHEMA = "schema"; + + @JsonProperty(FIELD_IDENTIFIER) + private Identifier identifier; + + @JsonProperty(FIELD_SCHEMA) + private Schema schema; + + @JsonCreator + public CreateTableRequest( + @JsonProperty(FIELD_IDENTIFIER) Identifier identifier, + @JsonProperty(FIELD_SCHEMA) Schema schema) { + this.identifier = identifier; + this.schema = schema; + } + + @JsonGetter(FIELD_IDENTIFIER) + public Identifier getIdentifier() { + return identifier; + } + + @JsonGetter(FIELD_SCHEMA) + public Schema getSchema() { + return schema; + } +} From 45ea722d036bd58460d46d8bfcf82bc1727a0fd2 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 16:28:13 +0800 Subject: [PATCH 06/23] add update table and drop table fix test fail --- .../org/apache/paimon/rest/RESTCatalog.java | 25 ++++++- .../rest/requests/UpdateTableRequest.java | 71 +++++++++++++++++++ .../paimon/rest/RESTObjectMapperTest.java | 2 +- 3 files changed, 94 insertions(+), 4 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index d1ba29779178..2ab16cbac127 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 @@ -35,6 +35,7 @@ import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.requests.CreateTableRequest; +import org.apache.paimon.rest.requests.UpdateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; @@ -223,14 +224,32 @@ protected void createTableImpl(Identifier identifier, Schema schema) } @Override - protected void renameTableImpl(Identifier fromTable, Identifier toTable) {} + protected void renameTableImpl(Identifier fromTable, Identifier toTable) { + updateTable(fromTable, toTable, new ArrayList<>()); + } @Override protected void alterTableImpl(Identifier identifier, List changes) - throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException {} + throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { + updateTable(identifier, null, changes); + } + + // todo: how know which exception to throw + private void updateTable(Identifier fromTable, Identifier toTable, List changes) { + UpdateTableRequest request = new UpdateTableRequest(fromTable, toTable, changes); + client.post( + resourcePaths.table(fromTable.getDatabaseName(), fromTable.getTableName()), + request, + GetTableResponse.class, + headers()); + } @Override - protected void dropTableImpl(Identifier identifier) {} + protected void dropTableImpl(Identifier identifier) { + client.delete( + resourcePaths.table(identifier.getDatabaseName(), identifier.getTableName()), + headers()); + } @Override public boolean caseSensitive() { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java new file mode 100644 index 000000000000..ca12a1152455 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.rest.RESTRequest; +import org.apache.paimon.schema.SchemaChange; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** Request for updating table. */ +public class UpdateTableRequest implements RESTRequest { + + private static final String FIELD_FROM_IDENTIFIER = "from"; + private static final String FIELD_TO_IDENTIFIER = "to"; + private static final String FIELD_SCHEMA_CHANGES = "changes"; + + @JsonProperty(FIELD_FROM_IDENTIFIER) + private Identifier fromIdentifier; + + @JsonProperty(FIELD_TO_IDENTIFIER) + private Identifier toIdentifier; + + @JsonProperty(FIELD_SCHEMA_CHANGES) + private List changes; + + @JsonCreator + public UpdateTableRequest( + @JsonProperty(FIELD_FROM_IDENTIFIER) Identifier fromIdentifier, + @JsonProperty(FIELD_TO_IDENTIFIER) Identifier toIdentifier, + @JsonProperty(FIELD_SCHEMA_CHANGES) List changes) { + this.fromIdentifier = fromIdentifier; + this.toIdentifier = toIdentifier; + this.changes = changes; + } + + @JsonGetter(FIELD_FROM_IDENTIFIER) + public Identifier getFromIdentifier() { + return fromIdentifier; + } + + @JsonGetter(FIELD_TO_IDENTIFIER) + public Identifier getToIdentifier() { + return toIdentifier; + } + + @JsonGetter(FIELD_SCHEMA_CHANGES) + public List getChanges() { + return changes; + } +} 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 0e5a71be39c0..1cf382a9f438 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 @@ -103,7 +103,7 @@ public void listDatabaseResponseParseTest() throws Exception { ListDatabasesResponse parseData = mapper.readValue(responseStr, ListDatabasesResponse.class); assertEquals(response.getDatabases().size(), parseData.getDatabases().size()); - assertEquals(name, parseData.getDatabases().get(0).getName()); + assertEquals(name, parseData.getDatabases().get(0)); } @Test From 194a7e9096f4287a733d9fed58dd1a3e7afddc58 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 17:23:44 +0800 Subject: [PATCH 07/23] change RESTCatalog implements Catalog remove change in AbstractCatalog remove change in AbstractCatalog --- .../paimon/catalog/AbstractCatalog.java | 36 ++- .../org/apache/paimon/rest/RESTCatalog.java | 238 ++++++++++-------- .../apache/paimon/rest/RESTCatalogTest.java | 98 ++++++-- .../open/api/RESTCatalogController.java | 137 +++++++++- 4 files changed, 360 insertions(+), 149 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index a6ce5859953c..d7447c37dd79 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -297,29 +297,22 @@ public void createTable(Identifier identifier, Schema schema, boolean ignoreIfEx } copyTableDefaultOptions(schema.options()); - try { - switch (Options.fromMap(schema.options()).get(TYPE)) { - case TABLE: - case MATERIALIZED_TABLE: - createTableImpl(identifier, schema); - break; - case OBJECT_TABLE: - createObjectTable(identifier, schema); - break; - case FORMAT_TABLE: - createFormatTable(identifier, schema); - break; - } - } catch (TableAlreadyExistException e) { - if (ignoreIfExists) { - return; - } - throw e; + + switch (Options.fromMap(schema.options()).get(TYPE)) { + case TABLE: + case MATERIALIZED_TABLE: + createTableImpl(identifier, schema); + break; + case OBJECT_TABLE: + createObjectTable(identifier, schema); + break; + case FORMAT_TABLE: + createFormatTable(identifier, schema); + break; } } - private void createObjectTable(Identifier identifier, Schema schema) - throws TableAlreadyExistException { + private void createObjectTable(Identifier identifier, Schema schema) { RowType rowType = schema.rowType(); checkArgument( rowType.getFields().isEmpty() @@ -334,8 +327,7 @@ private void createObjectTable(Identifier identifier, Schema schema) createTableImpl(identifier, schema.copy(ObjectTable.SCHEMA)); } - protected abstract void createTableImpl(Identifier identifier, Schema schema) - throws TableAlreadyExistException; + protected abstract void createTableImpl(Identifier identifier, Schema schema); @Override public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) 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 2ab16cbac127..c70a9c7d69d1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -18,13 +18,14 @@ package org.apache.paimon.rest; -import org.apache.paimon.catalog.AbstractCatalog; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.rest.auth.AuthSession; @@ -46,6 +47,7 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.Table; import org.apache.paimon.utils.Pair; import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; @@ -66,61 +68,92 @@ import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; /** A catalog implementation for REST. */ -public class RESTCatalog extends AbstractCatalog { +public class RESTCatalog implements Catalog { private static final ObjectMapper OBJECT_MAPPER = RESTObjectMapper.create(); private final RESTClient client; private final ResourcePaths resourcePaths; - private final Options options; + private final Map baseHeader; private final AuthSession catalogAuth; + private final CatalogContext context; + private final FileIO fileIO; private volatile ScheduledExecutorService refreshExecutor = null; - public RESTCatalog(CatalogContext context) { - this(context, getClient(context.options()), getCredentialsProvider(context.options())); - } - - public RESTCatalog( - CatalogContext context, RESTClient client, CredentialsProvider credentialsProvider) { - this( - context, - client, - credentialsProvider, - new Options( - fetchOptionsFromServer( - client, - RESTUtil.merge( - configHeaders(context.options().toMap()), - credentialsProvider.authHeader()), - context.options().toMap()))); - } - - public RESTCatalog( - CatalogContext context, - RESTClient client, - CredentialsProvider credentialsProvider, - Options optionsWithServer) { - super(getFileIOFromOptions(context, optionsWithServer), optionsWithServer); - this.client = client; - Map baseHeader = configHeaders(optionsWithServer.toMap()); + public RESTCatalog(CatalogContext catalogContext) { + Options catalogOptions = catalogContext.options(); + if (catalogOptions.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { + throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); + } + String uri = catalogOptions.get(RESTCatalogOptions.URI); + Optional connectTimeout = + catalogOptions.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); + Optional readTimeout = + catalogOptions.getOptional(RESTCatalogOptions.READ_TIMEOUT); + Integer threadPoolSize = catalogOptions.get(RESTCatalogOptions.THREAD_POOL_SIZE); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + uri, + connectTimeout, + readTimeout, + OBJECT_MAPPER, + threadPoolSize, + DefaultErrorHandler.getInstance()); + this.client = new HttpClient(httpClientOptions); + this.baseHeader = configHeaders(catalogOptions.toMap()); + CredentialsProvider credentialsProvider = + CredentialsProviderFactory.createCredentialsProvider( + catalogOptions, RESTCatalog.class.getClassLoader()); if (credentialsProvider.keepRefreshed()) { this.catalogAuth = AuthSession.fromRefreshCredentialsProvider( - tokenRefreshExecutor(), baseHeader, credentialsProvider); + tokenRefreshExecutor(), this.baseHeader, credentialsProvider); } else { - this.catalogAuth = new AuthSession(baseHeader, credentialsProvider); + this.catalogAuth = new AuthSession(this.baseHeader, credentialsProvider); } - this.options = optionsWithServer; + Map initHeaders = + RESTUtil.merge( + configHeaders(catalogOptions.toMap()), this.catalogAuth.getHeaders()); + Options options = new Options(fetchOptionsFromServer(initHeaders, initHeaders)); + this.context = + CatalogContext.create( + options, catalogContext.preferIO(), catalogContext.fallbackIO()); this.resourcePaths = - ResourcePaths.forCatalogProperties( - this.options.get(RESTCatalogInternalOptions.PREFIX)); + ResourcePaths.forCatalogProperties(options.get(RESTCatalogInternalOptions.PREFIX)); + this.fileIO = getFileIOFromOptions(catalogContext); + } + + // todo: whether it's ok + private static FileIO getFileIOFromOptions(CatalogContext context) { + Options options = context.options(); + String warehouseStr = options.get(CatalogOptions.WAREHOUSE); + Path warehousePath = new Path(warehouseStr); + FileIO fileIO; + CatalogContext contextWithNewOptions = + CatalogContext.create(options, context.preferIO(), context.fallbackIO()); + try { + fileIO = FileIO.get(warehousePath, contextWithNewOptions); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return fileIO; } @Override public String warehouse() { - throw new UnsupportedOperationException(); + return context.options().get(CatalogOptions.WAREHOUSE); + } + + @Override + public Map options() { + return context.options().toMap(); + } + + @Override + public FileIO fileIO() { + return this.fileIO; } @Override @@ -134,13 +167,21 @@ public List listDatabases() { } @Override - protected void createDatabaseImpl(String name, Map properties) { + public void createDatabase(String name, boolean ignoreIfExists, Map properties) + throws DatabaseAlreadyExistException { CreateDatabaseRequest request = new CreateDatabaseRequest(name, properties); - client.post(resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); + try { + client.post( + resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(name); + } + } } @Override - protected Database getDatabaseImpl(String name) throws DatabaseNotExistException { + public Database getDatabase(String name) throws DatabaseNotExistException { try { GetDatabaseResponse response = client.get(resourcePaths.database(name), GetDatabaseResponse.class, headers()); @@ -152,12 +193,22 @@ protected Database getDatabaseImpl(String name) throws DatabaseNotExistException } @Override - protected void dropDatabaseImpl(String name) { - client.delete(resourcePaths.database(name), headers()); + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException { + try { + if (!cascade && !this.listTables(name).isEmpty()) { + throw new DatabaseNotEmptyException(name); + } + client.delete(resourcePaths.database(name), headers()); + } catch (NoSuchResourceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(name); + } + } } @Override - protected void alterDatabaseImpl(String name, List changes) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { try { Pair, Set> setPropertiesToRemoveKeys = @@ -176,12 +227,14 @@ protected void alterDatabaseImpl(String name, List changes) throw new IllegalStateException("Failed to update properties"); } } catch (NoSuchResourceException e) { - throw new DatabaseNotExistException(name); + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(name); + } } } @Override - protected List listTablesImpl(String databaseName) { + public List listTables(String databaseName) throws DatabaseNotExistException { ListTablesResponse response = client.get(resourcePaths.tables(databaseName), ListTablesResponse.class, headers()); if (response.getTables() != null) { @@ -191,6 +244,10 @@ protected List listTablesImpl(String databaseName) { } @Override + public Table getTable(Identifier identifier) throws TableNotExistException { + throw new UnsupportedOperationException(); + } + protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { try { GetTableResponse response = @@ -209,8 +266,8 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } @Override - protected void createTableImpl(Identifier identifier, Schema schema) - throws TableAlreadyExistException { + public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException { try { CreateTableRequest request = new CreateTableRequest(identifier, schema); client.post( @@ -224,36 +281,45 @@ protected void createTableImpl(Identifier identifier, Schema schema) } @Override - protected void renameTableImpl(Identifier fromTable, Identifier toTable) { + public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException { updateTable(fromTable, toTable, new ArrayList<>()); } @Override - protected void alterTableImpl(Identifier identifier, List changes) + public void alterTable( + Identifier identifier, List changes, boolean ignoreIfNotExists) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { updateTable(identifier, null, changes); } - // todo: how know which exception to throw - private void updateTable(Identifier fromTable, Identifier toTable, List changes) { - UpdateTableRequest request = new UpdateTableRequest(fromTable, toTable, changes); - client.post( - resourcePaths.table(fromTable.getDatabaseName(), fromTable.getTableName()), - request, - GetTableResponse.class, - headers()); - } - @Override - protected void dropTableImpl(Identifier identifier) { + public void dropTable(Identifier identifier, boolean ignoreIfNotExists) + throws TableNotExistException { client.delete( resourcePaths.table(identifier.getDatabaseName(), identifier.getTableName()), headers()); } + @Override + public void createPartition(Identifier identifier, Map partitionSpec) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition(Identifier identifier, Map partitions) + throws TableNotExistException, PartitionNotExistException {} + + @Override + public List listPartitions(Identifier identifier) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + @Override public boolean caseSensitive() { - return options.getOptional(CASE_SENSITIVE).orElse(true); + return context.options().getOptional(CASE_SENSITIVE).orElse(true); } @Override @@ -267,51 +333,13 @@ public void close() throws Exception { } @VisibleForTesting - static Map fetchOptionsFromServer( - RESTClient client, Map headers, Map clientProperties) { + Map fetchOptionsFromServer( + Map headers, Map clientProperties) { ConfigResponse response = client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers); return response.merge(clientProperties); } - @VisibleForTesting - static RESTClient getClient(Options options) { - String uri = options.get(RESTCatalogOptions.URI); - Optional connectTimeout = - options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); - Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); - Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); - HttpClientOptions httpClientOptions = - new HttpClientOptions( - uri, - connectTimeout, - readTimeout, - OBJECT_MAPPER, - threadPoolSize, - DefaultErrorHandler.getInstance()); - return new HttpClient(httpClientOptions); - } - - // todo: whether it's ok - private static FileIO getFileIOFromOptions(CatalogContext context, Options options) { - String warehouseStr = options.get(CatalogOptions.WAREHOUSE); - Path warehousePath = new Path(warehouseStr); - FileIO fileIO; - CatalogContext contextWithNewOptions = - CatalogContext.create(options, context.preferIO(), context.fallbackIO()); - try { - fileIO = FileIO.get(warehousePath, contextWithNewOptions); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - return fileIO; - } - - private static CredentialsProvider getCredentialsProvider(Options options) { - return CredentialsProviderFactory.createCredentialsProvider( - options, RESTCatalog.class.getClassLoader()); - } - private static Map configHeaders(Map properties) { return RESTUtil.extractPrefixMap(properties, "header."); } @@ -320,6 +348,16 @@ private Map headers() { return catalogAuth.getHeaders(); } + // todo: how know which exception to throw + private void updateTable(Identifier fromTable, Identifier toTable, List changes) { + UpdateTableRequest request = new UpdateTableRequest(fromTable, toTable, changes); + client.post( + resourcePaths.table(fromTable.getDatabaseName(), fromTable.getTableName()), + request, + GetTableResponse.class, + headers()); + } + private ScheduledExecutorService tokenRefreshExecutor() { if (refreshExecutor == null) { synchronized (this) { 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 7bb608d5daa3..0bd5e8e42e8f 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 @@ -36,9 +36,7 @@ import okhttp3.mockwebserver.MockWebServer; import org.junit.After; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import java.io.IOException; import java.util.ArrayList; @@ -53,6 +51,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** Test for REST Catalog. */ public class RESTCatalogTest { @@ -61,31 +60,24 @@ public class RESTCatalogTest { private MockWebServer mockWebServer; private RESTCatalog restCatalog; private RESTCatalog mockRestCatalog; - private Options options; private CatalogContext context; - private String warehouseStr; - @Rule public TemporaryFolder folder = new TemporaryFolder(); @Before public void setUp() throws IOException { mockWebServer = new MockWebServer(); mockWebServer.start(); String baseUrl = mockWebServer.url("").toString(); - options = new Options(); + Options options = new Options(); options.set(RESTCatalogOptions.URI, baseUrl); String initToken = "init_token"; options.set(RESTCatalogOptions.TOKEN, initToken); options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); - context = CatalogContext.create(options); - warehouseStr = folder.getRoot().getPath(); String mockResponse = String.format( - "{\"defaults\": {\"%s\": \"%s\", \"%s\": \"%s\"}}", - RESTCatalogInternalOptions.PREFIX.key(), - "prefix", - CatalogOptions.WAREHOUSE.key(), - warehouseStr); + "{\"defaults\": {\"%s\": \"%s\"}}", + RESTCatalogInternalOptions.PREFIX.key(), "prefix"); mockResponse(mockResponse, 200); + context = CatalogContext.create(options); restCatalog = new RESTCatalog(context); mockRestCatalog = spy(restCatalog); } @@ -98,7 +90,7 @@ public void tearDown() throws IOException { @Test public void testInitFailWhenDefineWarehouse() { Options options = new Options(); - options.set(CatalogOptions.WAREHOUSE, warehouseStr); + options.set(CatalogOptions.WAREHOUSE, "/a/b/c"); assertThrows( IllegalArgumentException.class, () -> new RESTCatalog(CatalogContext.create(options))); @@ -111,9 +103,7 @@ public void testGetConfig() { String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); mockResponse(mockResponse, 200); Map header = new HashMap<>(); - RESTClient client = RESTCatalog.getClient(options); - Map response = - RESTCatalog.fetchOptionsFromServer(client, header, new HashMap<>()); + Map response = restCatalog.fetchOptionsFromServer(header, new HashMap<>()); assertEquals(value, response.get(key)); } @@ -128,11 +118,11 @@ public void testListDatabases() throws JsonProcessingException { } @Test - public void testCreateDatabaseImpl() throws Exception { + public void testCreateDatabase() throws Exception { String name = MockRESTMessage.databaseName(); CreateDatabaseResponse response = MockRESTMessage.createDatabaseResponse(name); mockResponse(mapper.writeValueAsString(response), 200); - assertDoesNotThrow(() -> restCatalog.createDatabaseImpl(name, response.getOptions())); + assertDoesNotThrow(() -> restCatalog.createDatabase(name, false, response.getOptions())); } @Test @@ -147,29 +137,85 @@ public void testGetDatabase() throws Exception { } @Test - public void testDropDatabaseImpl() throws Exception { + public void testDropDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + mockResponse("", 200); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, true)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(true)); + verify(mockRestCatalog, times(0)).listTables(eq(name)); + } + + @Test + public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsFalse() throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertThrows( + Catalog.DatabaseNotExistException.class, + () -> mockRestCatalog.dropDatabase(name, false, true)); + } + + @Test + public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsTrue() throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, true, true)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(true), eq(true)); + verify(mockRestCatalog, times(0)).listTables(eq(name)); + } + + @Test + public void testDropDatabaseWhenCascadeIsFalseAndNoTables() throws Exception { String name = MockRESTMessage.databaseName(); + boolean cascade = false; mockResponse("", 200); - assertDoesNotThrow(() -> mockRestCatalog.dropDatabaseImpl(name)); - verify(mockRestCatalog, times(1)).dropDatabaseImpl(eq(name)); + when(mockRestCatalog.listTables(name)).thenReturn(new ArrayList<>()); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, cascade)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); + verify(mockRestCatalog, times(1)).listTables(eq(name)); } @Test - public void testAlterDatabaseImpl() throws Exception { + public void testDropDatabaseWhenCascadeIsFalseAndTablesExist() throws Exception { + String name = MockRESTMessage.databaseName(); + boolean cascade = false; + mockResponse("", 200); + List tables = new ArrayList<>(); + tables.add("t1"); + when(mockRestCatalog.listTables(name)).thenReturn(tables); + assertThrows( + Catalog.DatabaseNotEmptyException.class, + () -> mockRestCatalog.dropDatabase(name, false, cascade)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); + verify(mockRestCatalog, times(1)).listTables(eq(name)); + } + + @Test + public void testAlterDatabase() throws Exception { String name = MockRESTMessage.databaseName(); AlterDatabaseResponse response = MockRESTMessage.alterDatabaseResponse(); mockResponse(mapper.writeValueAsString(response), 200); - assertDoesNotThrow(() -> mockRestCatalog.alterDatabaseImpl(name, new ArrayList<>())); + assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); } @Test - public void testAlterDatabaseImplWhenDatabaseNotExist() throws Exception { + public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsFalse() + throws Exception { String name = MockRESTMessage.databaseName(); ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); mockResponse(mapper.writeValueAsString(response), 404); assertThrows( Catalog.DatabaseNotExistException.class, - () -> mockRestCatalog.alterDatabaseImpl(name, new ArrayList<>())); + () -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), false)); + } + + @Test + public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsTrue() throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); } private void mockResponse(String mockResponse, int httpCode) { 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 ea33f195fef2..21a5ec7a5fec 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 @@ -21,12 +21,17 @@ import org.apache.paimon.rest.ResourcePaths; import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.CreateTableRequest; +import org.apache.paimon.rest.requests.UpdateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.GetTableResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; +import org.apache.paimon.rest.responses.ListTablesResponse; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; @@ -47,7 +52,7 @@ import java.util.HashMap; import java.util.Map; -/** * RESTCatalog management APIs. */ +/** RESTCatalog management APIs. */ @CrossOrigin(origins = "http://localhost:8081") @RestController public class RESTCatalogController { @@ -180,4 +185,134 @@ public AlterDatabaseResponse alterDatabase( Lists.newArrayList("add"), Lists.newArrayList("missing")); } + + @Operation( + summary = "List tables", + tags = {"table"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = {@Content(schema = @Schema(implementation = ListTablesResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping("/v1/{prefix}/databases/{database}") + public ListTablesResponse listTables( + @PathVariable String prefix, @PathVariable String database) { + return new ListTablesResponse(ImmutableList.of("user")); + } + + @Operation( + summary = "Get table", + tags = {"table"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = {@Content(schema = @Schema(implementation = ListTablesResponse.class))}), + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping("/v1/{prefix}/databases/{database}/tables/{table}") + public GetTableResponse getTable( + @PathVariable String prefix, + @PathVariable String database, + @PathVariable String table) { + return new GetTableResponse( + "location", + new TableSchema( + 1, + 1, + ImmutableList.of(), + 1, + ImmutableList.of(), + ImmutableList.of(), + new HashMap<>(), + "comment", + 1L)); + } + + @Operation( + summary = "Create table", + tags = {"table"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = {@Content(schema = @Schema(implementation = GetTableResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/v1/{prefix}/databases/{database}/tables") + public GetTableResponse createTable( + @PathVariable String prefix, + @PathVariable String database, + @RequestBody CreateTableRequest request) { + return new GetTableResponse( + "location", + new TableSchema( + 1, + 1, + ImmutableList.of(), + 1, + ImmutableList.of(), + ImmutableList.of(), + new HashMap<>(), + "comment", + 1L)); + } + + @Operation( + summary = "Update table", + tags = {"table"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = {@Content(schema = @Schema(implementation = GetTableResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/v1/{prefix}/databases/{database}/tables/table") + public GetTableResponse updateTable( + @PathVariable String prefix, + @PathVariable String database, + @PathVariable String table, + @RequestBody UpdateTableRequest request) { + return new GetTableResponse( + "location", + new TableSchema( + 1, + 1, + ImmutableList.of(), + 1, + ImmutableList.of(), + ImmutableList.of(), + new HashMap<>(), + "comment", + 1L)); + } + + @Operation( + summary = "Update table", + tags = {"table"}) + @ApiResponses({ + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @DeleteMapping("/v1/{prefix}/databases/{database}/tables/table") + public void dropTable( + @PathVariable String prefix, + @PathVariable String database, + @PathVariable String table) {} } From 1aee73d5b576cf7c9a4908e8fd0b80f327b9d7d2 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 18 Dec 2024 18:13:02 +0800 Subject: [PATCH 08/23] support get table follow Abstract Catalog --- .../org/apache/paimon/rest/RESTCatalog.java | 141 +++++++++++++++++- .../apache/paimon/rest/MockRESTMessage.java | 9 ++ .../apache/paimon/rest/RESTCatalogTest.java | 25 ++-- 3 files changed, 164 insertions(+), 11 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 c70a9c7d69d1..ef073420108b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -18,14 +18,21 @@ package org.apache.paimon.rest; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.TableType; +import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; +import org.apache.paimon.factories.FactoryUtil; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.operation.Lock; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.rest.auth.AuthSession; @@ -47,8 +54,14 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.CatalogEnvironment; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.Table; +import org.apache.paimon.table.object.ObjectTable; +import org.apache.paimon.table.system.SystemTableLoader; import org.apache.paimon.utils.Pair; +import org.apache.paimon.utils.Preconditions; import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; @@ -58,13 +71,18 @@ import java.io.UncheckedIOException; import java.time.Duration; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Supplier; import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; +import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; +import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; +import static org.apache.paimon.utils.Preconditions.checkNotNull; import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; /** A catalog implementation for REST. */ @@ -122,7 +140,7 @@ public RESTCatalog(CatalogContext catalogContext) { options, catalogContext.preferIO(), catalogContext.fallbackIO()); this.resourcePaths = ResourcePaths.forCatalogProperties(options.get(RESTCatalogInternalOptions.PREFIX)); - this.fileIO = getFileIOFromOptions(catalogContext); + this.fileIO = getFileIOFromOptions(context); } // todo: whether it's ok @@ -245,7 +263,13 @@ public List listTables(String databaseName) throws DatabaseNotExistExcep @Override public Table getTable(Identifier identifier) throws TableNotExistException { - throw new UnsupportedOperationException(); + if (SYSTEM_DATABASE_NAME.equals(identifier.getDatabaseName())) { + return getAllInSystemDatabase(identifier); + } else if (identifier.isSystemTable()) { + return getSystemTable(identifier); + } else { + return getDataOrFormatTable(identifier); + } } protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { @@ -358,6 +382,119 @@ private void updateTable(Identifier fromTable, Identifier toTable, List>> getAllTablePathsFunction = + () -> { + try { + Map> allPaths = new HashMap<>(); + for (String database : listDatabases()) { + Map tableMap = + allPaths.computeIfAbsent(database, d -> new HashMap<>()); + for (String table : listTables(database)) { + Path tableLocation = + getTableLocation(Identifier.create(database, table)); + tableMap.put(table, tableLocation); + } + } + return allPaths; + } catch (DatabaseNotExistException e) { + throw new RuntimeException("Database is deleted while listing", e); + } + }; + Table table = + SystemTableLoader.loadGlobal( + tableName, fileIO, getAllTablePathsFunction, context.options()); + if (table == null) { + throw new TableNotExistException(identifier); + } + return table; + } + + private Table getSystemTable(Identifier identifier) throws TableNotExistException { + Table originTable = + getDataOrFormatTable( + new Identifier( + identifier.getDatabaseName(), + identifier.getTableName(), + identifier.getBranchName(), + null)); + if (!(originTable instanceof FileStoreTable)) { + throw new UnsupportedOperationException( + String.format( + "Only data table support system tables, but this table %s is %s.", + identifier, originTable.getClass())); + } + Table table = + SystemTableLoader.load( + Preconditions.checkNotNull(identifier.getSystemTableName()), + (FileStoreTable) originTable); + if (table == null) { + throw new TableNotExistException(identifier); + } + return table; + } + + private Path getTableLocation(Identifier identifier) { + return new Path( + new Path(warehouse(), identifier.getDatabaseName() + DB_SUFFIX), + identifier.getTableName()); + } + + private Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { + Preconditions.checkArgument(identifier.getSystemTableName() == null); + TableSchema tableSchema = getDataTableSchema(identifier); + String uuid = null; + FileStoreTable table = + FileStoreTableFactory.create( + fileIO, + getTableLocation(identifier), + tableSchema, + new CatalogEnvironment( + identifier, + uuid, + Lock.factory( + lockFactory().orElse(null), + lockContext().orElse(null), + identifier), + null)); // todo: whether need MetastoreClient.Factory + CoreOptions options = table.coreOptions(); + if (options.type() == TableType.OBJECT_TABLE) { + String objectLocation = options.objectLocation(); + checkNotNull(objectLocation, "Object location should not be null for object table."); + table = + ObjectTable.builder() + .underlyingTable(table) + .objectLocation(objectLocation) + .objectFileIO(this.fileIO) + .build(); + } + return table; + } + + private boolean lockEnabled() { + return context.options().getOptional(LOCK_ENABLED).orElse(fileIO.isObjectStore()); + } + + private Optional lockFactory() { + if (!lockEnabled()) { + return Optional.empty(); + } + + String lock = context.options().get(LOCK_TYPE); + if (lock == null) { + return Optional.empty(); + } + + return Optional.of( + FactoryUtil.discoverFactory( + AbstractCatalog.class.getClassLoader(), CatalogLockFactory.class, lock)); + } + + private Optional lockContext() { + return Optional.of(CatalogLockContext.fromOptions(context.options())); + } + private ScheduledExecutorService tokenRefreshExecutor() { if (refreshExecutor == null) { synchronized (this) { 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 5a2a4fb8ac65..5f40b15f4f9d 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 @@ -25,6 +25,7 @@ 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.rest.responses.ListTablesResponse; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; @@ -81,4 +82,12 @@ public static AlterDatabaseResponse alterDatabaseResponse() { return new AlterDatabaseResponse( Lists.newArrayList("remove"), Lists.newArrayList("add"), new ArrayList<>()); } + + public static ListTablesResponse listTablesResponse() { + return new ListTablesResponse(Lists.newArrayList("table")); + } + + public static ListTablesResponse listTablesEmptyResponse() { + return new ListTablesResponse(Lists.newArrayList()); + } } 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 0bd5e8e42e8f..7fe3255f43a9 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 @@ -28,6 +28,7 @@ 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.rest.responses.ListTablesResponse; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -36,7 +37,9 @@ import okhttp3.mockwebserver.MockWebServer; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import java.io.IOException; import java.util.ArrayList; @@ -51,7 +54,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; /** Test for REST Catalog. */ public class RESTCatalogTest { @@ -61,6 +63,8 @@ public class RESTCatalogTest { private RESTCatalog restCatalog; private RESTCatalog mockRestCatalog; private CatalogContext context; + private String warehouseStr; + @Rule public TemporaryFolder folder = new TemporaryFolder(); @Before public void setUp() throws IOException { @@ -72,10 +76,14 @@ public void setUp() throws IOException { String initToken = "init_token"; options.set(RESTCatalogOptions.TOKEN, initToken); options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); + warehouseStr = folder.getRoot().getPath(); String mockResponse = String.format( - "{\"defaults\": {\"%s\": \"%s\"}}", - RESTCatalogInternalOptions.PREFIX.key(), "prefix"); + "{\"defaults\": {\"%s\": \"%s\", \"%s\": \"%s\"}}", + RESTCatalogInternalOptions.PREFIX.key(), + "prefix", + CatalogOptions.WAREHOUSE.key(), + warehouseStr); mockResponse(mockResponse, 200); context = CatalogContext.create(options); restCatalog = new RESTCatalog(context); @@ -90,7 +98,7 @@ public void tearDown() throws IOException { @Test public void testInitFailWhenDefineWarehouse() { Options options = new Options(); - options.set(CatalogOptions.WAREHOUSE, "/a/b/c"); + options.set(CatalogOptions.WAREHOUSE, warehouseStr); assertThrows( IllegalArgumentException.class, () -> new RESTCatalog(CatalogContext.create(options))); @@ -169,8 +177,9 @@ public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsTrue() throws Excep public void testDropDatabaseWhenCascadeIsFalseAndNoTables() throws Exception { String name = MockRESTMessage.databaseName(); boolean cascade = false; + ListTablesResponse response = MockRESTMessage.listTablesEmptyResponse(); + mockResponse(mapper.writeValueAsString(response), 200); mockResponse("", 200); - when(mockRestCatalog.listTables(name)).thenReturn(new ArrayList<>()); assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, cascade)); verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); verify(mockRestCatalog, times(1)).listTables(eq(name)); @@ -180,10 +189,8 @@ public void testDropDatabaseWhenCascadeIsFalseAndNoTables() throws Exception { public void testDropDatabaseWhenCascadeIsFalseAndTablesExist() throws Exception { String name = MockRESTMessage.databaseName(); boolean cascade = false; - mockResponse("", 200); - List tables = new ArrayList<>(); - tables.add("t1"); - when(mockRestCatalog.listTables(name)).thenReturn(tables); + ListTablesResponse response = MockRESTMessage.listTablesResponse(); + mockResponse(mapper.writeValueAsString(response), 200); assertThrows( Catalog.DatabaseNotEmptyException.class, () -> mockRestCatalog.dropDatabase(name, false, cascade)); From e852aa7e70ef0ba71af48a90a4403f5f014b7d68 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 19 Dec 2024 13:55:13 +0800 Subject: [PATCH 09/23] move some methods to CatalogUtils from AbstractCatalog --- .../paimon/catalog/AbstractCatalog.java | 70 +++--------------- .../apache/paimon/catalog/CatalogUtils.java | 72 +++++++++++++++++++ .../paimon/catalog/FileSystemCatalog.java | 5 +- .../org/apache/paimon/rest/RESTCatalog.java | 48 +++---------- .../org/apache/paimon/hive/HiveCatalog.java | 7 +- .../apache/paimon/hive/PaimonMetaHook.java | 4 +- .../apache/paimon/hive/CreateTableITCase.java | 20 +++--- .../apache/paimon/hive/HiveLocationTest.java | 4 +- .../paimon/hive/HiveReadITCaseBase.java | 6 +- 9 files changed, 117 insertions(+), 119 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index d7447c37dd79..ececa9dc0653 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -20,7 +20,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TableType; -import org.apache.paimon.factories.FactoryUtil; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; @@ -59,8 +58,10 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.CoreOptions.createCommitUser; -import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; -import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; +import static org.apache.paimon.catalog.CatalogUtils.checkNotBranch; +import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemTable; +import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase; +import static org.apache.paimon.catalog.CatalogUtils.lockFactory; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkNotNull; @@ -94,31 +95,16 @@ public FileIO fileIO() { return fileIO; } - public Optional lockFactory() { - if (!lockEnabled()) { - return Optional.empty(); - } - - String lock = catalogOptions.get(LOCK_TYPE); - if (lock == null) { - return defaultLockFactory(); - } - - return Optional.of( - FactoryUtil.discoverFactory( - AbstractCatalog.class.getClassLoader(), CatalogLockFactory.class, lock)); - } - public Optional defaultLockFactory() { return Optional.empty(); } public Optional lockContext() { - return Optional.of(CatalogLockContext.fromOptions(catalogOptions)); + return CatalogUtils.lockContext(catalogOptions); } protected boolean lockEnabled() { - return catalogOptions.getOptional(LOCK_ENABLED).orElse(fileIO.isObjectStore()); + return CatalogUtils.lockEnabled(catalogOptions, fileIO); } protected boolean allowCustomTablePath() { @@ -428,7 +414,8 @@ protected Table getDataOrFormatTable(Identifier identifier) throws TableNotExist identifier, tableMeta.uuid, Lock.factory( - lockFactory().orElse(null), + lockFactory(catalogOptions, fileIO(), defaultLockFactory()) + .orElse(null), lockContext().orElse(null), identifier), metastoreClientFactory(identifier).orElse(null))); @@ -472,7 +459,7 @@ public void createFormatTable(Identifier identifier, Schema schema) { * @return The warehouse path for the database */ public Path newDatabasePath(String database) { - return newDatabasePath(warehouse(), database); + return CatalogUtils.newDatabasePath(warehouse(), database); } public Map> allTablePaths() { @@ -507,16 +494,6 @@ public Path getTableLocation(Identifier identifier) { return new Path(newDatabasePath(identifier.getDatabaseName()), identifier.getTableName()); } - protected static void checkNotBranch(Identifier identifier, String method) { - if (identifier.getBranchName() != null) { - throw new IllegalArgumentException( - String.format( - "Cannot '%s' for branch table '%s', " - + "please modify the table with the default branch.", - method, identifier)); - } - } - protected void assertMainBranch(Identifier identifier) { if (identifier.getBranchName() != null && !DEFAULT_MAIN_BRANCH.equals(identifier.getBranchName())) { @@ -525,39 +502,10 @@ protected void assertMainBranch(Identifier identifier) { } } - protected static boolean isTableInSystemDatabase(Identifier identifier) { - return isSystemDatabase(identifier.getDatabaseName()) || identifier.isSystemTable(); - } - - protected static void checkNotSystemTable(Identifier identifier, String method) { - if (isTableInSystemDatabase(identifier)) { - throw new IllegalArgumentException( - String.format( - "Cannot '%s' for system table '%s', please use data table.", - method, identifier)); - } - } - private void copyTableDefaultOptions(Map options) { tableDefaultOptions.forEach(options::putIfAbsent); } - public static Path newTableLocation(String warehouse, Identifier identifier) { - checkNotBranch(identifier, "newTableLocation"); - checkNotSystemTable(identifier, "newTableLocation"); - return new Path( - newDatabasePath(warehouse, identifier.getDatabaseName()), - identifier.getTableName()); - } - - public static Path newDatabasePath(String warehouse, String database) { - return new Path(warehouse, database + DB_SUFFIX); - } - - public static boolean isSystemDatabase(String database) { - return SYSTEM_DATABASE_NAME.equals(database); - } - /** Validate database cannot be a system database. */ protected void checkNotSystemDatabase(String database) { if (isSystemDatabase(database)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java index 043da0504d7f..5b655dabb8e9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java @@ -18,12 +18,20 @@ package org.apache.paimon.catalog; +import org.apache.paimon.factories.FactoryUtil; +import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; import org.apache.paimon.schema.SchemaManager; import java.util.Map; +import java.util.Optional; +import static org.apache.paimon.catalog.Catalog.DB_SUFFIX; +import static org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME; import static org.apache.paimon.catalog.Catalog.TABLE_DEFAULT_OPTION_PREFIX; +import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; +import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; /** Utils for {@link Catalog}. */ @@ -60,4 +68,68 @@ public static String table(String path) { public static Map tableDefaultOptions(Map options) { return convertToPropertiesPrefixKey(options, TABLE_DEFAULT_OPTION_PREFIX); } + + public static boolean isSystemDatabase(String database) { + return SYSTEM_DATABASE_NAME.equals(database); + } + + public static boolean isTableInSystemDatabase(Identifier identifier) { + return isSystemDatabase(identifier.getDatabaseName()) || identifier.isSystemTable(); + } + + public static void checkNotSystemTable(Identifier identifier, String method) { + if (isTableInSystemDatabase(identifier)) { + throw new IllegalArgumentException( + String.format( + "Cannot '%s' for system table '%s', please use data table.", + method, identifier)); + } + } + + public static Path newDatabasePath(String warehouse, String database) { + return new Path(warehouse, database + DB_SUFFIX); + } + + public static Path newTableLocation(String warehouse, Identifier identifier) { + checkNotBranch(identifier, "newTableLocation"); + checkNotSystemTable(identifier, "newTableLocation"); + return new Path( + newDatabasePath(warehouse, identifier.getDatabaseName()), + identifier.getTableName()); + } + + public static void checkNotBranch(Identifier identifier, String method) { + if (identifier.getBranchName() != null) { + throw new IllegalArgumentException( + String.format( + "Cannot '%s' for branch table '%s', " + + "please modify the table with the default branch.", + method, identifier)); + } + } + + public static Optional lockFactory( + Options options, FileIO fileIO, Optional defaultLockFactoryOpt) { + boolean lockEnabled = lockEnabled(options, fileIO); + if (!lockEnabled) { + return Optional.empty(); + } + + String lock = options.get(LOCK_TYPE); + if (lock == null) { + return defaultLockFactoryOpt; + } + + return Optional.of( + FactoryUtil.discoverFactory( + AbstractCatalog.class.getClassLoader(), CatalogLockFactory.class, lock)); + } + + public static Optional lockContext(Options options) { + return Optional.of(CatalogLockContext.fromOptions(options)); + } + + public static boolean lockEnabled(Options options, FileIO fileIO) { + return options.getOptional(LOCK_ENABLED).orElse(fileIO != null && fileIO.isObjectStore()); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index cb0c358259f8..577dd9674ec8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.concurrent.Callable; +import static org.apache.paimon.catalog.CatalogUtils.lockFactory; import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; /** A catalog implementation for {@link FileIO}. */ @@ -123,7 +124,9 @@ public void createTableImpl(Identifier identifier, Schema schema) { private SchemaManager schemaManager(Identifier identifier) { Path path = getTableLocation(identifier); CatalogLock catalogLock = - lockFactory().map(fac -> fac.createLock(assertGetLockContext())).orElse(null); + lockFactory(catalogOptions, fileIO(), defaultLockFactory()) + .map(fac -> fac.createLock(assertGetLockContext())) + .orElse(null); return new SchemaManager(fileIO, path, identifier.getBranchNameOrDefault()) .withLock(catalogLock == null ? null : Lock.fromCatalog(catalogLock, identifier)); } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index ef073420108b..27ceff765c02 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 @@ -20,15 +20,11 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TableType; -import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; -import org.apache.paimon.catalog.CatalogLockContext; -import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; -import org.apache.paimon.factories.FactoryUtil; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; @@ -79,9 +75,10 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; +import static org.apache.paimon.catalog.CatalogUtils.lockContext; +import static org.apache.paimon.catalog.CatalogUtils.lockFactory; +import static org.apache.paimon.catalog.CatalogUtils.newTableLocation; import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; -import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; -import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.utils.Preconditions.checkNotNull; import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; @@ -393,7 +390,8 @@ private Table getAllInSystemDatabase(Identifier identifier) throws TableNotExist allPaths.computeIfAbsent(database, d -> new HashMap<>()); for (String table : listTables(database)) { Path tableLocation = - getTableLocation(Identifier.create(database, table)); + newTableLocation( + warehouse(), Identifier.create(database, table)); tableMap.put(table, tableLocation); } } @@ -435,12 +433,6 @@ private Table getSystemTable(Identifier identifier) throws TableNotExistExceptio return table; } - private Path getTableLocation(Identifier identifier) { - return new Path( - new Path(warehouse(), identifier.getDatabaseName() + DB_SUFFIX), - identifier.getTableName()); - } - private Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { Preconditions.checkArgument(identifier.getSystemTableName() == null); TableSchema tableSchema = getDataTableSchema(identifier); @@ -448,14 +440,15 @@ private Table getDataOrFormatTable(Identifier identifier) throws TableNotExistEx FileStoreTable table = FileStoreTableFactory.create( fileIO, - getTableLocation(identifier), + newTableLocation(warehouse(), identifier), tableSchema, new CatalogEnvironment( identifier, uuid, Lock.factory( - lockFactory().orElse(null), - lockContext().orElse(null), + lockFactory(context.options(), fileIO, Optional.empty()) + .orElse(null), + lockContext(context.options()).orElse(null), identifier), null)); // todo: whether need MetastoreClient.Factory CoreOptions options = table.coreOptions(); @@ -472,29 +465,6 @@ private Table getDataOrFormatTable(Identifier identifier) throws TableNotExistEx return table; } - private boolean lockEnabled() { - return context.options().getOptional(LOCK_ENABLED).orElse(fileIO.isObjectStore()); - } - - private Optional lockFactory() { - if (!lockEnabled()) { - return Optional.empty(); - } - - String lock = context.options().get(LOCK_TYPE); - if (lock == null) { - return Optional.empty(); - } - - return Optional.of( - FactoryUtil.discoverFactory( - AbstractCatalog.class.getClassLoader(), CatalogLockFactory.class, lock)); - } - - private Optional lockContext() { - return Optional.of(CatalogLockContext.fromOptions(context.options())); - } - private ScheduledExecutorService tokenRefreshExecutor() { if (refreshExecutor == null) { synchronized (this) { diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 0be872a58cbf..7588d58e93b4 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -100,6 +100,10 @@ import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_TIME; import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.TableType.FORMAT_TABLE; +import static org.apache.paimon.catalog.CatalogUtils.checkNotBranch; +import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemTable; +import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase; +import static org.apache.paimon.catalog.CatalogUtils.lockFactory; import static org.apache.paimon.hive.HiveCatalogLock.acquireTimeout; import static org.apache.paimon.hive.HiveCatalogLock.checkMaxSleep; import static org.apache.paimon.hive.HiveCatalogOptions.HADOOP_CONF_DIR; @@ -632,7 +636,8 @@ public org.apache.paimon.table.Table getDataOrFormatTable(Identifier identifier) identifier, tableMeta.uuid(), Lock.factory( - lockFactory().orElse(null), + lockFactory(catalogOptions, fileIO(), defaultLockFactory()) + .orElse(null), lockContext().orElse(null), identifier), metastoreClientFactory(identifier).orElse(null))); diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java index 5cc826b554bf..38fa4dfe4d39 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java @@ -19,8 +19,8 @@ package org.apache.paimon.hive; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -87,7 +87,7 @@ public void preCreateTable(Table table) throws MetaException { org.apache.hadoop.fs.Path hadoopPath = getDnsPath(new org.apache.hadoop.fs.Path(warehouse), conf); warehouse = hadoopPath.toUri().toString(); - location = AbstractCatalog.newTableLocation(warehouse, identifier).toUri().toString(); + location = CatalogUtils.newTableLocation(warehouse, identifier).toUri().toString(); table.getSd().setLocation(location); } diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java index 15856c3c06cd..992272b0f6ba 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/CreateTableITCase.java @@ -18,10 +18,10 @@ package org.apache.paimon.hive; -import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -94,7 +94,7 @@ public void testCreateExternalTableWithPaimonTable() throws Exception { Maps.newHashMap(), ""); Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema); // Create hive external table @@ -189,7 +189,7 @@ public void testCreateTableUsePartitionedBy() { // check the paimon table schema Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); Optional tableSchema = new SchemaManager(LocalFileIO.create(), tablePath).latest(); assertThat(tableSchema).isPresent(); @@ -245,7 +245,7 @@ public void testLowerTableName() throws Catalog.TableNotExistException { } // check the paimon table name and schema Identifier identifier = Identifier.create(DATABASE_TEST, tableName.toLowerCase()); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); Options conf = new Options(); conf.set(CatalogOptions.WAREHOUSE, path); CatalogContext catalogContext = CatalogContext.create(conf); @@ -310,7 +310,7 @@ public void testLowerDBName() throws Catalog.TableNotExistException { // check the paimon db name、table name and schema Identifier identifier = Identifier.create(upperDB.toLowerCase(), tableName.toLowerCase()); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); Options conf = new Options(); conf.set(CatalogOptions.WAREHOUSE, path); CatalogContext catalogContext = CatalogContext.create(conf); @@ -355,7 +355,7 @@ public void testCreateTableWithPrimaryKey() { // check the paimon table schema Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); Optional tableSchema = new SchemaManager(LocalFileIO.create(), tablePath).latest(); assertThat(tableSchema).isPresent(); @@ -397,7 +397,7 @@ public void testCreateTableWithPartition() { // check the paimon table schema Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); Optional tableSchema = new SchemaManager(LocalFileIO.create(), tablePath).latest(); assertThat(tableSchema).isPresent(); @@ -441,7 +441,7 @@ public void testCreateTableSpecifyProperties() { // check the paimon table schema Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); Optional tableSchema = new SchemaManager(LocalFileIO.create(), tablePath).latest(); assertThat(tableSchema).isPresent(); @@ -489,7 +489,7 @@ public void testCreateTableFailing() throws Exception { Maps.newHashMap(), ""); Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema); String hiveSql = @@ -533,7 +533,7 @@ public void testCreateTableFailing() throws Exception { } catch (Exception ignore) { } finally { Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); boolean isPresent = new SchemaManager(LocalFileIO.create(), tablePath).latest().isPresent(); Assertions.assertThat(isPresent).isFalse(); diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java index f3fe03fbba6d..7e52b892791b 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveLocationTest.java @@ -18,8 +18,8 @@ package org.apache.paimon.hive; -import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -257,7 +257,7 @@ public void testRWIT() { Identifier identifier = Identifier.create(dbName, tableName); String location = - AbstractCatalog.newTableLocation(warehouse, identifier).toUri().toString(); + CatalogUtils.newTableLocation(warehouse, identifier).toUri().toString(); String createTableSqlStr = getCreateTableSqlStr(tableName, location, locationInProperties); diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java index 4b16788ee716..882215f7c0cd 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveReadITCaseBase.java @@ -19,7 +19,7 @@ package org.apache.paimon.hive; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.AbstractCatalog; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Decimal; @@ -975,7 +975,7 @@ public void testReadExternalTableWithEmptyDataAndIgnoreCase() throws Exception { Maps.newHashMap(), ""); Identifier identifier = Identifier.create(DATABASE_TEST, tableName); - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema); // Create hive external table @@ -1057,7 +1057,7 @@ public void testReadExternalTableWithDataAndIgnoreCase() throws Exception { commit.close(); // add column, do some ddl which will generate a new version schema-n file. - Path tablePath = AbstractCatalog.newTableLocation(path, identifier); + Path tablePath = CatalogUtils.newTableLocation(path, identifier); SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), tablePath); schemaManager.commitChanges(SchemaChange.addColumn("N1", DataTypes.STRING())); From 3e5cde26e26a840fbd8fd1b26a8a519374cd85c9 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 19 Dec 2024 14:22:48 +0800 Subject: [PATCH 10/23] move getSystemTable to CatalogUtils --- .../paimon/catalog/AbstractCatalog.java | 23 ++------------- .../apache/paimon/catalog/CatalogUtils.java | 29 +++++++++++++++++++ .../org/apache/paimon/rest/RESTCatalog.java | 16 ++-------- .../org/apache/paimon/hive/HiveCatalog.java | 1 + 4 files changed, 34 insertions(+), 35 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index ececa9dc0653..ef6c0e33485e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -59,6 +59,7 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.CoreOptions.createCommitUser; import static org.apache.paimon.catalog.CatalogUtils.checkNotBranch; +import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemDatabase; import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemTable; import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase; import static org.apache.paimon.catalog.CatalogUtils.lockFactory; @@ -383,20 +384,7 @@ public Table getTable(Identifier identifier) throws TableNotExistException { identifier.getTableName(), identifier.getBranchName(), null)); - if (!(originTable instanceof FileStoreTable)) { - throw new UnsupportedOperationException( - String.format( - "Only data table support system tables, but this table %s is %s.", - identifier, originTable.getClass())); - } - Table table = - SystemTableLoader.load( - Preconditions.checkNotNull(identifier.getSystemTableName()), - (FileStoreTable) originTable); - if (table == null) { - throw new TableNotExistException(identifier); - } - return table; + return CatalogUtils.getSystemTable(identifier, originTable); } else { return getDataOrFormatTable(identifier); } @@ -506,13 +494,6 @@ private void copyTableDefaultOptions(Map options) { tableDefaultOptions.forEach(options::putIfAbsent); } - /** Validate database cannot be a system database. */ - protected void checkNotSystemDatabase(String database) { - if (isSystemDatabase(database)) { - throw new ProcessSystemDatabaseException(); - } - } - private void validateAutoCreateClose(Map options) { checkArgument( !Boolean.parseBoolean( diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java index 5b655dabb8e9..826e2c084722 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java @@ -23,6 +23,10 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.system.SystemTableLoader; +import org.apache.paimon.utils.Preconditions; import java.util.Map; import java.util.Optional; @@ -73,6 +77,13 @@ public static boolean isSystemDatabase(String database) { return SYSTEM_DATABASE_NAME.equals(database); } + /** Validate database cannot be a system database. */ + public static void checkNotSystemDatabase(String database) { + if (isSystemDatabase(database)) { + throw new Catalog.ProcessSystemDatabaseException(); + } + } + public static boolean isTableInSystemDatabase(Identifier identifier) { return isSystemDatabase(identifier.getDatabaseName()) || identifier.isSystemTable(); } @@ -132,4 +143,22 @@ public static Optional lockContext(Options options) { public static boolean lockEnabled(Options options, FileIO fileIO) { return options.getOptional(LOCK_ENABLED).orElse(fileIO != null && fileIO.isObjectStore()); } + + public static Table getSystemTable(Identifier identifier, Table originTable) + throws Catalog.TableNotExistException { + if (!(originTable instanceof FileStoreTable)) { + throw new UnsupportedOperationException( + String.format( + "Only data table support system tables, but this table %s is %s.", + identifier, originTable.getClass())); + } + Table table = + SystemTableLoader.load( + Preconditions.checkNotNull(identifier.getSystemTableName()), + (FileStoreTable) originTable); + if (table == null) { + throw new Catalog.TableNotExistException(identifier); + } + return table; + } } 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 27ceff765c02..335fe897e5c2 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 @@ -22,6 +22,7 @@ import org.apache.paimon.TableType; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; @@ -417,20 +418,7 @@ private Table getSystemTable(Identifier identifier) throws TableNotExistExceptio identifier.getTableName(), identifier.getBranchName(), null)); - if (!(originTable instanceof FileStoreTable)) { - throw new UnsupportedOperationException( - String.format( - "Only data table support system tables, but this table %s is %s.", - identifier, originTable.getClass())); - } - Table table = - SystemTableLoader.load( - Preconditions.checkNotNull(identifier.getSystemTableName()), - (FileStoreTable) originTable); - if (table == null) { - throw new TableNotExistException(identifier); - } - return table; + return CatalogUtils.getSystemTable(identifier, originTable); } private Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 7588d58e93b4..fd22ca20323e 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -101,6 +101,7 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.TableType.FORMAT_TABLE; import static org.apache.paimon.catalog.CatalogUtils.checkNotBranch; +import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemDatabase; import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemTable; import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase; import static org.apache.paimon.catalog.CatalogUtils.lockFactory; From 07e8ce026dc9ae574c5f2a3b99ec6f22a588d39a Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 19 Dec 2024 14:41:10 +0800 Subject: [PATCH 11/23] handle exception when alter or rename table and add check system database for database api --- .../org/apache/paimon/rest/RESTCatalog.java | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 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 335fe897e5c2..e56419d1c5ea 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 @@ -76,6 +76,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; +import static org.apache.paimon.catalog.CatalogUtils.checkNotSystemDatabase; +import static org.apache.paimon.catalog.CatalogUtils.isSystemDatabase; import static org.apache.paimon.catalog.CatalogUtils.lockContext; import static org.apache.paimon.catalog.CatalogUtils.lockFactory; import static org.apache.paimon.catalog.CatalogUtils.newTableLocation; @@ -185,6 +187,7 @@ public List listDatabases() { @Override public void createDatabase(String name, boolean ignoreIfExists, Map properties) throws DatabaseAlreadyExistException { + checkNotSystemDatabase(name); CreateDatabaseRequest request = new CreateDatabaseRequest(name, properties); try { client.post( @@ -198,6 +201,9 @@ public void createDatabase(String name, boolean ignoreIfExists, Map changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { + checkNotSystemDatabase(name); try { Pair, Set> setPropertiesToRemoveKeys = PropertyChange.getSetPropertiesToRemoveKeys(changes); @@ -305,22 +313,40 @@ public void createTable(Identifier identifier, Schema schema, boolean ignoreIfEx @Override public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) throws TableNotExistException, TableAlreadyExistException { - updateTable(fromTable, toTable, new ArrayList<>()); + try { + updateTable(fromTable, toTable, new ArrayList<>()); + } catch (NoSuchResourceException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(fromTable); + } + } } @Override public void alterTable( Identifier identifier, List changes, boolean ignoreIfNotExists) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { - updateTable(identifier, null, changes); + try { + updateTable(identifier, null, changes); + } catch (NoSuchResourceException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(identifier); + } + } } @Override public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throws TableNotExistException { - client.delete( - resourcePaths.table(identifier.getDatabaseName(), identifier.getTableName()), - headers()); + try { + client.delete( + resourcePaths.table(identifier.getDatabaseName(), identifier.getTableName()), + headers()); + } catch (NoSuchResourceException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(identifier); + } + } } @Override From 244ea9585d23ed8e23f55e54505ee8673de3c02b Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 20 Dec 2024 09:41:08 +0800 Subject: [PATCH 12/23] add exception TableNoPermissionException to define when no permission for table in RESTCatalog --- .../org/apache/paimon/catalog/Catalog.java | 22 ++++++++++ .../org/apache/paimon/rest/RESTCatalog.java | 42 +++++++++++-------- 2 files changed, 47 insertions(+), 17 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 37ea6fa5e203..28add9e6650c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -475,6 +475,28 @@ public Identifier identifier() { } } + /** + * Exception for trying to operate on a table that doesn't have permission. Define as a runtime + * exception: 1. Other engine has no this type exception. 2. It wouldn't bring api break change. + */ + class TableNoPermissionException extends RuntimeException { + private static final String MSG = "No permission for Table %s."; + private final Identifier identifier; + + public TableNoPermissionException(Identifier identifier) { + this(identifier, null); + } + + public TableNoPermissionException(Identifier identifier, Throwable cause) { + super(String.format(MSG, identifier.getFullName()), cause); + this.identifier = identifier; + } + + public Identifier identifier() { + return identifier; + } + } + /** Exception for trying to operate on a partition that doesn't exist. */ class PartitionNotExistException extends 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 e56419d1c5ea..f099a431b16b 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 @@ -36,6 +36,7 @@ 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.ForbiddenException; import org.apache.paimon.rest.exceptions.NoSuchResourceException; import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; @@ -278,23 +279,6 @@ public Table getTable(Identifier identifier) throws TableNotExistException { } } - protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { - try { - GetTableResponse response = - client.get( - resourcePaths.table( - identifier.getDatabaseName(), identifier.getTableName()), - GetTableResponse.class, - headers()); - if (response.getSchema() != null) { - return response.getSchema(); - } - } catch (NoSuchResourceException e) { - throw new TableNotExistException(identifier); - } - throw new TableNotExistException(identifier); - } - @Override public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException { @@ -319,6 +303,10 @@ public void renameTable(Identifier fromTable, Identifier toTable, boolean ignore if (!ignoreIfNotExists) { throw new TableNotExistException(fromTable); } + } catch (ForbiddenException e) { + throw new TableNoPermissionException(fromTable); + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistException(toTable); } } @@ -332,6 +320,8 @@ public void alterTable( if (!ignoreIfNotExists) { throw new TableNotExistException(identifier); } + } catch (ForbiddenException e) { + throw new TableNoPermissionException(identifier); } } @@ -346,6 +336,8 @@ public void dropTable(Identifier identifier, boolean ignoreIfNotExists) if (!ignoreIfNotExists) { throw new TableNotExistException(identifier); } + } catch (ForbiddenException e) { + throw new TableNoPermissionException(identifier); } } @@ -396,6 +388,22 @@ private Map headers() { return catalogAuth.getHeaders(); } + protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { + try { + GetTableResponse response = + client.get( + resourcePaths.table( + identifier.getDatabaseName(), identifier.getTableName()), + GetTableResponse.class, + headers()); + return response.getSchema(); + } catch (NoSuchResourceException e) { + throw new TableNotExistException(identifier); + } catch (ForbiddenException e) { + throw new TableNoPermissionException(identifier, e); + } + } + // todo: how know which exception to throw private void updateTable(Identifier fromTable, Identifier toTable, List changes) { UpdateTableRequest request = new UpdateTableRequest(fromTable, toTable, changes); From 19b12628b46968c6a765d00d9d847a9d8d6e2368 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 20 Dec 2024 10:13:50 +0800 Subject: [PATCH 13/23] update TableNoPermissionException to NoPermissionException for supporting database --- .../org/apache/paimon/catalog/Catalog.java | 45 ++++++++++--------- .../org/apache/paimon/rest/RESTCatalog.java | 16 +++++-- 2 files changed, 35 insertions(+), 26 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 28add9e6650c..3a18d2be8c8e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -364,6 +364,29 @@ default void repairTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); } + /** + * Exception for trying to operate on a resource that doesn't have permission. Define as a + * runtime exception: 1. Other engine has no this type exception. 2. It wouldn't bring api break + * change. + */ + class NoPermissionException extends RuntimeException { + private static final String MSG = "No permission for %s %s."; + + public static NoPermissionException createDatabaseNoPermissionException( + String databaseName, Throwable cause) { + return new NoPermissionException("database", databaseName, cause); + } + + public static NoPermissionException createTableNoPermissionException( + Identifier identifier, Throwable cause) { + return new NoPermissionException("table", identifier.getFullName(), cause); + } + + public NoPermissionException(String resourceType, String resourceName, Throwable cause) { + super(String.format(MSG, resourceType, resourceName), cause); + } + } + /** Exception for trying to drop on a database that is not empty. */ class DatabaseNotEmptyException extends Exception { private static final String MSG = "Database %s is not empty."; @@ -475,28 +498,6 @@ public Identifier identifier() { } } - /** - * Exception for trying to operate on a table that doesn't have permission. Define as a runtime - * exception: 1. Other engine has no this type exception. 2. It wouldn't bring api break change. - */ - class TableNoPermissionException extends RuntimeException { - private static final String MSG = "No permission for Table %s."; - private final Identifier identifier; - - public TableNoPermissionException(Identifier identifier) { - this(identifier, null); - } - - public TableNoPermissionException(Identifier identifier, Throwable cause) { - super(String.format(MSG, identifier.getFullName()), cause); - this.identifier = identifier; - } - - public Identifier identifier() { - return identifier; - } - } - /** Exception for trying to operate on a partition that doesn't exist. */ class PartitionNotExistException extends 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 f099a431b16b..33adc1b0a09e 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 @@ -197,6 +197,8 @@ public void createDatabase(String name, boolean ignoreIfExists, Map null)); } catch (NoSuchResourceException e) { throw new DatabaseNotExistException(name); + } catch (ForbiddenException e) { + throw NoPermissionException.createDatabaseNoPermissionException(name, e); } } @@ -228,6 +232,8 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade if (!ignoreIfNotExists) { throw new DatabaseNotExistException(name); } + } catch (ForbiddenException e) { + throw NoPermissionException.createDatabaseNoPermissionException(name, e); } } @@ -255,6 +261,8 @@ public void alterDatabase(String name, List changes, boolean ign if (!ignoreIfNotExists) { throw new DatabaseNotExistException(name); } + } catch (ForbiddenException e) { + throw NoPermissionException.createDatabaseNoPermissionException(name, e); } } @@ -304,7 +312,7 @@ public void renameTable(Identifier fromTable, Identifier toTable, boolean ignore throw new TableNotExistException(fromTable); } } catch (ForbiddenException e) { - throw new TableNoPermissionException(fromTable); + throw NoPermissionException.createTableNoPermissionException(fromTable, e); } catch (AlreadyExistsException e) { throw new TableAlreadyExistException(toTable); } @@ -321,7 +329,7 @@ public void alterTable( throw new TableNotExistException(identifier); } } catch (ForbiddenException e) { - throw new TableNoPermissionException(identifier); + throw NoPermissionException.createTableNoPermissionException(identifier, e); } } @@ -337,7 +345,7 @@ public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throw new TableNotExistException(identifier); } } catch (ForbiddenException e) { - throw new TableNoPermissionException(identifier); + throw NoPermissionException.createTableNoPermissionException(identifier, e); } } @@ -400,7 +408,7 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } catch (NoSuchResourceException e) { throw new TableNotExistException(identifier); } catch (ForbiddenException e) { - throw new TableNoPermissionException(identifier, e); + throw NoPermissionException.createTableNoPermissionException(identifier, e); } } From bb03611c51bafb5aa74a6888a41623fd14ee787d Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 20 Dec 2024 12:34:33 +0800 Subject: [PATCH 14/23] add test for CreateTableRequest and DataField json parse --- .../org/apache/paimon/catalog/Catalog.java | 6 +- .../apache/paimon/rest/RESTObjectMapper.java | 21 +++ .../rest/requests/CreateTableRequest.java | 52 ++++++-- .../paimon/rest/requests/TableSchema.java | 123 ++++++++++++++++++ .../apache/paimon/utils/JsonSerdeUtil.java | 30 ++--- .../apache/paimon/rest/MockRESTMessage.java | 21 +++ .../paimon/rest/RESTObjectMapperTest.java | 33 +++++ 7 files changed, 258 insertions(+), 28 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 3a18d2be8c8e..e1d044e95b3d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -371,15 +371,17 @@ default void repairTable(Identifier identifier) throws TableNotExistException { */ class NoPermissionException extends RuntimeException { private static final String MSG = "No permission for %s %s."; + private static final String DATABASE_TYPE_NAME = "database"; + private static final String TABLE_TYPE_NAME = "table"; public static NoPermissionException createDatabaseNoPermissionException( String databaseName, Throwable cause) { - return new NoPermissionException("database", databaseName, cause); + return new NoPermissionException(DATABASE_TYPE_NAME, databaseName, cause); } public static NoPermissionException createTableNoPermissionException( Identifier identifier, Throwable cause) { - return new NoPermissionException("table", identifier.getFullName(), cause); + return new NoPermissionException(TABLE_TYPE_NAME, identifier.getFullName(), cause); } public NoPermissionException(String resourceType, String resourceName, Throwable cause) { 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..fa66e754ac85 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -18,18 +18,39 @@ package org.apache.paimon.rest; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeJsonParser; + import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.Module; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.SerializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.module.SimpleModule; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import static org.apache.paimon.utils.JsonSerdeUtil.registerJsonObjects; + /** Object mapper for REST request and response. */ public class RESTObjectMapper { public static ObjectMapper create() { ObjectMapper mapper = new ObjectMapper(); mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); + mapper.registerModule(createPaimonRestJacksonModule()); mapper.registerModule(new JavaTimeModule()); return mapper; } + + public static Module createPaimonRestJacksonModule() { + SimpleModule module = new SimpleModule("Paimon_REST"); + registerJsonObjects( + module, + DataField.class, + DataField::serializeJson, + DataTypeJsonParser::parseDataField); + registerJsonObjects( + module, DataType.class, DataType::serializeJson, DataTypeJsonParser::parseDataType); + return module; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java index 1e152d7f1f56..3a3262932522 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java @@ -29,30 +29,60 @@ /** Request for creating table. */ public class CreateTableRequest implements RESTRequest { - private static final String FIELD_IDENTIFIER = "identifier"; + private static final String FIELD_DATABASE_NAME = "database"; + private static final String FIELD_TABLE_NAME = "table"; + private static final String FIELD_BRANCH_NAME = "branch"; private static final String FIELD_SCHEMA = "schema"; - @JsonProperty(FIELD_IDENTIFIER) - private Identifier identifier; + @JsonProperty(FIELD_DATABASE_NAME) + private String databaseName; + + @JsonProperty(FIELD_TABLE_NAME) + private String tableName; + + @JsonProperty(FIELD_BRANCH_NAME) + private String branchName; @JsonProperty(FIELD_SCHEMA) - private Schema schema; + private TableSchema schema; @JsonCreator public CreateTableRequest( - @JsonProperty(FIELD_IDENTIFIER) Identifier identifier, - @JsonProperty(FIELD_SCHEMA) Schema schema) { - this.identifier = identifier; + @JsonProperty(FIELD_DATABASE_NAME) String databaseName, + @JsonProperty(FIELD_TABLE_NAME) String tableName, + @JsonProperty(FIELD_BRANCH_NAME) String branchName, + @JsonProperty(FIELD_SCHEMA) TableSchema schema) { + this.databaseName = databaseName; + this.tableName = tableName; + this.branchName = branchName; this.schema = schema; } - @JsonGetter(FIELD_IDENTIFIER) - public Identifier getIdentifier() { - return identifier; + public CreateTableRequest(Identifier identifier, Schema schema) { + this( + identifier.getDatabaseName(), + identifier.getTableName(), + identifier.getBranchName(), + new TableSchema(schema)); + } + + @JsonGetter(FIELD_DATABASE_NAME) + public String getDatabaseName() { + return databaseName; + } + + @JsonGetter(FIELD_TABLE_NAME) + public String getTableName() { + return tableName; + } + + @JsonGetter(FIELD_BRANCH_NAME) + public String getBranchName() { + return branchName; } @JsonGetter(FIELD_SCHEMA) - public Schema getSchema() { + public TableSchema getSchema() { return schema; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java new file mode 100644 index 000000000000..d49e4062eea5 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataField; + +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.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Wrap the {@link Schema} class to support RESTCatalog. Define a class as: 1. This class in rest + * catalog is easy to maintain. 2. It's easy to manage rest API fields. + */ +public class TableSchema { + private static final String FIELD_FILED_NAME = "fields"; + private static final String FIELD_PARTITION_KEYS_NAME = "partitionKeys"; + private static final String FIELD_PRIMARY_KEYS_NAME = "primaryKeys"; + private static final String FIELD_OPTIONS_NAME = "options"; + private static final String FIELD_COMMENT_NAME = "comment"; + + @JsonProperty(FIELD_FILED_NAME) + private final List fields; + + @JsonProperty(FIELD_PARTITION_KEYS_NAME) + private final List partitionKeys; + + @JsonProperty(FIELD_PRIMARY_KEYS_NAME) + private final List primaryKeys; + + @JsonProperty(FIELD_OPTIONS_NAME) + private final Map options; + + @JsonProperty(FIELD_COMMENT_NAME) + private final String comment; + + @JsonCreator + public TableSchema( + @JsonProperty(FIELD_FILED_NAME) List fields, + @JsonProperty(FIELD_PARTITION_KEYS_NAME) List partitionKeys, + @JsonProperty(FIELD_PRIMARY_KEYS_NAME) List primaryKeys, + @JsonProperty(FIELD_OPTIONS_NAME) Map options, + @JsonProperty(FIELD_COMMENT_NAME) String comment) { + this.fields = fields; + this.partitionKeys = partitionKeys; + this.primaryKeys = primaryKeys; + this.options = options; + this.comment = comment; + } + + public TableSchema(Schema schema) { + this.fields = schema.fields(); + this.partitionKeys = schema.partitionKeys(); + this.primaryKeys = schema.primaryKeys(); + this.options = schema.options(); + this.comment = schema.comment(); + } + + @JsonGetter(FIELD_FILED_NAME) + public List getFields() { + return fields; + } + + @JsonGetter(FIELD_PARTITION_KEYS_NAME) + public List getPartitionKeys() { + return partitionKeys; + } + + @JsonGetter(FIELD_PRIMARY_KEYS_NAME) + public List getPrimaryKeys() { + return primaryKeys; + } + + @JsonGetter(FIELD_OPTIONS_NAME) + public Map getOptions() { + return options; + } + + @JsonGetter(FIELD_COMMENT_NAME) + public String getComment() { + return comment; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } else { + TableSchema that = (TableSchema) o; + return Objects.equals(fields, that.fields) + && Objects.equals(partitionKeys, that.partitionKeys) + && Objects.equals(primaryKeys, that.primaryKeys) + && Objects.equals(options, that.options) + && Objects.equals(comment, that.comment); + } + } + + @Override + public int hashCode() { + return Objects.hash(fields, partitionKeys, primaryKeys, options, comment); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java index a919d83c8741..edc6dac5f992 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java @@ -154,21 +154,7 @@ public static String toFlatJson(T t) { } } - private static Module createPaimonJacksonModule() { - SimpleModule module = new SimpleModule("Paimon"); - registerJsonObjects( - module, TableSchema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); - registerJsonObjects( - module, - DataField.class, - DataField::serializeJson, - DataTypeJsonParser::parseDataField); - registerJsonObjects( - module, DataType.class, DataType::serializeJson, DataTypeJsonParser::parseDataType); - return module; - } - - private static void registerJsonObjects( + public static void registerJsonObjects( SimpleModule module, Class clazz, JsonSerializer serializer, @@ -192,6 +178,20 @@ public T deserialize(JsonParser parser, DeserializationContext context) }); } + private static Module createPaimonJacksonModule() { + SimpleModule module = new SimpleModule("Paimon"); + registerJsonObjects( + module, TableSchema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); + registerJsonObjects( + module, + DataField.class, + DataField::serializeJson, + DataTypeJsonParser::parseDataField); + registerJsonObjects( + module, DataType.class, DataType::serializeJson, DataTypeJsonParser::parseDataType); + return module; + } + /** * Parses the provided JSON string and casts it to the specified type of {@link JsonNode}. * 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 5f40b15f4f9d..a701b54c9dbd 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java @@ -18,14 +18,18 @@ package org.apache.paimon.rest; +import org.apache.paimon.catalog.Identifier; import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.CreateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; 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.rest.responses.ListTablesResponse; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; @@ -90,4 +94,21 @@ public static ListTablesResponse listTablesResponse() { public static ListTablesResponse listTablesEmptyResponse() { return new ListTablesResponse(Lists.newArrayList()); } + + public static CreateTableRequest createTableRequest(String name) { + Identifier identifier = Identifier.create(databaseName(), name); + Map options = new HashMap<>(); + options.put("k1", "v1"); + Schema schema = + Schema.newBuilder() + .column("pt", DataTypes.INT()) + .column("pk", DataTypes.INT()) + .column("col1", DataTypes.INT()) + .column("col2", DataTypes.STRING()) + .partitionKeys("pt") + .primaryKey("pk", "pt") + .options(options) + .build(); + return new CreateTableRequest(identifier, schema); + } } 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 1cf382a9f438..6da61932a6e7 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 @@ -20,12 +20,16 @@ import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.requests.CreateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; 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.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.IntType; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -125,4 +129,33 @@ public void alterDatabaseResponseParseTest() throws Exception { assertEquals(response.getUpdated().size(), parseData.getUpdated().size()); assertEquals(response.getMissing().size(), parseData.getMissing().size()); } + + @Test + public void createTableRequestParseTest() throws Exception { + CreateTableRequest request = MockRESTMessage.createTableRequest("t1"); + String requestStr = mapper.writeValueAsString(request); + CreateTableRequest parseData = mapper.readValue(requestStr, CreateTableRequest.class); + assertEquals(request.getDatabaseName(), parseData.getDatabaseName()); + assertEquals(request.getTableName(), parseData.getTableName()); + assertEquals(request.getBranchName(), parseData.getBranchName()); + assertEquals(request.getSchema(), parseData.getSchema()); + } + + // This test is to guarantee the compatibility of field name in RESTCatalog. + @Test + public void dataFieldParseTest() throws Exception { + int id = 1; + String name = "col1"; + IntType type = DataTypes.INT(); + String descStr = "desc"; + String dataFieldStr = + String.format( + "{\"id\": %d,\"name\":\"%s\",\"type\":\"%s\", \"description\":\"%s\"}", + id, name, type, descStr); + DataField parseData = mapper.readValue(dataFieldStr, DataField.class); + assertEquals(id, parseData.id()); + assertEquals(name, parseData.name()); + assertEquals(type, parseData.type()); + assertEquals(descStr, parseData.description()); + } } From 219a47a30b59cad716c899500493e59b55b88126 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 23 Dec 2024 12:48:34 +0800 Subject: [PATCH 15/23] add SchemaSerializer and IdentifierSerializer to to support json format --- .../org/apache/paimon/rest/RESTCatalog.java | 4 +- .../apache/paimon/rest/RESTObjectMapper.java | 11 ++ .../rest/requests/CreateTableRequest.java | 51 ++----- .../paimon/rest/requests/SchemaChanges.java | 131 ++++++++++++++++++ .../paimon/rest/requests/TableSchema.java | 123 ---------------- .../rest/requests/UpdateTableRequest.java | 31 ++--- .../rest/serializer/IdentifierSerializer.java | 72 ++++++++++ .../rest/serializer/SchemaSerializer.java | 109 +++++++++++++++ ...alizer.java => TableSchemaSerializer.java} | 4 +- .../apache/paimon/utils/JsonSerdeUtil.java | 7 +- .../apache/paimon/rest/MockRESTMessage.java | 7 + .../paimon/rest/RESTObjectMapperTest.java | 14 +- 12 files changed, 375 insertions(+), 189 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java rename paimon-core/src/main/java/org/apache/paimon/schema/{SchemaSerializer.java => TableSchemaSerializer.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 33adc1b0a09e..f436254f1dcd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -41,6 +41,7 @@ import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.requests.CreateTableRequest; +import org.apache.paimon.rest.requests.SchemaChanges; import org.apache.paimon.rest.requests.UpdateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; @@ -414,7 +415,8 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE // todo: how know which exception to throw private void updateTable(Identifier fromTable, Identifier toTable, List changes) { - UpdateTableRequest request = new UpdateTableRequest(fromTable, toTable, changes); + UpdateTableRequest request = + new UpdateTableRequest(fromTable, toTable, new SchemaChanges(changes)); client.post( resourcePaths.table(fromTable.getDatabaseName(), fromTable.getTableName()), request, diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java index fa66e754ac85..b0f9ba7b83e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -18,6 +18,10 @@ package org.apache.paimon.rest; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.rest.serializer.IdentifierSerializer; +import org.apache.paimon.rest.serializer.SchemaSerializer; +import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeJsonParser; @@ -49,8 +53,15 @@ public static Module createPaimonRestJacksonModule() { DataField.class, DataField::serializeJson, DataTypeJsonParser::parseDataField); + registerJsonObjects( + module, Schema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); registerJsonObjects( module, DataType.class, DataType::serializeJson, DataTypeJsonParser::parseDataType); + registerJsonObjects( + module, + Identifier.class, + IdentifierSerializer.INSTANCE, + IdentifierSerializer.INSTANCE); return module; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java index 3a3262932522..3f80e0fee5f9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java @@ -29,60 +29,29 @@ /** Request for creating table. */ public class CreateTableRequest implements RESTRequest { - private static final String FIELD_DATABASE_NAME = "database"; - private static final String FIELD_TABLE_NAME = "table"; - private static final String FIELD_BRANCH_NAME = "branch"; + private static final String FIELD_IDENTIFIER_NAME = "identifier"; private static final String FIELD_SCHEMA = "schema"; - @JsonProperty(FIELD_DATABASE_NAME) - private String databaseName; - - @JsonProperty(FIELD_TABLE_NAME) - private String tableName; - - @JsonProperty(FIELD_BRANCH_NAME) - private String branchName; + @JsonProperty(FIELD_IDENTIFIER_NAME) + private Identifier identifier; @JsonProperty(FIELD_SCHEMA) - private TableSchema schema; + private Schema schema; @JsonCreator public CreateTableRequest( - @JsonProperty(FIELD_DATABASE_NAME) String databaseName, - @JsonProperty(FIELD_TABLE_NAME) String tableName, - @JsonProperty(FIELD_BRANCH_NAME) String branchName, - @JsonProperty(FIELD_SCHEMA) TableSchema schema) { - this.databaseName = databaseName; - this.tableName = tableName; - this.branchName = branchName; + @JsonProperty(FIELD_IDENTIFIER_NAME) Identifier identifier, + @JsonProperty(FIELD_SCHEMA) Schema schema) { this.schema = schema; } - public CreateTableRequest(Identifier identifier, Schema schema) { - this( - identifier.getDatabaseName(), - identifier.getTableName(), - identifier.getBranchName(), - new TableSchema(schema)); - } - - @JsonGetter(FIELD_DATABASE_NAME) - public String getDatabaseName() { - return databaseName; - } - - @JsonGetter(FIELD_TABLE_NAME) - public String getTableName() { - return tableName; - } - - @JsonGetter(FIELD_BRANCH_NAME) - public String getBranchName() { - return branchName; + @JsonGetter(FIELD_IDENTIFIER_NAME) + public Identifier getIdentifier() { + return identifier; } @JsonGetter(FIELD_SCHEMA) - public TableSchema getSchema() { + public Schema getSchema() { return schema; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java new file mode 100644 index 000000000000..698525f604b9 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.schema.SchemaChange; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class SchemaChanges { + private static final String FIELD_SET_OPTIONS_NAME = "set-options"; + private static final String FIELD_REMOVE_OPTIONS_NAME = "remove-options"; + private static final String FIELD_COMMENT_NAME = "comment"; + private static final String FIELD_ADD_COLUMNS_NAME = "add-columns"; + private static final String FIELD_RENAME_COLUMNS_NAME = "rename-columns"; + + private Map setOptions; + private List removeOptions; + private String comment; + private List addColumns; + private List renameColumns; + private List dropColumns; + private List updateColumnTypes; + private List updateColumnNullabilities; + private List updateColumnComments; + private List updateColumnPositions; + + public SchemaChanges( + Map setOptions, + List removeOptions, + String comment, + List addColumns, + List renameColumns, + List dropColumns, + List updateColumnTypes, + List updateColumnNullabilities, + List updateColumnComments, + List updateColumnPositions) { + this.setOptions = setOptions; + this.removeOptions = removeOptions; + this.comment = comment; + this.addColumns = addColumns; + this.renameColumns = renameColumns; + this.dropColumns = dropColumns; + this.updateColumnTypes = updateColumnTypes; + this.updateColumnNullabilities = updateColumnNullabilities; + this.updateColumnComments = updateColumnComments; + this.updateColumnPositions = updateColumnPositions; + } + + public SchemaChanges(List changes) { + Map setOptions = null; + List removeOptions = new ArrayList<>(); + String comment = null; + List addColumns = new ArrayList<>(); + for (SchemaChange change : changes) { + if (change instanceof SchemaChange.SetOption) { + setOptions.put( + ((SchemaChange.SetOption) change).key(), + ((SchemaChange.SetOption) change).value()); + } else if (change instanceof SchemaChange.RemoveOption) { + removeOptions.add(((SchemaChange.RemoveOption) change).key()); + } else if (change instanceof SchemaChange.UpdateComment) { + comment = ((SchemaChange.UpdateComment) change).comment(); + } else if (change instanceof SchemaChange.AddColumn) { + addColumns.add((SchemaChange.AddColumn) change); + } + } + this.setOptions = setOptions; + this.removeOptions = removeOptions; + this.comment = comment; + this.addColumns = addColumns; + } + + public Map getSetOptions() { + return setOptions; + } + + public List getRemoveOptions() { + return removeOptions; + } + + public String getComment() { + return comment; + } + + public List getAddColumns() { + return addColumns; + } + + public List getRenameColumns() { + return renameColumns; + } + + public List getDropColumns() { + return dropColumns; + } + + public List getUpdateColumnTypes() { + return updateColumnTypes; + } + + public List getUpdateColumnNullabilities() { + return updateColumnNullabilities; + } + + public List getUpdateColumnComments() { + return updateColumnComments; + } + + public List getUpdateColumnPositions() { + return updateColumnPositions; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java deleted file mode 100644 index d49e4062eea5..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/TableSchema.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest.requests; - -import org.apache.paimon.schema.Schema; -import org.apache.paimon.types.DataField; - -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.util.List; -import java.util.Map; -import java.util.Objects; - -/** - * Wrap the {@link Schema} class to support RESTCatalog. Define a class as: 1. This class in rest - * catalog is easy to maintain. 2. It's easy to manage rest API fields. - */ -public class TableSchema { - private static final String FIELD_FILED_NAME = "fields"; - private static final String FIELD_PARTITION_KEYS_NAME = "partitionKeys"; - private static final String FIELD_PRIMARY_KEYS_NAME = "primaryKeys"; - private static final String FIELD_OPTIONS_NAME = "options"; - private static final String FIELD_COMMENT_NAME = "comment"; - - @JsonProperty(FIELD_FILED_NAME) - private final List fields; - - @JsonProperty(FIELD_PARTITION_KEYS_NAME) - private final List partitionKeys; - - @JsonProperty(FIELD_PRIMARY_KEYS_NAME) - private final List primaryKeys; - - @JsonProperty(FIELD_OPTIONS_NAME) - private final Map options; - - @JsonProperty(FIELD_COMMENT_NAME) - private final String comment; - - @JsonCreator - public TableSchema( - @JsonProperty(FIELD_FILED_NAME) List fields, - @JsonProperty(FIELD_PARTITION_KEYS_NAME) List partitionKeys, - @JsonProperty(FIELD_PRIMARY_KEYS_NAME) List primaryKeys, - @JsonProperty(FIELD_OPTIONS_NAME) Map options, - @JsonProperty(FIELD_COMMENT_NAME) String comment) { - this.fields = fields; - this.partitionKeys = partitionKeys; - this.primaryKeys = primaryKeys; - this.options = options; - this.comment = comment; - } - - public TableSchema(Schema schema) { - this.fields = schema.fields(); - this.partitionKeys = schema.partitionKeys(); - this.primaryKeys = schema.primaryKeys(); - this.options = schema.options(); - this.comment = schema.comment(); - } - - @JsonGetter(FIELD_FILED_NAME) - public List getFields() { - return fields; - } - - @JsonGetter(FIELD_PARTITION_KEYS_NAME) - public List getPartitionKeys() { - return partitionKeys; - } - - @JsonGetter(FIELD_PRIMARY_KEYS_NAME) - public List getPrimaryKeys() { - return primaryKeys; - } - - @JsonGetter(FIELD_OPTIONS_NAME) - public Map getOptions() { - return options; - } - - @JsonGetter(FIELD_COMMENT_NAME) - public String getComment() { - return comment; - } - - @Override - public boolean equals(Object o) { - if (o == null || getClass() != o.getClass()) { - return false; - } else { - TableSchema that = (TableSchema) o; - return Objects.equals(fields, that.fields) - && Objects.equals(partitionKeys, that.partitionKeys) - && Objects.equals(primaryKeys, that.primaryKeys) - && Objects.equals(options, that.options) - && Objects.equals(comment, that.comment); - } - } - - @Override - public int hashCode() { - return Objects.hash(fields, partitionKeys, primaryKeys, options, comment); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java index ca12a1152455..ab1c7b18a59e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java @@ -20,52 +20,49 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.rest.RESTRequest; -import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - /** Request for updating table. */ public class UpdateTableRequest implements RESTRequest { - private static final String FIELD_FROM_IDENTIFIER = "from"; - private static final String FIELD_TO_IDENTIFIER = "to"; - private static final String FIELD_SCHEMA_CHANGES = "changes"; + private static final String FIELD_FROM_IDENTIFIER_NAME = "from"; + private static final String FIELD_TO_IDENTIFIER_NAME = "to"; + private static final String FIELD_SCHEMA_CHANGES_NAME = "changes"; - @JsonProperty(FIELD_FROM_IDENTIFIER) + @JsonProperty(FIELD_FROM_IDENTIFIER_NAME) private Identifier fromIdentifier; - @JsonProperty(FIELD_TO_IDENTIFIER) + @JsonProperty(FIELD_TO_IDENTIFIER_NAME) private Identifier toIdentifier; - @JsonProperty(FIELD_SCHEMA_CHANGES) - private List changes; + @JsonProperty(FIELD_SCHEMA_CHANGES_NAME) + private SchemaChanges changes; @JsonCreator public UpdateTableRequest( - @JsonProperty(FIELD_FROM_IDENTIFIER) Identifier fromIdentifier, - @JsonProperty(FIELD_TO_IDENTIFIER) Identifier toIdentifier, - @JsonProperty(FIELD_SCHEMA_CHANGES) List changes) { + @JsonProperty(FIELD_FROM_IDENTIFIER_NAME) Identifier fromIdentifier, + @JsonProperty(FIELD_TO_IDENTIFIER_NAME) Identifier toIdentifier, + @JsonProperty(FIELD_SCHEMA_CHANGES_NAME) SchemaChanges changes) { this.fromIdentifier = fromIdentifier; this.toIdentifier = toIdentifier; this.changes = changes; } - @JsonGetter(FIELD_FROM_IDENTIFIER) + @JsonGetter(FIELD_FROM_IDENTIFIER_NAME) public Identifier getFromIdentifier() { return fromIdentifier; } - @JsonGetter(FIELD_TO_IDENTIFIER) + @JsonGetter(FIELD_TO_IDENTIFIER_NAME) public Identifier getToIdentifier() { return toIdentifier; } - @JsonGetter(FIELD_SCHEMA_CHANGES) - public List getChanges() { + @JsonGetter(FIELD_SCHEMA_CHANGES_NAME) + public SchemaChanges getChanges() { return changes; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java new file mode 100644 index 000000000000..b58583ee2a13 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.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.serializer; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; + +import java.io.IOException; + +/** Serializer for {@link Identifier}. */ +public class IdentifierSerializer + implements JsonSerializer, JsonDeserializer { + + public static final IdentifierSerializer INSTANCE = new IdentifierSerializer(); + + private static final String FIELD_DATABASE_NAME = "database"; + private static final String FIELD_TABLE_NAME = "table"; + private static final String FIELD_BRANCH_NAME = "branch"; + + @Override + public void serialize(Identifier identifier, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeStringField(FIELD_DATABASE_NAME, identifier.getDatabaseName()); + if (identifier.getTableName() != null) { + generator.writeStringField(FIELD_TABLE_NAME, identifier.getTableName()); + } + if (identifier.getBranchName() != null) { + generator.writeStringField(FIELD_BRANCH_NAME, identifier.getBranchName()); + } + generator.writeEndObject(); + } + + @Override + public Identifier deserialize(JsonNode node) { + JsonNode databaseNode = node.get(FIELD_DATABASE_NAME); + String databaseName = null; + if (databaseNode != null) { + databaseName = databaseNode.asText(); + } + JsonNode tableNode = node.get(FIELD_TABLE_NAME); + String tableName = null; + if (tableNode != null) { + tableName = tableNode.asText(); + } + JsonNode branchNode = node.get(FIELD_BRANCH_NAME); + String branchName = null; + if (branchNode != null) { + branchName = branchNode.asText(); + } + return new Identifier(databaseName, tableName, branchName); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java new file mode 100644 index 000000000000..ed4bde4c9d48 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java @@ -0,0 +1,109 @@ +/* + * 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.serializer; + +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypeJsonParser; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** Serializer for the {@link Schema} class to support RESTCatalog. */ +public class SchemaSerializer implements JsonSerializer, JsonDeserializer { + + public static final SchemaSerializer INSTANCE = new SchemaSerializer(); + + private static final String FIELD_FILED_NAME = "fields"; + private static final String FIELD_PARTITION_KEYS_NAME = "partitionKeys"; + private static final String FIELD_PRIMARY_KEYS_NAME = "primaryKeys"; + private static final String FIELD_OPTIONS_NAME = "options"; + private static final String FIELD_COMMENT_NAME = "comment"; + + @Override + public void serialize(Schema schema, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeArrayFieldStart(FIELD_FILED_NAME); + for (DataField field : schema.fields()) { + field.serializeJson(generator); + } + generator.writeEndArray(); + generator.writeArrayFieldStart(FIELD_PARTITION_KEYS_NAME); + for (String partitionKey : schema.partitionKeys()) { + generator.writeString(partitionKey); + } + generator.writeEndArray(); + generator.writeArrayFieldStart(FIELD_PRIMARY_KEYS_NAME); + for (String partitionKey : schema.primaryKeys()) { + generator.writeString(partitionKey); + } + generator.writeEndArray(); + generator.writeObjectFieldStart(FIELD_OPTIONS_NAME); + for (Map.Entry entry : schema.options().entrySet()) { + generator.writeStringField(entry.getKey(), entry.getValue()); + } + generator.writeEndObject(); + if (schema.comment() != null) { + generator.writeStringField(FIELD_COMMENT_NAME, schema.comment()); + } + generator.writeEndObject(); + } + + @Override + public Schema deserialize(JsonNode node) { + Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); + List fields = new ArrayList<>(); + while (fieldJsons.hasNext()) { + fields.add(DataTypeJsonParser.parseDataField(fieldJsons.next())); + } + Iterator partitionJsons = node.get(FIELD_PARTITION_KEYS_NAME).elements(); + List partitionKeys = new ArrayList<>(); + while (partitionJsons.hasNext()) { + partitionKeys.add(partitionJsons.next().asText()); + } + + Iterator primaryJsons = node.get(FIELD_PRIMARY_KEYS_NAME).elements(); + List primaryKeys = new ArrayList<>(); + while (primaryJsons.hasNext()) { + primaryKeys.add(primaryJsons.next().asText()); + } + JsonNode optionsJson = node.get(FIELD_OPTIONS_NAME); + Map options = new HashMap<>(); + Iterator optionsKeys = optionsJson.fieldNames(); + while (optionsKeys.hasNext()) { + String key = optionsKeys.next(); + options.put(key, optionsJson.get(key).asText()); + } + JsonNode commentNode = node.get(FIELD_COMMENT_NAME); + String comment = null; + if (commentNode != null) { + comment = commentNode.asText(); + } + return new Schema(fields, partitionKeys, primaryKeys, options, comment); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchemaSerializer.java similarity index 97% rename from paimon-core/src/main/java/org/apache/paimon/schema/SchemaSerializer.java rename to paimon-core/src/main/java/org/apache/paimon/schema/TableSchemaSerializer.java index 4fb28359c92f..f6b6c7302f2c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchemaSerializer.java @@ -39,10 +39,10 @@ import static org.apache.paimon.schema.TableSchema.PAIMON_08_VERSION; /** A {@link JsonSerializer} for {@link TableSchema}. */ -public class SchemaSerializer +public class TableSchemaSerializer implements JsonSerializer, JsonDeserializer { - public static final SchemaSerializer INSTANCE = new SchemaSerializer(); + public static final TableSchemaSerializer INSTANCE = new TableSchemaSerializer(); @Override public void serialize(TableSchema tableSchema, JsonGenerator generator) throws IOException { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java index edc6dac5f992..7c36f4261347 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java @@ -18,8 +18,8 @@ package org.apache.paimon.utils; -import org.apache.paimon.schema.SchemaSerializer; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.schema.TableSchemaSerializer; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeJsonParser; @@ -181,7 +181,10 @@ public T deserialize(JsonParser parser, DeserializationContext context) private static Module createPaimonJacksonModule() { SimpleModule module = new SimpleModule("Paimon"); registerJsonObjects( - module, TableSchema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); + module, + TableSchema.class, + TableSchemaSerializer.INSTANCE, + TableSchemaSerializer.INSTANCE); registerJsonObjects( module, DataField.class, 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 a701b54c9dbd..ddf78d59c8ae 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 @@ -22,6 +22,7 @@ import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.requests.CreateTableRequest; +import org.apache.paimon.rest.requests.UpdateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; @@ -111,4 +112,10 @@ public static CreateTableRequest createTableRequest(String name) { .build(); return new CreateTableRequest(identifier, schema); } + + public static UpdateTableRequest updateTableRequest(String fromTableName, String toTableName) { + Identifier fromIdentifier = Identifier.create(databaseName(), fromTableName); + Identifier toIdentifier = Identifier.create(databaseName(), toTableName); + return new UpdateTableRequest(fromIdentifier, toIdentifier, null); + } } 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 6da61932a6e7..987bb1cd4c5f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.requests.CreateTableRequest; +import org.apache.paimon.rest.requests.UpdateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; @@ -135,9 +136,7 @@ public void createTableRequestParseTest() throws Exception { CreateTableRequest request = MockRESTMessage.createTableRequest("t1"); String requestStr = mapper.writeValueAsString(request); CreateTableRequest parseData = mapper.readValue(requestStr, CreateTableRequest.class); - assertEquals(request.getDatabaseName(), parseData.getDatabaseName()); - assertEquals(request.getTableName(), parseData.getTableName()); - assertEquals(request.getBranchName(), parseData.getBranchName()); + assertEquals(request.getIdentifier(), parseData.getIdentifier()); assertEquals(request.getSchema(), parseData.getSchema()); } @@ -158,4 +157,13 @@ public void dataFieldParseTest() throws Exception { assertEquals(type, parseData.type()); assertEquals(descStr, parseData.description()); } + + @Test + public void updateTableRequestParseTest() throws Exception { + UpdateTableRequest request = MockRESTMessage.updateTableRequest("t1", "t2"); + String requestStr = mapper.writeValueAsString(request); + UpdateTableRequest parseData = mapper.readValue(requestStr, UpdateTableRequest.class); + assertEquals(request.getFromIdentifier(), parseData.getFromIdentifier()); + assertEquals(request.getToIdentifier(), parseData.getToIdentifier()); + } } From d1cbfeac3bac7036b67340d1544908c6e3516a76 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 23 Dec 2024 16:41:36 +0800 Subject: [PATCH 16/23] add serializer to support schema change --- .../rest/serializer/AddColumnSerializer.java | 4 ++ .../rest/serializer/DropColumnSerializer.java | 44 +++++++++++++++ .../rest/serializer/MoveSerializer.java | 4 ++ .../serializer/RenameColumnSerializer.java | 53 +++++++++++++++++++ .../UpdateColumnCommentSerializer.java | 44 +++++++++++++++ .../UpdateColumnNullabilitySerializer.java | 47 ++++++++++++++++ .../UpdateColumnTypeSerializer.java | 53 +++++++++++++++++++ .../open/api/RESTCatalogController.java | 2 +- 8 files changed, 250 insertions(+), 1 deletion(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java new file mode 100644 index 000000000000..14eb2d70a2b6 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java @@ -0,0 +1,4 @@ +package org.apache.paimon.rest.serializer; + +public class AddColumnSerializer { +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java new file mode 100644 index 000000000000..fb9bbd7f42b4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java @@ -0,0 +1,44 @@ +package org.apache.paimon.rest.serializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +/** Serializer for {@link SchemaChange.RenameColumn}. */ +public class RenameColumnSerializer implements JsonSerializer, JsonDeserializer { + + public static final RenameColumnSerializer INSTANCE = new RenameColumnSerializer(); + + private static final String FIELD_FILED_NAME = "field-names"; + private static final String FIELD_NEW_NAME = "new-name"; + @Override + public SchemaChange.RenameColumn deserialize(JsonNode node) { + Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); + List fieldNames = new ArrayList<>(); + while (fieldJsons.hasNext()) { + fieldNames.add(fieldJsons.next().asText()); + } + String newName = node.has(FIELD_NEW_NAME) ? node.get(FIELD_NEW_NAME).asText() : null; + return (SchemaChange.RenameColumn) SchemaChange.renameColumn(fieldNames.toArray(new String[0]), newName); + } + + @Override + public void serialize(SchemaChange.RenameColumn renameColumn, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeArrayFieldStart(FIELD_FILED_NAME); + for (String fieldName : renameColumn.fieldNames()) { + generator.writeString(fieldName); + } + generator.writeEndArray(); + if(renameColumn.newName() != null) { + generator.writeStringField(FIELD_NEW_NAME, renameColumn.newName()); + } + generator.writeEndObject(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java new file mode 100644 index 000000000000..0e0d43fea2b5 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java @@ -0,0 +1,4 @@ +package org.apache.paimon.rest.serializer; + +public class MoveSerializer { +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java new file mode 100644 index 000000000000..ad673d242da4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java @@ -0,0 +1,53 @@ +package org.apache.paimon.rest.serializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeJsonParser; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +public class AddColumnSerializer implements JsonSerializer, JsonDeserializer { + + public static final AddColumnSerializer INSTANCE = new AddColumnSerializer(); + + private static final String FIELD_FILED_NAME = "field-names"; + private static final String FIELD_DATA_TYPE_NAME = "data-types"; + private static final String FIELD_DESCRIPTION_NAME = "desc"; + private static final String FIELD_MOVE_NAME = "move"; + @Override + public SchemaChange.AddColumn deserialize(JsonNode node) { + Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); + List fieldNames = new ArrayList<>(); + while (fieldJsons.hasNext()) { + fieldNames.add(fieldJsons.next().asText()); + } + DataType dataType = DataTypeJsonParser.parseDataType(node.get(FIELD_DATA_TYPE_NAME)); + String description = node.has(FIELD_DESCRIPTION_NAME) ? node.get(FIELD_DESCRIPTION_NAME).asText() : null; + SchemaChange.Move move = node.has(FIELD_MOVE_NAME) ? MoveSerializer.INSTANCE.deserialize(node.get(FIELD_MOVE_NAME)) : null; + return (SchemaChange.AddColumn) SchemaChange.addColumn(fieldNames.toArray(new String[0]), dataType, description, move); + } + + @Override + public void serialize(SchemaChange.AddColumn addColumn, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeArrayFieldStart(FIELD_FILED_NAME); + for (String fieldName : addColumn.fieldNames()) { + generator.writeString(fieldName); + } + generator.writeEndArray(); + generator.writeObjectField(FIELD_DATA_TYPE_NAME, addColumn.dataType()); + if(addColumn.description() != null) { + generator.writeStringField(FIELD_DESCRIPTION_NAME, addColumn.description()); + } + if(addColumn.move() != null) { + generator.writeObjectField(FIELD_MOVE_NAME, addColumn.move()); + } + generator.writeEndObject(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java new file mode 100644 index 000000000000..2c85797bfab3 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java @@ -0,0 +1,44 @@ +package org.apache.paimon.rest.serializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeJsonParser; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +/** Serializer for {@link SchemaChange.UpdateColumnNullability}. */ +public class UpdateColumnNullabilitySerializer implements JsonSerializer, JsonDeserializer { + + public static final UpdateColumnNullabilitySerializer INSTANCE = new UpdateColumnNullabilitySerializer(); + + private static final String FIELD_FILED = "field-names"; + private static final String FIELD_NEW_NULLABILITY = "new-nullability"; + @Override + public SchemaChange.UpdateColumnNullability deserialize(JsonNode node) { + Iterator fieldJsons = node.get(FIELD_FILED).elements(); + List fieldNames = new ArrayList<>(); + while (fieldJsons.hasNext()) { + fieldNames.add(fieldJsons.next().asText()); + } + boolean newNullability = node.get(FIELD_NEW_NULLABILITY).asBoolean(); + return (SchemaChange.UpdateColumnNullability) SchemaChange.updateColumnNullability(fieldNames.toArray(new String[0]), newNullability); + } + + @Override + public void serialize(SchemaChange.UpdateColumnNullability updateColumnNullability, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeArrayFieldStart(FIELD_FILED); + for (String fieldName : updateColumnNullability.fieldNames()) { + generator.writeString(fieldName); + } + generator.writeEndArray(); + generator.writeBooleanField(FIELD_NEW_NULLABILITY, updateColumnNullability.newNullability()); + generator.writeEndObject(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java new file mode 100644 index 000000000000..eda9be5a3ee4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java @@ -0,0 +1,47 @@ +package org.apache.paimon.rest.serializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeJsonParser; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +/** Serializer for {@link SchemaChange.UpdateColumnType}. */ +public class UpdateColumnTypeSerializer implements JsonSerializer, JsonDeserializer { + + public static final UpdateColumnTypeSerializer INSTANCE = new UpdateColumnTypeSerializer(); + + private static final String FIELD_FILED = "field-names"; + private static final String FIELD_NEW_DATA_TYPE = "new-data-types"; + private static final String FIELD_KEEP_NULLABILITY = "keep-nullability"; + @Override + public SchemaChange.UpdateColumnType deserialize(JsonNode node) { + Iterator fieldJsons = node.get(FIELD_FILED).elements(); + List fieldNames = new ArrayList<>(); + while (fieldJsons.hasNext()) { + fieldNames.add(fieldJsons.next().asText()); + } + DataType dataType = DataTypeJsonParser.parseDataType(node.get(FIELD_NEW_DATA_TYPE)); + boolean keepNullability = node.has(FIELD_KEEP_NULLABILITY) ? node.get(FIELD_KEEP_NULLABILITY).asBoolean() : false; + return (SchemaChange.UpdateColumnType) SchemaChange.updateColumnType(fieldNames.toArray(new String[0]), dataType, keepNullability); + } + + @Override + public void serialize(SchemaChange.UpdateColumnType updateColumnType, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeArrayFieldStart(FIELD_FILED); + for (String fieldName : updateColumnType.fieldNames()) { + generator.writeString(fieldName); + } + generator.writeEndArray(); + generator.writeObjectField(FIELD_NEW_DATA_TYPE, updateColumnType.newDataType()); + generator.writeBooleanField(FIELD_KEEP_NULLABILITY, updateColumnType.keepNullability()); + generator.writeEndObject(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java new file mode 100644 index 000000000000..ad673d242da4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java @@ -0,0 +1,53 @@ +package org.apache.paimon.rest.serializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeJsonParser; +import org.apache.paimon.utils.JsonDeserializer; +import org.apache.paimon.utils.JsonSerializer; + +public class AddColumnSerializer implements JsonSerializer, JsonDeserializer { + + public static final AddColumnSerializer INSTANCE = new AddColumnSerializer(); + + private static final String FIELD_FILED_NAME = "field-names"; + private static final String FIELD_DATA_TYPE_NAME = "data-types"; + private static final String FIELD_DESCRIPTION_NAME = "desc"; + private static final String FIELD_MOVE_NAME = "move"; + @Override + public SchemaChange.AddColumn deserialize(JsonNode node) { + Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); + List fieldNames = new ArrayList<>(); + while (fieldJsons.hasNext()) { + fieldNames.add(fieldJsons.next().asText()); + } + DataType dataType = DataTypeJsonParser.parseDataType(node.get(FIELD_DATA_TYPE_NAME)); + String description = node.has(FIELD_DESCRIPTION_NAME) ? node.get(FIELD_DESCRIPTION_NAME).asText() : null; + SchemaChange.Move move = node.has(FIELD_MOVE_NAME) ? MoveSerializer.INSTANCE.deserialize(node.get(FIELD_MOVE_NAME)) : null; + return (SchemaChange.AddColumn) SchemaChange.addColumn(fieldNames.toArray(new String[0]), dataType, description, move); + } + + @Override + public void serialize(SchemaChange.AddColumn addColumn, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeArrayFieldStart(FIELD_FILED_NAME); + for (String fieldName : addColumn.fieldNames()) { + generator.writeString(fieldName); + } + generator.writeEndArray(); + generator.writeObjectField(FIELD_DATA_TYPE_NAME, addColumn.dataType()); + if(addColumn.description() != null) { + generator.writeStringField(FIELD_DESCRIPTION_NAME, addColumn.description()); + } + if(addColumn.move() != null) { + generator.writeObjectField(FIELD_MOVE_NAME, addColumn.move()); + } + generator.writeEndObject(); + } +} 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 21a5ec7a5fec..6e9b8c5f02d9 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 @@ -197,7 +197,7 @@ public AlterDatabaseResponse alterDatabase( responseCode = "500", content = {@Content(schema = @Schema())}) }) - @GetMapping("/v1/{prefix}/databases/{database}") + @GetMapping("/v1/{prefix}/databases/{database}/tables") public ListTablesResponse listTables( @PathVariable String prefix, @PathVariable String database) { return new ListTablesResponse(ImmutableList.of("user")); From 525c2b1e670fd05195fe45bdfe40e7250801a7db Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 23 Dec 2024 17:45:20 +0800 Subject: [PATCH 17/23] support json format inner object format object json key --- .../org/apache/paimon/types/DataField.java | 19 ++ .../org/apache/paimon/catalog/Identifier.java | 32 ++- .../apache/paimon/rest/RESTObjectMapper.java | 11 -- .../rest/requests/CreateTableRequest.java | 8 +- .../paimon/rest/requests/SchemaChanges.java | 113 +++++++++-- .../rest/serializer/AddColumnSerializer.java | 4 - .../rest/serializer/DropColumnSerializer.java | 44 ----- .../rest/serializer/IdentifierSerializer.java | 72 ------- .../rest/serializer/MoveSerializer.java | 4 - .../serializer/RenameColumnSerializer.java | 53 ----- .../rest/serializer/SchemaSerializer.java | 109 ----------- .../UpdateColumnCommentSerializer.java | 44 ----- .../UpdateColumnNullabilitySerializer.java | 47 ----- .../UpdateColumnTypeSerializer.java | 53 ----- .../java/org/apache/paimon/schema/Schema.java | 53 ++++- .../apache/paimon/schema/SchemaChange.java | 184 +++++++++++++++++- .../apache/paimon/rest/MockRESTMessage.java | 50 ++++- 17 files changed, 421 insertions(+), 479 deletions(-) delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java index 209118023ba5..a7fccd63c1e5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.Public; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; import javax.annotation.Nullable; @@ -37,6 +38,7 @@ * @since 0.4.0 */ @Public +@JsonIgnoreProperties(ignoreUnknown = true) public final class DataField implements Serializable { private static final long serialVersionUID = 1L; @@ -76,6 +78,23 @@ public DataType type() { return type; } + public int getId() { + return id; + } + + public String getName() { + return name; + } + + public DataType getType() { + return type; + } + + @Nullable + public String getDescription() { + return description; + } + public DataField newId(int newid) { return new DataField(newid, name, type, description); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java index 01456f0b3ae1..301a92fbc639 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java @@ -26,6 +26,11 @@ import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StringUtils; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + import javax.annotation.Nullable; import java.io.Serializable; @@ -40,10 +45,15 @@ * @since 0.4.0 */ @Public +@JsonIgnoreProperties(ignoreUnknown = true) public class Identifier implements Serializable { private static final long serialVersionUID = 1L; + private static final String FIELD_DATABASE_NAME = "database"; + private static final String FIELD_TABLE_NAME = "table"; + private static final String FIELD_BRANCH_NAME = "branch"; + public static final RowType SCHEMA = new RowType( false, @@ -53,11 +63,17 @@ public class Identifier implements Serializable { public static final String UNKNOWN_DATABASE = "unknown"; + @JsonProperty(FIELD_DATABASE_NAME) private final String database; + private final String object; + @JsonProperty(FIELD_TABLE_NAME) private transient String table; + + @JsonProperty(FIELD_BRANCH_NAME) private transient String branch; + private transient String systemTable; public Identifier(String database, String object) { @@ -65,7 +81,11 @@ public Identifier(String database, String object) { this.object = object; } - public Identifier(String database, String table, @Nullable String branch) { + @JsonCreator + public Identifier( + @JsonProperty(FIELD_DATABASE_NAME) String database, + @JsonProperty(FIELD_TABLE_NAME) String table, + @JsonProperty(FIELD_BRANCH_NAME) @Nullable String branch) { this(database, table, branch, null); } @@ -89,40 +109,48 @@ public Identifier( this.systemTable = systemTable; } + @JsonGetter(FIELD_DATABASE_NAME) public String getDatabaseName() { return database; } + @JsonIgnoreProperties public String getObjectName() { return object; } + @JsonIgnoreProperties public String getFullName() { return UNKNOWN_DATABASE.equals(this.database) ? object : String.format("%s.%s", database, object); } + @JsonGetter(FIELD_TABLE_NAME) public String getTableName() { splitObjectName(); return table; } + @JsonGetter(FIELD_BRANCH_NAME) public @Nullable String getBranchName() { splitObjectName(); return branch; } + @JsonIgnoreProperties public String getBranchNameOrDefault() { String branch = getBranchName(); return branch == null ? BranchManager.DEFAULT_MAIN_BRANCH : branch; } + @JsonIgnoreProperties public @Nullable String getSystemTableName() { splitObjectName(); return systemTable; } + @JsonIgnoreProperties public boolean isSystemTable() { return getSystemTableName() != null; } @@ -158,10 +186,12 @@ private void splitObjectName() { } } + @JsonIgnoreProperties public String getEscapedFullName() { return getEscapedFullName('`'); } + @JsonIgnoreProperties public String getEscapedFullName(char escapeChar) { return String.format( "%c%s%c.%c%s%c", escapeChar, database, escapeChar, escapeChar, object, escapeChar); 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 b0f9ba7b83e2..fa66e754ac85 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -18,10 +18,6 @@ package org.apache.paimon.rest; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.rest.serializer.IdentifierSerializer; -import org.apache.paimon.rest.serializer.SchemaSerializer; -import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeJsonParser; @@ -53,15 +49,8 @@ public static Module createPaimonRestJacksonModule() { DataField.class, DataField::serializeJson, DataTypeJsonParser::parseDataField); - registerJsonObjects( - module, Schema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); registerJsonObjects( module, DataType.class, DataType::serializeJson, DataTypeJsonParser::parseDataType); - registerJsonObjects( - module, - Identifier.class, - IdentifierSerializer.INSTANCE, - IdentifierSerializer.INSTANCE); return module; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java index 3f80e0fee5f9..f566db3dcc6a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java @@ -29,10 +29,10 @@ /** Request for creating table. */ public class CreateTableRequest implements RESTRequest { - private static final String FIELD_IDENTIFIER_NAME = "identifier"; + private static final String FIELD_IDENTIFIER = "identifier"; private static final String FIELD_SCHEMA = "schema"; - @JsonProperty(FIELD_IDENTIFIER_NAME) + @JsonProperty(FIELD_IDENTIFIER) private Identifier identifier; @JsonProperty(FIELD_SCHEMA) @@ -40,12 +40,12 @@ public class CreateTableRequest implements RESTRequest { @JsonCreator public CreateTableRequest( - @JsonProperty(FIELD_IDENTIFIER_NAME) Identifier identifier, + @JsonProperty(FIELD_IDENTIFIER) Identifier identifier, @JsonProperty(FIELD_SCHEMA) Schema schema) { this.schema = schema; } - @JsonGetter(FIELD_IDENTIFIER_NAME) + @JsonGetter(FIELD_IDENTIFIER) public Identifier getIdentifier() { return identifier; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java index 698525f604b9..5abe48049a36 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java @@ -20,39 +20,78 @@ import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; +/** Schema changes to serialize List of SchemaChange . */ +@JsonIgnoreProperties(ignoreUnknown = true) public class SchemaChanges { - private static final String FIELD_SET_OPTIONS_NAME = "set-options"; - private static final String FIELD_REMOVE_OPTIONS_NAME = "remove-options"; - private static final String FIELD_COMMENT_NAME = "comment"; - private static final String FIELD_ADD_COLUMNS_NAME = "add-columns"; - private static final String FIELD_RENAME_COLUMNS_NAME = "rename-columns"; + private static final String FIELD_SET_OPTIONS = "set-options"; + private static final String FIELD_REMOVE_OPTIONS = "remove-options"; + private static final String FIELD_COMMENT = "comment"; + private static final String FIELD_ADD_COLUMNS = "add-columns"; + private static final String FIELD_RENAME_COLUMNS = "rename-columns"; + private static final String FIELD_DROP_COLUMNS = "drop-columns"; + private static final String FIELD_UPDATE_COLUMN_TYPES = "update-column-types"; + private static final String FIELD_UPDATE_COLUMN_NULLABILITIES = "update-column-nullabilities"; + private static final String FIELD_UPDATE_COLUMN_COMMENTS = "update-column-comments"; + private static final String FIELD_UPDATE_COLUMN_POSITIONS = "update-column-positions"; + + @JsonProperty(FIELD_SET_OPTIONS) private Map setOptions; + + @JsonProperty(FIELD_REMOVE_OPTIONS) private List removeOptions; + + @JsonProperty(FIELD_COMMENT) private String comment; + + @JsonProperty(FIELD_ADD_COLUMNS) private List addColumns; + + @JsonProperty(FIELD_RENAME_COLUMNS) private List renameColumns; - private List dropColumns; + + @JsonProperty(FIELD_DROP_COLUMNS) + private List dropColumns; + + @JsonProperty(FIELD_UPDATE_COLUMN_TYPES) private List updateColumnTypes; + + @JsonProperty(FIELD_UPDATE_COLUMN_NULLABILITIES) private List updateColumnNullabilities; + + @JsonProperty(FIELD_UPDATE_COLUMN_COMMENTS) private List updateColumnComments; - private List updateColumnPositions; + @JsonProperty(FIELD_UPDATE_COLUMN_POSITIONS) + private List updateColumnPositions; + + @JsonCreator public SchemaChanges( - Map setOptions, - List removeOptions, - String comment, - List addColumns, - List renameColumns, - List dropColumns, - List updateColumnTypes, - List updateColumnNullabilities, - List updateColumnComments, - List updateColumnPositions) { + @JsonProperty(FIELD_SET_OPTIONS) Map setOptions, + @JsonProperty(FIELD_REMOVE_OPTIONS) List removeOptions, + @JsonProperty(FIELD_COMMENT) String comment, + @JsonProperty(FIELD_ADD_COLUMNS) List addColumns, + @JsonProperty(FIELD_RENAME_COLUMNS) List renameColumns, + @JsonProperty(FIELD_DROP_COLUMNS) List dropColumns, + @JsonProperty(FIELD_UPDATE_COLUMN_TYPES) + List updateColumnTypes, + @JsonProperty(FIELD_UPDATE_COLUMN_NULLABILITIES) + List updateColumnNullabilities, + @JsonProperty(FIELD_UPDATE_COLUMN_COMMENTS) + List updateColumnComments, + @JsonProperty(FIELD_UPDATE_COLUMN_POSITIONS) + List updateColumnPositions) { this.setOptions = setOptions; this.removeOptions = removeOptions; this.comment = comment; @@ -66,10 +105,16 @@ public SchemaChanges( } public SchemaChanges(List changes) { - Map setOptions = null; + Map setOptions = new HashMap<>(); List removeOptions = new ArrayList<>(); String comment = null; List addColumns = new ArrayList<>(); + List renameColumns = new ArrayList<>(); + List dropColumns = new ArrayList<>(); + List updateColumnTypes = new ArrayList<>(); + List updateColumnNullabilities = new ArrayList<>(); + List updateColumnComments = new ArrayList<>(); + List updateColumnPositions = new ArrayList<>(); for (SchemaChange change : changes) { if (change instanceof SchemaChange.SetOption) { setOptions.put( @@ -81,51 +126,79 @@ public SchemaChanges(List changes) { comment = ((SchemaChange.UpdateComment) change).comment(); } else if (change instanceof SchemaChange.AddColumn) { addColumns.add((SchemaChange.AddColumn) change); + } else if (change instanceof SchemaChange.RenameColumn) { + renameColumns.add((SchemaChange.RenameColumn) change); + } else if (change instanceof SchemaChange.DropColumn) { + dropColumns.addAll(Arrays.asList(((SchemaChange.DropColumn) change).fieldNames())); + } else if (change instanceof SchemaChange.UpdateColumnType) { + updateColumnTypes.add((SchemaChange.UpdateColumnType) change); + } else if (change instanceof SchemaChange.UpdateColumnNullability) { + updateColumnNullabilities.add((SchemaChange.UpdateColumnNullability) change); + } else if (change instanceof SchemaChange.UpdateColumnComment) { + updateColumnComments.add((SchemaChange.UpdateColumnComment) change); + } else if (change instanceof SchemaChange.UpdateColumnPosition) { + updateColumnPositions.add(((SchemaChange.UpdateColumnPosition) change).move()); } } this.setOptions = setOptions; this.removeOptions = removeOptions; this.comment = comment; this.addColumns = addColumns; + this.renameColumns = renameColumns; + this.dropColumns = dropColumns; + this.updateColumnTypes = updateColumnTypes; + this.updateColumnNullabilities = updateColumnNullabilities; + this.updateColumnComments = updateColumnComments; + this.updateColumnPositions = updateColumnPositions; } + @JsonGetter(FIELD_SET_OPTIONS) public Map getSetOptions() { return setOptions; } + @JsonGetter(FIELD_REMOVE_OPTIONS) public List getRemoveOptions() { return removeOptions; } + @JsonGetter(FIELD_COMMENT) public String getComment() { return comment; } + @JsonGetter(FIELD_ADD_COLUMNS) public List getAddColumns() { return addColumns; } + @JsonGetter(FIELD_RENAME_COLUMNS) public List getRenameColumns() { return renameColumns; } - public List getDropColumns() { + @JsonGetter(FIELD_DROP_COLUMNS) + public List getDropColumns() { return dropColumns; } + @JsonGetter(FIELD_UPDATE_COLUMN_TYPES) public List getUpdateColumnTypes() { return updateColumnTypes; } + @JsonGetter(FIELD_UPDATE_COLUMN_NULLABILITIES) public List getUpdateColumnNullabilities() { return updateColumnNullabilities; } + @JsonGetter(FIELD_UPDATE_COLUMN_COMMENTS) public List getUpdateColumnComments() { return updateColumnComments; } - public List getUpdateColumnPositions() { + @JsonGetter(FIELD_UPDATE_COLUMN_POSITIONS) + public List getUpdateColumnPositions() { return updateColumnPositions; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java deleted file mode 100644 index 14eb2d70a2b6..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/AddColumnSerializer.java +++ /dev/null @@ -1,4 +0,0 @@ -package org.apache.paimon.rest.serializer; - -public class AddColumnSerializer { -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java deleted file mode 100644 index fb9bbd7f42b4..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/DropColumnSerializer.java +++ /dev/null @@ -1,44 +0,0 @@ -package org.apache.paimon.rest.serializer; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import org.apache.paimon.schema.SchemaChange; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -/** Serializer for {@link SchemaChange.RenameColumn}. */ -public class RenameColumnSerializer implements JsonSerializer, JsonDeserializer { - - public static final RenameColumnSerializer INSTANCE = new RenameColumnSerializer(); - - private static final String FIELD_FILED_NAME = "field-names"; - private static final String FIELD_NEW_NAME = "new-name"; - @Override - public SchemaChange.RenameColumn deserialize(JsonNode node) { - Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); - List fieldNames = new ArrayList<>(); - while (fieldJsons.hasNext()) { - fieldNames.add(fieldJsons.next().asText()); - } - String newName = node.has(FIELD_NEW_NAME) ? node.get(FIELD_NEW_NAME).asText() : null; - return (SchemaChange.RenameColumn) SchemaChange.renameColumn(fieldNames.toArray(new String[0]), newName); - } - - @Override - public void serialize(SchemaChange.RenameColumn renameColumn, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeArrayFieldStart(FIELD_FILED_NAME); - for (String fieldName : renameColumn.fieldNames()) { - generator.writeString(fieldName); - } - generator.writeEndArray(); - if(renameColumn.newName() != null) { - generator.writeStringField(FIELD_NEW_NAME, renameColumn.newName()); - } - generator.writeEndObject(); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java deleted file mode 100644 index b58583ee2a13..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/IdentifierSerializer.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest.serializer; - -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import java.io.IOException; - -/** Serializer for {@link Identifier}. */ -public class IdentifierSerializer - implements JsonSerializer, JsonDeserializer { - - public static final IdentifierSerializer INSTANCE = new IdentifierSerializer(); - - private static final String FIELD_DATABASE_NAME = "database"; - private static final String FIELD_TABLE_NAME = "table"; - private static final String FIELD_BRANCH_NAME = "branch"; - - @Override - public void serialize(Identifier identifier, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeStringField(FIELD_DATABASE_NAME, identifier.getDatabaseName()); - if (identifier.getTableName() != null) { - generator.writeStringField(FIELD_TABLE_NAME, identifier.getTableName()); - } - if (identifier.getBranchName() != null) { - generator.writeStringField(FIELD_BRANCH_NAME, identifier.getBranchName()); - } - generator.writeEndObject(); - } - - @Override - public Identifier deserialize(JsonNode node) { - JsonNode databaseNode = node.get(FIELD_DATABASE_NAME); - String databaseName = null; - if (databaseNode != null) { - databaseName = databaseNode.asText(); - } - JsonNode tableNode = node.get(FIELD_TABLE_NAME); - String tableName = null; - if (tableNode != null) { - tableName = tableNode.asText(); - } - JsonNode branchNode = node.get(FIELD_BRANCH_NAME); - String branchName = null; - if (branchNode != null) { - branchName = branchNode.asText(); - } - return new Identifier(databaseName, tableName, branchName); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java deleted file mode 100644 index 0e0d43fea2b5..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/MoveSerializer.java +++ /dev/null @@ -1,4 +0,0 @@ -package org.apache.paimon.rest.serializer; - -public class MoveSerializer { -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java deleted file mode 100644 index ad673d242da4..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/RenameColumnSerializer.java +++ /dev/null @@ -1,53 +0,0 @@ -package org.apache.paimon.rest.serializer; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import org.apache.paimon.schema.SchemaChange; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeJsonParser; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -public class AddColumnSerializer implements JsonSerializer, JsonDeserializer { - - public static final AddColumnSerializer INSTANCE = new AddColumnSerializer(); - - private static final String FIELD_FILED_NAME = "field-names"; - private static final String FIELD_DATA_TYPE_NAME = "data-types"; - private static final String FIELD_DESCRIPTION_NAME = "desc"; - private static final String FIELD_MOVE_NAME = "move"; - @Override - public SchemaChange.AddColumn deserialize(JsonNode node) { - Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); - List fieldNames = new ArrayList<>(); - while (fieldJsons.hasNext()) { - fieldNames.add(fieldJsons.next().asText()); - } - DataType dataType = DataTypeJsonParser.parseDataType(node.get(FIELD_DATA_TYPE_NAME)); - String description = node.has(FIELD_DESCRIPTION_NAME) ? node.get(FIELD_DESCRIPTION_NAME).asText() : null; - SchemaChange.Move move = node.has(FIELD_MOVE_NAME) ? MoveSerializer.INSTANCE.deserialize(node.get(FIELD_MOVE_NAME)) : null; - return (SchemaChange.AddColumn) SchemaChange.addColumn(fieldNames.toArray(new String[0]), dataType, description, move); - } - - @Override - public void serialize(SchemaChange.AddColumn addColumn, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeArrayFieldStart(FIELD_FILED_NAME); - for (String fieldName : addColumn.fieldNames()) { - generator.writeString(fieldName); - } - generator.writeEndArray(); - generator.writeObjectField(FIELD_DATA_TYPE_NAME, addColumn.dataType()); - if(addColumn.description() != null) { - generator.writeStringField(FIELD_DESCRIPTION_NAME, addColumn.description()); - } - if(addColumn.move() != null) { - generator.writeObjectField(FIELD_MOVE_NAME, addColumn.move()); - } - generator.writeEndObject(); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java deleted file mode 100644 index ed4bde4c9d48..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/SchemaSerializer.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.rest.serializer; - -import org.apache.paimon.schema.Schema; -import org.apache.paimon.types.DataField; -import org.apache.paimon.types.DataTypeJsonParser; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -/** Serializer for the {@link Schema} class to support RESTCatalog. */ -public class SchemaSerializer implements JsonSerializer, JsonDeserializer { - - public static final SchemaSerializer INSTANCE = new SchemaSerializer(); - - private static final String FIELD_FILED_NAME = "fields"; - private static final String FIELD_PARTITION_KEYS_NAME = "partitionKeys"; - private static final String FIELD_PRIMARY_KEYS_NAME = "primaryKeys"; - private static final String FIELD_OPTIONS_NAME = "options"; - private static final String FIELD_COMMENT_NAME = "comment"; - - @Override - public void serialize(Schema schema, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeArrayFieldStart(FIELD_FILED_NAME); - for (DataField field : schema.fields()) { - field.serializeJson(generator); - } - generator.writeEndArray(); - generator.writeArrayFieldStart(FIELD_PARTITION_KEYS_NAME); - for (String partitionKey : schema.partitionKeys()) { - generator.writeString(partitionKey); - } - generator.writeEndArray(); - generator.writeArrayFieldStart(FIELD_PRIMARY_KEYS_NAME); - for (String partitionKey : schema.primaryKeys()) { - generator.writeString(partitionKey); - } - generator.writeEndArray(); - generator.writeObjectFieldStart(FIELD_OPTIONS_NAME); - for (Map.Entry entry : schema.options().entrySet()) { - generator.writeStringField(entry.getKey(), entry.getValue()); - } - generator.writeEndObject(); - if (schema.comment() != null) { - generator.writeStringField(FIELD_COMMENT_NAME, schema.comment()); - } - generator.writeEndObject(); - } - - @Override - public Schema deserialize(JsonNode node) { - Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); - List fields = new ArrayList<>(); - while (fieldJsons.hasNext()) { - fields.add(DataTypeJsonParser.parseDataField(fieldJsons.next())); - } - Iterator partitionJsons = node.get(FIELD_PARTITION_KEYS_NAME).elements(); - List partitionKeys = new ArrayList<>(); - while (partitionJsons.hasNext()) { - partitionKeys.add(partitionJsons.next().asText()); - } - - Iterator primaryJsons = node.get(FIELD_PRIMARY_KEYS_NAME).elements(); - List primaryKeys = new ArrayList<>(); - while (primaryJsons.hasNext()) { - primaryKeys.add(primaryJsons.next().asText()); - } - JsonNode optionsJson = node.get(FIELD_OPTIONS_NAME); - Map options = new HashMap<>(); - Iterator optionsKeys = optionsJson.fieldNames(); - while (optionsKeys.hasNext()) { - String key = optionsKeys.next(); - options.put(key, optionsJson.get(key).asText()); - } - JsonNode commentNode = node.get(FIELD_COMMENT_NAME); - String comment = null; - if (commentNode != null) { - comment = commentNode.asText(); - } - return new Schema(fields, partitionKeys, primaryKeys, options, comment); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java deleted file mode 100644 index 2c85797bfab3..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnCommentSerializer.java +++ /dev/null @@ -1,44 +0,0 @@ -package org.apache.paimon.rest.serializer; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import org.apache.paimon.schema.SchemaChange; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeJsonParser; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -/** Serializer for {@link SchemaChange.UpdateColumnNullability}. */ -public class UpdateColumnNullabilitySerializer implements JsonSerializer, JsonDeserializer { - - public static final UpdateColumnNullabilitySerializer INSTANCE = new UpdateColumnNullabilitySerializer(); - - private static final String FIELD_FILED = "field-names"; - private static final String FIELD_NEW_NULLABILITY = "new-nullability"; - @Override - public SchemaChange.UpdateColumnNullability deserialize(JsonNode node) { - Iterator fieldJsons = node.get(FIELD_FILED).elements(); - List fieldNames = new ArrayList<>(); - while (fieldJsons.hasNext()) { - fieldNames.add(fieldJsons.next().asText()); - } - boolean newNullability = node.get(FIELD_NEW_NULLABILITY).asBoolean(); - return (SchemaChange.UpdateColumnNullability) SchemaChange.updateColumnNullability(fieldNames.toArray(new String[0]), newNullability); - } - - @Override - public void serialize(SchemaChange.UpdateColumnNullability updateColumnNullability, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeArrayFieldStart(FIELD_FILED); - for (String fieldName : updateColumnNullability.fieldNames()) { - generator.writeString(fieldName); - } - generator.writeEndArray(); - generator.writeBooleanField(FIELD_NEW_NULLABILITY, updateColumnNullability.newNullability()); - generator.writeEndObject(); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java deleted file mode 100644 index eda9be5a3ee4..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnNullabilitySerializer.java +++ /dev/null @@ -1,47 +0,0 @@ -package org.apache.paimon.rest.serializer; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import org.apache.paimon.schema.SchemaChange; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeJsonParser; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -/** Serializer for {@link SchemaChange.UpdateColumnType}. */ -public class UpdateColumnTypeSerializer implements JsonSerializer, JsonDeserializer { - - public static final UpdateColumnTypeSerializer INSTANCE = new UpdateColumnTypeSerializer(); - - private static final String FIELD_FILED = "field-names"; - private static final String FIELD_NEW_DATA_TYPE = "new-data-types"; - private static final String FIELD_KEEP_NULLABILITY = "keep-nullability"; - @Override - public SchemaChange.UpdateColumnType deserialize(JsonNode node) { - Iterator fieldJsons = node.get(FIELD_FILED).elements(); - List fieldNames = new ArrayList<>(); - while (fieldJsons.hasNext()) { - fieldNames.add(fieldJsons.next().asText()); - } - DataType dataType = DataTypeJsonParser.parseDataType(node.get(FIELD_NEW_DATA_TYPE)); - boolean keepNullability = node.has(FIELD_KEEP_NULLABILITY) ? node.get(FIELD_KEEP_NULLABILITY).asBoolean() : false; - return (SchemaChange.UpdateColumnType) SchemaChange.updateColumnType(fieldNames.toArray(new String[0]), dataType, keepNullability); - } - - @Override - public void serialize(SchemaChange.UpdateColumnType updateColumnType, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeArrayFieldStart(FIELD_FILED); - for (String fieldName : updateColumnType.fieldNames()) { - generator.writeString(fieldName); - } - generator.writeEndArray(); - generator.writeObjectField(FIELD_NEW_DATA_TYPE, updateColumnType.newDataType()); - generator.writeBooleanField(FIELD_KEEP_NULLABILITY, updateColumnType.keepNullability()); - generator.writeEndObject(); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java b/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java deleted file mode 100644 index ad673d242da4..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/rest/serializer/UpdateColumnTypeSerializer.java +++ /dev/null @@ -1,53 +0,0 @@ -package org.apache.paimon.rest.serializer; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import org.apache.paimon.schema.SchemaChange; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeJsonParser; -import org.apache.paimon.utils.JsonDeserializer; -import org.apache.paimon.utils.JsonSerializer; - -public class AddColumnSerializer implements JsonSerializer, JsonDeserializer { - - public static final AddColumnSerializer INSTANCE = new AddColumnSerializer(); - - private static final String FIELD_FILED_NAME = "field-names"; - private static final String FIELD_DATA_TYPE_NAME = "data-types"; - private static final String FIELD_DESCRIPTION_NAME = "desc"; - private static final String FIELD_MOVE_NAME = "move"; - @Override - public SchemaChange.AddColumn deserialize(JsonNode node) { - Iterator fieldJsons = node.get(FIELD_FILED_NAME).elements(); - List fieldNames = new ArrayList<>(); - while (fieldJsons.hasNext()) { - fieldNames.add(fieldJsons.next().asText()); - } - DataType dataType = DataTypeJsonParser.parseDataType(node.get(FIELD_DATA_TYPE_NAME)); - String description = node.has(FIELD_DESCRIPTION_NAME) ? node.get(FIELD_DESCRIPTION_NAME).asText() : null; - SchemaChange.Move move = node.has(FIELD_MOVE_NAME) ? MoveSerializer.INSTANCE.deserialize(node.get(FIELD_MOVE_NAME)) : null; - return (SchemaChange.AddColumn) SchemaChange.addColumn(fieldNames.toArray(new String[0]), dataType, description, move); - } - - @Override - public void serialize(SchemaChange.AddColumn addColumn, JsonGenerator generator) throws IOException { - generator.writeStartObject(); - generator.writeArrayFieldStart(FIELD_FILED_NAME); - for (String fieldName : addColumn.fieldNames()) { - generator.writeString(fieldName); - } - generator.writeEndArray(); - generator.writeObjectField(FIELD_DATA_TYPE_NAME, addColumn.dataType()); - if(addColumn.description() != null) { - generator.writeStringField(FIELD_DESCRIPTION_NAME, addColumn.description()); - } - if(addColumn.move() != null) { - generator.writeObjectField(FIELD_MOVE_NAME, addColumn.move()); - } - generator.writeEndObject(); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java index a3b30d81a3dd..59f5af6c870b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java @@ -26,6 +26,11 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + import javax.annotation.Nullable; import java.util.ArrayList; @@ -46,24 +51,37 @@ * @since 0.4.0 */ @Public +@JsonIgnoreProperties(ignoreUnknown = true) public class Schema { + private static final String FIELD_FIELDS = "fields"; + private static final String FIELD_PARTITION_KEYS = "partition-keys"; + private static final String FIELD_PRIMARY_KEYS = "primary-keys"; + private static final String FIELD_OPTIONS = "options"; + private static final String FIELD_COMMENT = "comment"; + + @JsonProperty(FIELD_FIELDS) private final List fields; + @JsonProperty(FIELD_PARTITION_KEYS) private final List partitionKeys; + @JsonProperty(FIELD_PRIMARY_KEYS) private final List primaryKeys; + @JsonProperty(FIELD_OPTIONS) private final Map options; + @JsonProperty(FIELD_COMMENT) private final String comment; + @JsonCreator public Schema( - List fields, - List partitionKeys, - List primaryKeys, - Map options, - String comment) { + @JsonProperty(FIELD_FIELDS) List fields, + @JsonProperty(FIELD_PARTITION_KEYS) List partitionKeys, + @JsonProperty(FIELD_PRIMARY_KEYS) List primaryKeys, + @JsonProperty(FIELD_OPTIONS) Map options, + @JsonProperty(FIELD_COMMENT) String comment) { this.options = new HashMap<>(options); this.partitionKeys = normalizePartitionKeys(partitionKeys); this.primaryKeys = normalizePrimaryKeys(primaryKeys); @@ -95,6 +113,31 @@ public String comment() { return comment; } + @JsonGetter(FIELD_FIELDS) + public List getFields() { + return fields; + } + + @JsonGetter(FIELD_PARTITION_KEYS) + public List getPartitionKeys() { + return partitionKeys; + } + + @JsonGetter(FIELD_PRIMARY_KEYS) + public List getPrimaryKeys() { + return primaryKeys; + } + + @JsonGetter(FIELD_OPTIONS) + public Map getOptions() { + return options; + } + + @JsonGetter(FIELD_COMMENT) + public String getComment() { + return comment; + } + public Schema copy(RowType rowType) { return new Schema(rowType.getFields(), partitionKeys, primaryKeys, options, comment); } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java index cefa3c6eb9e7..de6842908b73 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java @@ -21,6 +21,11 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.types.DataType; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + import javax.annotation.Nullable; import java.io.Serializable; @@ -221,16 +226,34 @@ public int hashCode() { } /** A SchemaChange to add a field. */ + @JsonIgnoreProperties(ignoreUnknown = true) final class AddColumn implements SchemaChange { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-names"; + private static final String FIELD_DATA_TYPES = "data-types"; + private static final String FIELD_COMMENT = "comment"; + private static final String FIELD_MOVE = "move"; + + @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; + + @JsonProperty(FIELD_DATA_TYPES) private final DataType dataType; + + @JsonProperty(FIELD_COMMENT) private final String description; + + @JsonProperty(FIELD_MOVE) private final Move move; - private AddColumn(String[] fieldNames, DataType dataType, String description, Move move) { + @JsonCreator + private AddColumn( + @JsonProperty(FIELD_FILED_NAMES) String[] fieldNames, + @JsonProperty(FIELD_DATA_TYPES) DataType dataType, + @JsonProperty(FIELD_COMMENT) String description, + @JsonProperty(FIELD_MOVE) Move move) { this.fieldNames = fieldNames; this.dataType = dataType; this.description = description; @@ -255,6 +278,28 @@ public Move move() { return move; } + @JsonGetter(FIELD_FILED_NAMES) + public String[] getFieldNames() { + return fieldNames; + } + + @JsonGetter(FIELD_DATA_TYPES) + public DataType getDataType() { + return dataType; + } + + @JsonGetter(FIELD_COMMENT) + @Nullable + public String getDescription() { + return description; + } + + @JsonGetter(FIELD_MOVE) + @Nullable + public Move getMove() { + return move; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -280,14 +325,24 @@ public int hashCode() { } /** A SchemaChange to rename a field. */ + @JsonIgnoreProperties(ignoreUnknown = true) final class RenameColumn implements SchemaChange { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-names"; + private static final String FIELD_NEW_NAME = "new-name"; + + @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; + + @JsonProperty(FIELD_NEW_NAME) private final String newName; - private RenameColumn(String[] fieldNames, String newName) { + @JsonCreator + private RenameColumn( + @JsonProperty(FIELD_FILED_NAMES) String[] fieldNames, + @JsonProperty(FIELD_NEW_NAME) String newName) { this.fieldNames = fieldNames; this.newName = newName; } @@ -300,6 +355,16 @@ public String newName() { return newName; } + @JsonGetter(FIELD_FILED_NAMES) + public String[] getFieldNames() { + return fieldNames; + } + + @JsonGetter(FIELD_NEW_NAME) + public String getNewName() { + return newName; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -322,13 +387,18 @@ public int hashCode() { } /** A SchemaChange to drop a field. */ + @JsonIgnoreProperties(ignoreUnknown = true) final class DropColumn implements SchemaChange { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-names"; + + @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; - private DropColumn(String[] fieldNames) { + @JsonCreator + private DropColumn(@JsonProperty(FIELD_FILED_NAMES) String[] fieldNames) { this.fieldNames = fieldNames; } @@ -336,6 +406,11 @@ public String[] fieldNames() { return fieldNames; } + @JsonGetter(FIELD_FILED_NAMES) + public String[] getFieldNames() { + return fieldNames; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -355,17 +430,28 @@ public int hashCode() { } /** A SchemaChange to update the field type. */ + @JsonIgnoreProperties(ignoreUnknown = true) final class UpdateColumnType implements SchemaChange { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-names"; + private static final String FIELD_NEW_DATA_TYPE = "new-data-types"; + private static final String FIELD_KEEP_NULLABILITY = "keep-nullability"; + @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; + + @JsonProperty(FIELD_NEW_DATA_TYPE) private final DataType newDataType; // If true, do not change the target field nullability + @JsonProperty(FIELD_KEEP_NULLABILITY) private final boolean keepNullability; + @JsonCreator private UpdateColumnType( - String[] fieldNames, DataType newDataType, boolean keepNullability) { + @JsonProperty(FIELD_FILED_NAMES) String[] fieldNames, + @JsonProperty(FIELD_NEW_DATA_TYPE) DataType newDataType, + @JsonProperty(FIELD_KEEP_NULLABILITY) boolean keepNullability) { this.fieldNames = fieldNames; this.newDataType = newDataType; this.keepNullability = keepNullability; @@ -383,6 +469,21 @@ public boolean keepNullability() { return keepNullability; } + @JsonGetter(FIELD_FILED_NAMES) + public String[] getFieldNames() { + return fieldNames; + } + + @JsonGetter(FIELD_NEW_DATA_TYPE) + public DataType getNewDataType() { + return newDataType; + } + + @JsonGetter(FIELD_KEEP_NULLABILITY) + public boolean isKeepNullability() { + return keepNullability; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -438,6 +539,7 @@ public int hashCode() { } /** Represents a requested column move in a struct. */ + @JsonIgnoreProperties(ignoreUnknown = true) class Move implements Serializable { public enum MoveType { @@ -465,11 +567,24 @@ public static Move last(String fieldName) { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-name"; + private static final String FIELD_REFERENCE_FIELD_NAME = "reference-field-name"; + private static final String FIELD_TYPE = "type"; + + @JsonProperty(FIELD_FILED_NAMES) private final String fieldName; + + @JsonProperty(FIELD_REFERENCE_FIELD_NAME) private final String referenceFieldName; + + @JsonProperty(FIELD_TYPE) private final MoveType type; - public Move(String fieldName, String referenceFieldName, MoveType type) { + @JsonCreator + public Move( + @JsonProperty(FIELD_FILED_NAMES) String fieldName, + @JsonProperty(FIELD_REFERENCE_FIELD_NAME) String referenceFieldName, + @JsonProperty(FIELD_TYPE) MoveType type) { this.fieldName = fieldName; this.referenceFieldName = referenceFieldName; this.type = type; @@ -487,6 +602,21 @@ public MoveType type() { return type; } + @JsonGetter(FIELD_FILED_NAMES) + public String getFieldName() { + return fieldName; + } + + @JsonGetter(FIELD_REFERENCE_FIELD_NAME) + public String getReferenceFieldName() { + return referenceFieldName; + } + + @JsonGetter(FIELD_TYPE) + public MoveType getType() { + return type; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -508,14 +638,24 @@ public int hashCode() { } /** A SchemaChange to update the (nested) field nullability. */ + @JsonIgnoreProperties(ignoreUnknown = true) final class UpdateColumnNullability implements SchemaChange { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-names"; + private static final String FIELD_NEW_NULLABILITY = "new-nullability"; + + @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; + + @JsonProperty(FIELD_NEW_NULLABILITY) private final boolean newNullability; - public UpdateColumnNullability(String[] fieldNames, boolean newNullability) { + @JsonCreator + public UpdateColumnNullability( + @JsonProperty(FIELD_FILED_NAMES) String[] fieldNames, + @JsonProperty(FIELD_NEW_NULLABILITY) boolean newNullability) { this.fieldNames = fieldNames; this.newNullability = newNullability; } @@ -528,6 +668,16 @@ public boolean newNullability() { return newNullability; } + @JsonGetter(FIELD_FILED_NAMES) + public String[] getFieldNames() { + return fieldNames; + } + + @JsonGetter(FIELD_NEW_NULLABILITY) + public boolean isNewNullability() { + return newNullability; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -550,14 +700,24 @@ public int hashCode() { } /** A SchemaChange to update the (nested) field comment. */ + @JsonIgnoreProperties(ignoreUnknown = true) final class UpdateColumnComment implements SchemaChange { private static final long serialVersionUID = 1L; + private static final String FIELD_FILED_NAMES = "field-names"; + private static final String FIELD_NEW_COMMENT = "new-comment"; + + @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; + + @JsonProperty(FIELD_NEW_COMMENT) private final String newDescription; - public UpdateColumnComment(String[] fieldNames, String newDescription) { + @JsonCreator + public UpdateColumnComment( + @JsonProperty(FIELD_FILED_NAMES) String[] fieldNames, + @JsonProperty(FIELD_NEW_COMMENT) String newDescription) { this.fieldNames = fieldNames; this.newDescription = newDescription; } @@ -570,6 +730,16 @@ public String newDescription() { return newDescription; } + @JsonGetter(FIELD_FILED_NAMES) + public String[] getFieldNames() { + return fieldNames; + } + + @JsonGetter(FIELD_NEW_COMMENT) + public String getNewDescription() { + return newDescription; + } + @Override public boolean equals(Object o) { if (this == o) { 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 ddf78d59c8ae..11c69187680c 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 @@ -22,6 +22,7 @@ import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.requests.CreateTableRequest; +import org.apache.paimon.rest.requests.SchemaChanges; import org.apache.paimon.rest.requests.UpdateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; @@ -30,6 +31,7 @@ import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.rest.responses.ListTablesResponse; import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.types.DataTypes; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; @@ -116,6 +118,52 @@ public static CreateTableRequest createTableRequest(String name) { public static UpdateTableRequest updateTableRequest(String fromTableName, String toTableName) { Identifier fromIdentifier = Identifier.create(databaseName(), fromTableName); Identifier toIdentifier = Identifier.create(databaseName(), toTableName); - return new UpdateTableRequest(fromIdentifier, toIdentifier, null); + // add option + SchemaChange addOption = SchemaChange.setOption("snapshot.time-retained", "2h"); + // remove option + SchemaChange removeOption = SchemaChange.removeOption("compaction.max.file-num"); + // add column + SchemaChange addColumn = SchemaChange.addColumn("col1_after", DataTypes.STRING()); + // add a column after col1 + SchemaChange.Move after = SchemaChange.Move.after("col1_after", "col1"); + SchemaChange addColumnAfterField = + SchemaChange.addColumn("col7", DataTypes.STRING(), "", after); + // rename column + SchemaChange renameColumn = SchemaChange.renameColumn("col3", "col3_new_name"); + // drop column + SchemaChange dropColumn = SchemaChange.dropColumn("col6"); + // update column comment + SchemaChange updateColumnComment = + SchemaChange.updateColumnComment(new String[] {"col4"}, "col4 field"); + // update nested column comment + SchemaChange updateNestedColumnComment = + SchemaChange.updateColumnComment(new String[] {"col5", "f1"}, "col5 f1 field"); + // update column type + SchemaChange updateColumnType = SchemaChange.updateColumnType("col4", DataTypes.DOUBLE()); + // update column position, you need to pass in a parameter of type Move + SchemaChange updateColumnPosition = + SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4")); + // update column nullability + SchemaChange updateColumnNullability = + SchemaChange.updateColumnNullability(new String[] {"col4"}, false); + // update nested column nullability + SchemaChange updateNestedColumnNullability = + SchemaChange.updateColumnNullability(new String[] {"col5", "f2"}, false); + + List schemaChanges = new ArrayList<>(); + schemaChanges.add(addOption); + schemaChanges.add(removeOption); + schemaChanges.add(addColumn); + schemaChanges.add(addColumnAfterField); + schemaChanges.add(renameColumn); + schemaChanges.add(dropColumn); + schemaChanges.add(updateColumnComment); + schemaChanges.add(updateNestedColumnComment); + schemaChanges.add(updateColumnType); + schemaChanges.add(updateColumnPosition); + schemaChanges.add(updateColumnNullability); + schemaChanges.add(updateNestedColumnNullability); + SchemaChanges changes = new SchemaChanges(schemaChanges); + return new UpdateTableRequest(fromIdentifier, toIdentifier, changes); } } From 1d5a8b2e6c74d2fb495d14a38db656907b0d824c Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 24 Dec 2024 09:24:23 +0800 Subject: [PATCH 18/23] generate table open api define update open api --- .../org/apache/paimon/types/DataType.java | 3 + .../org/apache/paimon/catalog/Identifier.java | 15 +- .../rest/requests/CreateTableRequest.java | 1 + .../org/apache/paimon/schema/TableSchema.java | 45 +++ .../apache/paimon/rest/MockRESTMessage.java | 93 +++-- paimon-open-api/rest-catalog-open-api.yaml | 351 +++++++++++++++++- .../open/api/RESTCatalogController.java | 6 +- .../paimon/open/api/config/OpenAPIConfig.java | 5 +- 8 files changed, 466 insertions(+), 53 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java index ac1e5b2a645e..8f84009a6a63 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java @@ -21,6 +21,7 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; import java.io.IOException; @@ -54,6 +55,7 @@ public DataType(boolean isNullable, DataTypeRoot typeRoot) { } /** Returns whether a value of this type can be {@code null}. */ + @JsonIgnore public boolean isNullable() { return isNullable; } @@ -61,6 +63,7 @@ public boolean isNullable() { /** * Returns the root of this type. It is an essential description without additional parameters. */ + @JsonIgnore public DataTypeRoot getTypeRoot() { return typeRoot; } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java index 301a92fbc639..f193929cef11 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java @@ -28,6 +28,7 @@ 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.JsonIgnore; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; @@ -114,12 +115,12 @@ public String getDatabaseName() { return database; } - @JsonIgnoreProperties + @JsonIgnore public String getObjectName() { return object; } - @JsonIgnoreProperties + @JsonIgnore public String getFullName() { return UNKNOWN_DATABASE.equals(this.database) ? object @@ -138,19 +139,19 @@ public String getTableName() { return branch; } - @JsonIgnoreProperties + @JsonIgnore public String getBranchNameOrDefault() { String branch = getBranchName(); return branch == null ? BranchManager.DEFAULT_MAIN_BRANCH : branch; } - @JsonIgnoreProperties + @JsonIgnore public @Nullable String getSystemTableName() { splitObjectName(); return systemTable; } - @JsonIgnoreProperties + @JsonIgnore public boolean isSystemTable() { return getSystemTableName() != null; } @@ -186,12 +187,12 @@ private void splitObjectName() { } } - @JsonIgnoreProperties + @JsonIgnore public String getEscapedFullName() { return getEscapedFullName('`'); } - @JsonIgnoreProperties + @JsonIgnore public String getEscapedFullName(char escapeChar) { return String.format( "%c%s%c.%c%s%c", escapeChar, database, escapeChar, escapeChar, object, escapeChar); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java index f566db3dcc6a..794dd33c465b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateTableRequest.java @@ -43,6 +43,7 @@ public CreateTableRequest( @JsonProperty(FIELD_IDENTIFIER) Identifier identifier, @JsonProperty(FIELD_SCHEMA) Schema schema) { this.schema = schema; + this.identifier = identifier; } @JsonGetter(FIELD_IDENTIFIER) diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java index 791269dc73b5..d8c84fa15111 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java @@ -358,4 +358,49 @@ public static TableSchema tryFromPath(FileIO fileIO, Path path) throws FileNotFo throw new UncheckedIOException(e); } } + + public int getVersion() { + return version; + } + + public long getId() { + return id; + } + + public List getFields() { + return fields; + } + + public int getHighestFieldId() { + return highestFieldId; + } + + public List getPartitionKeys() { + return partitionKeys; + } + + public List getPrimaryKeys() { + return primaryKeys; + } + + public List getBucketKeys() { + return bucketKeys; + } + + public int getNumBucket() { + return numBucket; + } + + public Map getOptions() { + return options; + } + + @Nullable + public String getComment() { + return comment; + } + + public long getTimeMillis() { + return timeMillis; + } } 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 11c69187680c..0a9c5bc2d3b6 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 @@ -32,7 +32,9 @@ import org.apache.paimon.rest.responses.ListTablesResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; @@ -123,46 +125,67 @@ public static UpdateTableRequest updateTableRequest(String fromTableName, String // remove option SchemaChange removeOption = SchemaChange.removeOption("compaction.max.file-num"); // add column - SchemaChange addColumn = SchemaChange.addColumn("col1_after", DataTypes.STRING()); - // add a column after col1 - SchemaChange.Move after = SchemaChange.Move.after("col1_after", "col1"); - SchemaChange addColumnAfterField = - SchemaChange.addColumn("col7", DataTypes.STRING(), "", after); - // rename column - SchemaChange renameColumn = SchemaChange.renameColumn("col3", "col3_new_name"); - // drop column - SchemaChange dropColumn = SchemaChange.dropColumn("col6"); - // update column comment - SchemaChange updateColumnComment = - SchemaChange.updateColumnComment(new String[] {"col4"}, "col4 field"); - // update nested column comment - SchemaChange updateNestedColumnComment = - SchemaChange.updateColumnComment(new String[] {"col5", "f1"}, "col5 f1 field"); - // update column type - SchemaChange updateColumnType = SchemaChange.updateColumnType("col4", DataTypes.DOUBLE()); - // update column position, you need to pass in a parameter of type Move - SchemaChange updateColumnPosition = - SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4")); - // update column nullability - SchemaChange updateColumnNullability = - SchemaChange.updateColumnNullability(new String[] {"col4"}, false); - // update nested column nullability - SchemaChange updateNestedColumnNullability = - SchemaChange.updateColumnNullability(new String[] {"col5", "f2"}, false); + SchemaChange addColumn = + SchemaChange.addColumn("col1_after", DataTypes.ARRAY(DataTypes.STRING())); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.BINARY(1), + DataTypes.VARBINARY(1), + DataTypes.MAP(DataTypes.VARCHAR(8), DataTypes.VARCHAR(8)), + DataTypes.MULTISET(DataTypes.VARCHAR(8)) + }, + new String[] {"pt", "a", "b", "c", "d", "e", "f"}); + SchemaChange addColumnMap = + SchemaChange.addColumn( + "col11_map_type", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())); + SchemaChange addColumnRowType = SchemaChange.addColumn("col11_row_type", rowType); + // // add a column after col1 + // SchemaChange.Move after = SchemaChange.Move.after("col1_after", "col1"); + // SchemaChange addColumnAfterField = + // SchemaChange.addColumn("col7", DataTypes.STRING(), "", after); + // // rename column + // SchemaChange renameColumn = SchemaChange.renameColumn("col3", "col3_new_name"); + // // drop column + // SchemaChange dropColumn = SchemaChange.dropColumn("col6"); + // // update column comment + // SchemaChange updateColumnComment = + // SchemaChange.updateColumnComment(new String[] {"col4"}, "col4 field"); + // // update nested column comment + // SchemaChange updateNestedColumnComment = + // SchemaChange.updateColumnComment(new String[] {"col5", "f1"}, "col5 f1 + // field"); + // // update column type + // SchemaChange updateColumnType = SchemaChange.updateColumnType("col4", + // DataTypes.DOUBLE()); + // // update column position, you need to pass in a parameter of type Move + // SchemaChange updateColumnPosition = + // SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4")); + // // update column nullability + // SchemaChange updateColumnNullability = + // SchemaChange.updateColumnNullability(new String[] {"col4"}, false); + // // update nested column nullability + // SchemaChange updateNestedColumnNullability = + // SchemaChange.updateColumnNullability(new String[] {"col5", "f2"}, false); List schemaChanges = new ArrayList<>(); schemaChanges.add(addOption); schemaChanges.add(removeOption); schemaChanges.add(addColumn); - schemaChanges.add(addColumnAfterField); - schemaChanges.add(renameColumn); - schemaChanges.add(dropColumn); - schemaChanges.add(updateColumnComment); - schemaChanges.add(updateNestedColumnComment); - schemaChanges.add(updateColumnType); - schemaChanges.add(updateColumnPosition); - schemaChanges.add(updateColumnNullability); - schemaChanges.add(updateNestedColumnNullability); + schemaChanges.add(addColumnMap); + schemaChanges.add(addColumnRowType); + // schemaChanges.add(addColumnAfterField); + // schemaChanges.add(renameColumn); + // schemaChanges.add(dropColumn); + // schemaChanges.add(updateColumnComment); + // schemaChanges.add(updateNestedColumnComment); + // schemaChanges.add(updateColumnType); + // schemaChanges.add(updateColumnPosition); + // schemaChanges.add(updateColumnNullability); + // schemaChanges.add(updateNestedColumnNullability); SchemaChanges changes = new SchemaChanges(schemaChanges); return new UpdateTableRequest(fromIdentifier, toIdentifier, changes); } diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index f7f9529f53dd..d7669d085291 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -80,6 +80,159 @@ paths: $ref: '#/components/schemas/ErrorResponse' "500": description: Internal Server Error + /v1/{prefix}/databases/{database}/tables: + get: + tags: + - table + summary: List tables + operationId: listTables + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/ListTablesResponse' + "500": + description: Internal Server Error + post: + tags: + - table + summary: Create table + operationId: createTable + 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/CreateTableRequest' + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/GetTableResponse' + "500": + description: Internal Server Error + /v1/{prefix}/databases/{database}/tables/{table}: + get: + tags: + - table + summary: Get table + operationId: getTable + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + - name: table + in: path + required: true + schema: + type: string + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/ListTablesResponse' + "404": + description: Resource not found + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error + post: + tags: + - table + summary: Update table + operationId: updateTable + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + - name: table + in: path + required: true + schema: + type: string + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/UpdateTableRequest' + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/GetTableResponse' + "500": + description: Internal Server Error + delete: + tags: + - table + summary: Drop table + operationId: dropTable + 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: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error /v1/{prefix}/databases/{database}/properties: post: tags: @@ -200,6 +353,15 @@ components: type: object additionalProperties: type: string + CreateDatabaseResponse: + type: object + properties: + name: + type: string + options: + type: object + additionalProperties: + type: string ErrorResponse: type: object properties: @@ -212,15 +374,183 @@ components: type: array items: type: string - CreateDatabaseResponse: + CreateTableRequest: + type: object + properties: + identifier: + $ref: '#/components/schemas/Identifier' + schema: + $ref: '#/components/schemas/Schema' + DataField: type: object properties: + id: + type: integer + format: int32 name: type: string + type: + $ref: '#/components/schemas/DataType' + description: + type: string + DataType: + anyOf: + - type: string + - type: object + Identifier: + type: object + properties: + database: + type: string + table: + type: string + branch: + type: string + Schema: + type: object + properties: + fields: + type: array + items: + $ref: '#/components/schemas/DataField' + partition-keys: + type: array + items: + type: string + primary-keys: + type: array + items: + type: string options: type: object additionalProperties: type: string + comment: + type: string + GetTableResponse: + type: object + properties: + location: + type: string + schema: + $ref: '#/components/schemas/TableSchema' + TableSchema: + type: object + AddColumn: + type: object + properties: + field-names: + type: array + items: + type: string + data-types: + $ref: '#/components/schemas/DataType' + comment: + type: string + move: + $ref: '#/components/schemas/Move' + Move: + type: object + properties: + field-name: + type: string + reference-field-name: + type: string + type: + type: string + enum: + - FIRST + - AFTER + - BEFORE + - LAST + RenameColumn: + type: object + properties: + field-names: + type: array + items: + type: string + new-name: + type: string + SchemaChanges: + type: object + properties: + set-options: + type: object + additionalProperties: + type: string + remove-options: + type: array + items: + type: string + comment: + type: string + add-columns: + type: array + items: + $ref: '#/components/schemas/AddColumn' + rename-columns: + type: array + items: + $ref: '#/components/schemas/RenameColumn' + drop-columns: + type: array + items: + type: string + update-column-types: + type: array + items: + $ref: '#/components/schemas/UpdateColumnType' + update-column-nullabilities: + type: array + items: + $ref: '#/components/schemas/UpdateColumnNullability' + update-column-comments: + type: array + items: + $ref: '#/components/schemas/UpdateColumnComment' + update-column-positions: + type: array + items: + $ref: '#/components/schemas/Move' + UpdateColumnComment: + type: object + properties: + field-names: + type: array + items: + type: string + new-comment: + type: string + UpdateColumnNullability: + type: object + properties: + field-names: + type: array + items: + type: string + new-nullability: + type: boolean + UpdateColumnType: + type: object + properties: + field-names: + type: array + items: + type: string + new-data-types: + $ref: '#/components/schemas/DataType' + keep-nullability: + type: boolean + UpdateTableRequest: + type: object + properties: + from: + $ref: '#/components/schemas/Identifier' + to: + $ref: '#/components/schemas/Identifier' + changes: + $ref: '#/components/schemas/SchemaChanges' AlterDatabaseRequest: type: object properties: @@ -247,18 +577,13 @@ components: type: array items: type: string - DatabaseName: - type: object - properties: - name: - type: string ListDatabasesResponse: type: object properties: databases: type: array items: - $ref: '#/components/schemas/DatabaseName' + type: string GetDatabaseResponse: type: object properties: @@ -268,6 +593,13 @@ components: type: object additionalProperties: type: string + ListTablesResponse: + type: object + properties: + tables: + type: array + items: + type: string ConfigResponse: type: object properties: @@ -279,3 +611,8 @@ components: type: object additionalProperties: type: string + + securitySchemes: + BearerAuth: + type: http + scheme: bearer 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 6e9b8c5f02d9..0341b2556e79 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 @@ -209,7 +209,7 @@ public ListTablesResponse listTables( @ApiResponses({ @ApiResponse( responseCode = "200", - content = {@Content(schema = @Schema(implementation = ListTablesResponse.class))}), + content = {@Content(schema = @Schema(implementation = GetTableResponse.class))}), @ApiResponse( responseCode = "404", description = "Resource not found", @@ -278,7 +278,7 @@ public GetTableResponse createTable( responseCode = "500", content = {@Content(schema = @Schema())}) }) - @PostMapping("/v1/{prefix}/databases/{database}/tables/table") + @PostMapping("/v1/{prefix}/databases/{database}/tables/{table}") public GetTableResponse updateTable( @PathVariable String prefix, @PathVariable String database, @@ -299,7 +299,7 @@ public GetTableResponse updateTable( } @Operation( - summary = "Update table", + summary = "Drop table", tags = {"table"}) @ApiResponses({ @ApiResponse( diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java index 71ac066d4a70..0074cde62770 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -22,6 +22,7 @@ import io.swagger.v3.oas.models.info.Info; import io.swagger.v3.oas.models.info.License; import io.swagger.v3.oas.models.responses.ApiResponses; +import io.swagger.v3.oas.models.security.SecurityRequirement; import io.swagger.v3.oas.models.servers.Server; import org.springdoc.core.customizers.OpenApiCustomiser; import org.springframework.beans.factory.annotation.Value; @@ -56,9 +57,11 @@ public OpenAPI restCatalogOpenAPI() { .version("1.0") .description("This API exposes endpoints to RESTCatalog.") .license(mitLicense); + SecurityRequirement securityRequirement = new SecurityRequirement(); + securityRequirement.addList("BearerAuth"); List servers = new ArrayList<>(); servers.add(server); - return new OpenAPI().info(info).servers(servers); + return new OpenAPI().info(info).servers(servers).addSecurityItem(securityRequirement); } /** Sort response alphabetically. So the api generate will in same order everytime. */ From f5c0286ecd24304dce9186a5d3a676283f8d6e1a Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 24 Dec 2024 14:11:31 +0800 Subject: [PATCH 19/23] add ut update open api add table scheme define delete no need code and fix ut fail update update table api add comment for getDataOrFormatTable --- .../org/apache/paimon/rest/RESTCatalog.java | 134 +++++++++--------- .../paimon/rest/RESTCatalogFactory.java | 6 - .../apache/paimon/rest/RESTObjectMapper.java | 7 + .../paimon/rest/requests/SchemaChanges.java | 34 +++++ .../rest/requests/UpdateTableRequest.java | 29 ++-- .../apache/paimon/schema/SchemaChange.java | 12 +- .../org/apache/paimon/schema/TableSchema.java | 45 ------ .../apache/paimon/rest/MockRESTMessage.java | 117 +++++++++------ .../apache/paimon/rest/RESTCatalogTest.java | 134 ++++++++++++++++++ .../paimon/rest/RESTObjectMapperTest.java | 25 +++- paimon-open-api/rest-catalog-open-api.yaml | 35 ++++- 11 files changed, 383 insertions(+), 195 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 f436254f1dcd..4b36c3fc6e37 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 @@ -66,8 +66,9 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import java.io.IOException; -import java.io.UncheckedIOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; @@ -90,6 +91,7 @@ /** A catalog implementation for REST. */ public class RESTCatalog implements Catalog { + private static final Logger LOG = LoggerFactory.getLogger(RESTCatalog.class); private static final ObjectMapper OBJECT_MAPPER = RESTObjectMapper.create(); private final RESTClient client; @@ -97,7 +99,7 @@ public class RESTCatalog implements Catalog { private final Map baseHeader; private final AuthSession catalogAuth; private final CatalogContext context; - private final FileIO fileIO; + private final Optional fileIOOptional; private volatile ScheduledExecutorService refreshExecutor = null; @@ -142,23 +144,21 @@ public RESTCatalog(CatalogContext catalogContext) { options, catalogContext.preferIO(), catalogContext.fallbackIO()); this.resourcePaths = ResourcePaths.forCatalogProperties(options.get(RESTCatalogInternalOptions.PREFIX)); - this.fileIO = getFileIOFromOptions(context); + this.fileIOOptional = getFileIOFromOptions(context); } - // todo: whether it's ok - private static FileIO getFileIOFromOptions(CatalogContext context) { - Options options = context.options(); - String warehouseStr = options.get(CatalogOptions.WAREHOUSE); - Path warehousePath = new Path(warehouseStr); - FileIO fileIO; - CatalogContext contextWithNewOptions = - CatalogContext.create(options, context.preferIO(), context.fallbackIO()); + private static Optional getFileIOFromOptions(CatalogContext context) { try { - fileIO = FileIO.get(warehousePath, contextWithNewOptions); - } catch (IOException e) { - throw new UncheckedIOException(e); + Options options = context.options(); + String warehouseStr = options.get(CatalogOptions.WAREHOUSE); + Path warehousePath = new Path(warehouseStr); + CatalogContext contextWithNewOptions = + CatalogContext.create(options, context.preferIO(), context.fallbackIO()); + return Optional.of(FileIO.get(warehousePath, contextWithNewOptions)); + } catch (Exception ignore) { + LOG.warn("Can not get FileIO from options."); } - return fileIO; + return Optional.empty(); } @Override @@ -173,7 +173,10 @@ public Map options() { @Override public FileIO fileIO() { - return this.fileIO; + if (this.fileIOOptional.isPresent()) { + return this.fileIOOptional.get(); + } + throw new RuntimeException("FileIO is not configured."); } @Override @@ -299,7 +302,9 @@ public void createTable(Identifier identifier, Schema schema, boolean ignoreIfEx GetTableResponse.class, headers()); } catch (AlreadyExistsException e) { - throw new TableAlreadyExistException(identifier); + if (!ignoreIfExists) { + throw new TableAlreadyExistException(identifier); + } } } @@ -324,7 +329,7 @@ public void alterTable( Identifier identifier, List changes, boolean ignoreIfNotExists) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { try { - updateTable(identifier, null, changes); + updateTable(identifier, identifier, changes); } catch (NoSuchResourceException e) { if (!ignoreIfNotExists) { throw new TableNotExistException(identifier); @@ -389,12 +394,46 @@ Map fetchOptionsFromServer( return response.merge(clientProperties); } - private static Map configHeaders(Map properties) { - return RESTUtil.extractPrefixMap(properties, "header."); + @VisibleForTesting + void updateTable( + Identifier fromTable, Identifier newTableIdentifier, List changes) { + UpdateTableRequest request = + new UpdateTableRequest(newTableIdentifier, new SchemaChanges(changes)); + client.post( + resourcePaths.table(fromTable.getDatabaseName(), fromTable.getTableName()), + request, + GetTableResponse.class, + headers()); } - private Map headers() { - return catalogAuth.getHeaders(); + @VisibleForTesting + Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { + Preconditions.checkArgument(identifier.getSystemTableName() == null); + TableSchema tableSchema = getDataTableSchema(identifier); + Lock.Factory lockFactory = + Lock.factory( + lockFactory(context.options(), fileIO(), Optional.empty()).orElse(null), + lockContext(context.options()).orElse(null), + identifier); + // MetastoreClient is not used in RESTCatalog so null is ok. + FileStoreTable table = + FileStoreTableFactory.create( + fileIO(), + newTableLocation(warehouse(), identifier), + tableSchema, + new CatalogEnvironment(identifier, null, lockFactory, null)); + CoreOptions options = table.coreOptions(); + if (options.type() == TableType.OBJECT_TABLE) { + String objectLocation = options.objectLocation(); + checkNotNull(objectLocation, "Object location should not be null for object table."); + table = + ObjectTable.builder() + .underlyingTable(table) + .objectLocation(objectLocation) + .objectFileIO(this.fileIO()) + .build(); + } + return table; } protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { @@ -413,15 +452,12 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } } - // todo: how know which exception to throw - private void updateTable(Identifier fromTable, Identifier toTable, List changes) { - UpdateTableRequest request = - new UpdateTableRequest(fromTable, toTable, new SchemaChanges(changes)); - client.post( - resourcePaths.table(fromTable.getDatabaseName(), fromTable.getTableName()), - request, - GetTableResponse.class, - headers()); + private static Map configHeaders(Map properties) { + return RESTUtil.extractPrefixMap(properties, "header."); + } + + private Map headers() { + return catalogAuth.getHeaders(); } private Table getAllInSystemDatabase(Identifier identifier) throws TableNotExistException { @@ -447,7 +483,7 @@ private Table getAllInSystemDatabase(Identifier identifier) throws TableNotExist }; Table table = SystemTableLoader.loadGlobal( - tableName, fileIO, getAllTablePathsFunction, context.options()); + tableName, fileIO(), getAllTablePathsFunction, context.options()); if (table == null) { throw new TableNotExistException(identifier); } @@ -465,38 +501,6 @@ private Table getSystemTable(Identifier identifier) throws TableNotExistExceptio return CatalogUtils.getSystemTable(identifier, originTable); } - private Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { - Preconditions.checkArgument(identifier.getSystemTableName() == null); - TableSchema tableSchema = getDataTableSchema(identifier); - String uuid = null; - FileStoreTable table = - FileStoreTableFactory.create( - fileIO, - newTableLocation(warehouse(), identifier), - tableSchema, - new CatalogEnvironment( - identifier, - uuid, - Lock.factory( - lockFactory(context.options(), fileIO, Optional.empty()) - .orElse(null), - lockContext(context.options()).orElse(null), - identifier), - null)); // todo: whether need MetastoreClient.Factory - CoreOptions options = table.coreOptions(); - if (options.type() == TableType.OBJECT_TABLE) { - String objectLocation = options.objectLocation(); - checkNotNull(objectLocation, "Object location should not be null for object table."); - table = - ObjectTable.builder() - .underlyingTable(table) - .objectLocation(objectLocation) - .objectFileIO(this.fileIO) - .build(); - } - return table; - } - private ScheduledExecutorService tokenRefreshExecutor() { if (refreshExecutor == null) { synchronized (this) { diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index b03c9ca4248d..aa6e6f4d4113 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -21,8 +21,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; -import org.apache.paimon.options.CatalogOptions; -import org.apache.paimon.options.Options; /** Factory to create {@link RESTCatalog}. */ public class RESTCatalogFactory implements CatalogFactory { @@ -35,10 +33,6 @@ public String identifier() { @Override public Catalog create(CatalogContext context) { - Options options = context.options(); - if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { - throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); - } return new RESTCatalog(context); } } 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 fa66e754ac85..d9220bceb4dd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -18,6 +18,8 @@ package org.apache.paimon.rest; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.schema.TableSchemaSerializer; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeJsonParser; @@ -44,6 +46,11 @@ public static ObjectMapper create() { public static Module createPaimonRestJacksonModule() { SimpleModule module = new SimpleModule("Paimon_REST"); + registerJsonObjects( + module, + TableSchema.class, + TableSchemaSerializer.INSTANCE, + TableSchemaSerializer.INSTANCE); registerJsonObjects( module, DataField.class, diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java index 5abe48049a36..1c3e419f13d2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/SchemaChanges.java @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; /** Schema changes to serialize List of SchemaChange . */ @JsonIgnoreProperties(ignoreUnknown = true) @@ -201,4 +202,37 @@ public List getUpdateColumnComments() { public List getUpdateColumnPositions() { return updateColumnPositions; } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + SchemaChanges that = (SchemaChanges) o; + return Objects.equals(setOptions, that.setOptions) + && Objects.equals(removeOptions, that.removeOptions) + && Objects.equals(comment, that.comment) + && Objects.equals(addColumns, that.addColumns) + && Objects.equals(renameColumns, that.renameColumns) + && Objects.equals(dropColumns, that.dropColumns) + && Objects.equals(updateColumnTypes, that.updateColumnTypes) + && Objects.equals(updateColumnNullabilities, that.updateColumnNullabilities) + && Objects.equals(updateColumnComments, that.updateColumnComments) + && Objects.equals(updateColumnPositions, that.updateColumnPositions); + } + + @Override + public int hashCode() { + return Objects.hash( + setOptions, + removeOptions, + comment, + addColumns, + renameColumns, + dropColumns, + updateColumnTypes, + updateColumnNullabilities, + updateColumnComments, + updateColumnPositions); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java index ab1c7b18a59e..b522dc8ea10e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/UpdateTableRequest.java @@ -28,37 +28,26 @@ /** Request for updating table. */ public class UpdateTableRequest implements RESTRequest { - private static final String FIELD_FROM_IDENTIFIER_NAME = "from"; - private static final String FIELD_TO_IDENTIFIER_NAME = "to"; - private static final String FIELD_SCHEMA_CHANGES_NAME = "changes"; + private static final String FIELD_IDENTIFIER_NAME = "identifier-change"; + private static final String FIELD_SCHEMA_CHANGES_NAME = "schema-changes"; - @JsonProperty(FIELD_FROM_IDENTIFIER_NAME) - private Identifier fromIdentifier; - - @JsonProperty(FIELD_TO_IDENTIFIER_NAME) - private Identifier toIdentifier; + @JsonProperty(FIELD_IDENTIFIER_NAME) + private Identifier identifierChange; @JsonProperty(FIELD_SCHEMA_CHANGES_NAME) private SchemaChanges changes; @JsonCreator public UpdateTableRequest( - @JsonProperty(FIELD_FROM_IDENTIFIER_NAME) Identifier fromIdentifier, - @JsonProperty(FIELD_TO_IDENTIFIER_NAME) Identifier toIdentifier, + @JsonProperty(FIELD_IDENTIFIER_NAME) Identifier identifierChange, @JsonProperty(FIELD_SCHEMA_CHANGES_NAME) SchemaChanges changes) { - this.fromIdentifier = fromIdentifier; - this.toIdentifier = toIdentifier; + this.identifierChange = identifierChange; this.changes = changes; } - @JsonGetter(FIELD_FROM_IDENTIFIER_NAME) - public Identifier getFromIdentifier() { - return fromIdentifier; - } - - @JsonGetter(FIELD_TO_IDENTIFIER_NAME) - public Identifier getToIdentifier() { - return toIdentifier; + @JsonGetter(FIELD_IDENTIFIER_NAME) + public Identifier getIdentifierChange() { + return identifierChange; } @JsonGetter(FIELD_SCHEMA_CHANGES_NAME) diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java index de6842908b73..ec8f09e84734 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java @@ -232,14 +232,14 @@ final class AddColumn implements SchemaChange { private static final long serialVersionUID = 1L; private static final String FIELD_FILED_NAMES = "field-names"; - private static final String FIELD_DATA_TYPES = "data-types"; + private static final String FIELD_DATA_TYPE = "data-type"; private static final String FIELD_COMMENT = "comment"; private static final String FIELD_MOVE = "move"; @JsonProperty(FIELD_FILED_NAMES) private final String[] fieldNames; - @JsonProperty(FIELD_DATA_TYPES) + @JsonProperty(FIELD_DATA_TYPE) private final DataType dataType; @JsonProperty(FIELD_COMMENT) @@ -251,7 +251,7 @@ final class AddColumn implements SchemaChange { @JsonCreator private AddColumn( @JsonProperty(FIELD_FILED_NAMES) String[] fieldNames, - @JsonProperty(FIELD_DATA_TYPES) DataType dataType, + @JsonProperty(FIELD_DATA_TYPE) DataType dataType, @JsonProperty(FIELD_COMMENT) String description, @JsonProperty(FIELD_MOVE) Move move) { this.fieldNames = fieldNames; @@ -283,7 +283,7 @@ public String[] getFieldNames() { return fieldNames; } - @JsonGetter(FIELD_DATA_TYPES) + @JsonGetter(FIELD_DATA_TYPE) public DataType getDataType() { return dataType; } @@ -312,7 +312,7 @@ public boolean equals(Object o) { return Arrays.equals(fieldNames, addColumn.fieldNames) && dataType.equals(addColumn.dataType) && Objects.equals(description, addColumn.description) - && move.equals(addColumn.move); + && Objects.equals(move, addColumn.move); } @Override @@ -435,7 +435,7 @@ final class UpdateColumnType implements SchemaChange { private static final long serialVersionUID = 1L; private static final String FIELD_FILED_NAMES = "field-names"; - private static final String FIELD_NEW_DATA_TYPE = "new-data-types"; + private static final String FIELD_NEW_DATA_TYPE = "new-data-type"; private static final String FIELD_KEEP_NULLABILITY = "keep-nullability"; @JsonProperty(FIELD_FILED_NAMES) diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java index d8c84fa15111..791269dc73b5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java @@ -358,49 +358,4 @@ public static TableSchema tryFromPath(FileIO fileIO, Path path) throws FileNotFo throw new UncheckedIOException(e); } } - - public int getVersion() { - return version; - } - - public long getId() { - return id; - } - - public List getFields() { - return fields; - } - - public int getHighestFieldId() { - return highestFieldId; - } - - public List getPartitionKeys() { - return partitionKeys; - } - - public List getPrimaryKeys() { - return primaryKeys; - } - - public List getBucketKeys() { - return bucketKeys; - } - - public int getNumBucket() { - return numBucket; - } - - public Map getOptions() { - return options; - } - - @Nullable - public String getComment() { - return comment; - } - - public long getTimeMillis() { - return timeMillis; - } } 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 0a9c5bc2d3b6..3e9f32ba08e1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; @@ -28,17 +29,23 @@ import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.GetTableResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.rest.responses.ListTablesResponse; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -117,9 +124,13 @@ public static CreateTableRequest createTableRequest(String name) { return new CreateTableRequest(identifier, schema); } - public static UpdateTableRequest updateTableRequest(String fromTableName, String toTableName) { - Identifier fromIdentifier = Identifier.create(databaseName(), fromTableName); - Identifier toIdentifier = Identifier.create(databaseName(), toTableName); + public static UpdateTableRequest updateTableRequest(String toTableName) { + Identifier identifierChange = Identifier.create(databaseName(), toTableName); + SchemaChanges changes = new SchemaChanges(getChanges()); + return new UpdateTableRequest(identifierChange, changes); + } + + public static List getChanges() { // add option SchemaChange addOption = SchemaChange.setOption("snapshot.time-retained", "2h"); // remove option @@ -127,6 +138,9 @@ public static UpdateTableRequest updateTableRequest(String fromTableName, String // add column SchemaChange addColumn = SchemaChange.addColumn("col1_after", DataTypes.ARRAY(DataTypes.STRING())); + SchemaChange addColumnMap = + SchemaChange.addColumn( + "col1_map_type", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())); RowType rowType = RowType.of( new DataType[] { @@ -139,37 +153,32 @@ public static UpdateTableRequest updateTableRequest(String fromTableName, String DataTypes.MULTISET(DataTypes.VARCHAR(8)) }, new String[] {"pt", "a", "b", "c", "d", "e", "f"}); - SchemaChange addColumnMap = - SchemaChange.addColumn( - "col11_map_type", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())); - SchemaChange addColumnRowType = SchemaChange.addColumn("col11_row_type", rowType); - // // add a column after col1 - // SchemaChange.Move after = SchemaChange.Move.after("col1_after", "col1"); - // SchemaChange addColumnAfterField = - // SchemaChange.addColumn("col7", DataTypes.STRING(), "", after); - // // rename column - // SchemaChange renameColumn = SchemaChange.renameColumn("col3", "col3_new_name"); - // // drop column - // SchemaChange dropColumn = SchemaChange.dropColumn("col6"); - // // update column comment - // SchemaChange updateColumnComment = - // SchemaChange.updateColumnComment(new String[] {"col4"}, "col4 field"); - // // update nested column comment - // SchemaChange updateNestedColumnComment = - // SchemaChange.updateColumnComment(new String[] {"col5", "f1"}, "col5 f1 - // field"); - // // update column type - // SchemaChange updateColumnType = SchemaChange.updateColumnType("col4", - // DataTypes.DOUBLE()); - // // update column position, you need to pass in a parameter of type Move - // SchemaChange updateColumnPosition = - // SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4")); - // // update column nullability - // SchemaChange updateColumnNullability = - // SchemaChange.updateColumnNullability(new String[] {"col4"}, false); - // // update nested column nullability - // SchemaChange updateNestedColumnNullability = - // SchemaChange.updateColumnNullability(new String[] {"col5", "f2"}, false); + SchemaChange addColumnRowType = SchemaChange.addColumn("col_row_type", rowType); + // add a column after col1 + SchemaChange.Move after = SchemaChange.Move.after("col1_after", "col1"); + SchemaChange addColumnAfterField = + SchemaChange.addColumn("col7", DataTypes.STRING(), "", after); + // rename column + SchemaChange renameColumn = SchemaChange.renameColumn("col3", "col3_new_name"); + // drop column + SchemaChange dropColumn = SchemaChange.dropColumn("col6"); + // update column comment + SchemaChange updateColumnComment = + SchemaChange.updateColumnComment(new String[] {"col4"}, "col4 field"); + // update nested column comment + SchemaChange updateNestedColumnComment = + SchemaChange.updateColumnComment(new String[] {"col5", "f1"}, "col5 f1 field"); + // update column type + SchemaChange updateColumnType = SchemaChange.updateColumnType("col4", DataTypes.DOUBLE()); + // update column position, you need to pass in a parameter of type Move + SchemaChange updateColumnPosition = + SchemaChange.updateColumnPosition(SchemaChange.Move.first("col4")); + // update column nullability + SchemaChange updateColumnNullability = + SchemaChange.updateColumnNullability(new String[] {"col4"}, false); + // update nested column nullability + SchemaChange updateNestedColumnNullability = + SchemaChange.updateColumnNullability(new String[] {"col5", "f2"}, false); List schemaChanges = new ArrayList<>(); schemaChanges.add(addOption); @@ -177,16 +186,34 @@ public static UpdateTableRequest updateTableRequest(String fromTableName, String schemaChanges.add(addColumn); schemaChanges.add(addColumnMap); schemaChanges.add(addColumnRowType); - // schemaChanges.add(addColumnAfterField); - // schemaChanges.add(renameColumn); - // schemaChanges.add(dropColumn); - // schemaChanges.add(updateColumnComment); - // schemaChanges.add(updateNestedColumnComment); - // schemaChanges.add(updateColumnType); - // schemaChanges.add(updateColumnPosition); - // schemaChanges.add(updateColumnNullability); - // schemaChanges.add(updateNestedColumnNullability); - SchemaChanges changes = new SchemaChanges(schemaChanges); - return new UpdateTableRequest(fromIdentifier, toIdentifier, changes); + schemaChanges.add(addColumnAfterField); + schemaChanges.add(renameColumn); + schemaChanges.add(dropColumn); + schemaChanges.add(updateColumnComment); + schemaChanges.add(updateNestedColumnComment); + schemaChanges.add(updateColumnType); + schemaChanges.add(updateColumnPosition); + schemaChanges.add(updateColumnNullability); + schemaChanges.add(updateNestedColumnNullability); + return schemaChanges; + } + + public static GetTableResponse getTableResponse() { + return new GetTableResponse("location", tableSchema()); + } + + private static TableSchema tableSchema() { + List fields = + Arrays.asList( + new DataField(0, "f0", new IntType()), + new DataField(1, "f1", new IntType())); + List partitionKeys = Collections.singletonList("f0"); + List primaryKeys = Arrays.asList("f0", "f1"); + Map options = new HashMap<>(); + options.put("option-1", "value-1"); + options.put("option-2", "value-2"); + // set path for test as if not set system will add one + options.put(CoreOptions.PATH.key(), "/a/b/c"); + return new TableSchema(1, fields, 1, partitionKeys, primaryKeys, options, "comment"); } } 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 7fe3255f43a9..a3c4ab34012d 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 @@ -21,14 +21,19 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.Identifier; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; +import org.apache.paimon.rest.requests.CreateTableRequest; import org.apache.paimon.rest.responses.AlterDatabaseResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.GetTableResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.rest.responses.ListTablesResponse; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.table.Table; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -50,6 +55,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -225,6 +232,133 @@ public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsTrue() th assertDoesNotThrow(() -> mockRestCatalog.alterDatabase(name, new ArrayList<>(), true)); } + @Test + public void testListTables() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + ListTablesResponse response = MockRESTMessage.listTablesResponse(); + mockResponse(mapper.writeValueAsString(response), 200); + List result = restCatalog.listTables(databaseName); + assertEquals(response.getTables().size(), result.size()); + } + + @Test + public void testGetTable() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + GetTableResponse response = MockRESTMessage.getTableResponse(); + mockResponse(mapper.writeValueAsString(response), 200); + Table result = mockRestCatalog.getTable(Identifier.create(databaseName, "table")); + assertEquals(response.getSchema().options().size(), result.options().size()); + verify(mockRestCatalog, times(1)).getDataOrFormatTable(any()); + } + + @Test + public void testCreateTable() throws Exception { + CreateTableRequest request = MockRESTMessage.createTableRequest("table"); + GetTableResponse response = MockRESTMessage.getTableResponse(); + mockResponse(mapper.writeValueAsString(response), 200); + assertDoesNotThrow( + () -> restCatalog.createTable(request.getIdentifier(), request.getSchema(), false)); + } + + @Test + public void testCreateTableWhenTableAlreadyExistAndIgnoreIfExistsIsFalse() throws Exception { + CreateTableRequest request = MockRESTMessage.createTableRequest("table"); + mockResponse("", 409); + assertThrows( + Catalog.TableAlreadyExistException.class, + () -> restCatalog.createTable(request.getIdentifier(), request.getSchema(), false)); + } + + @Test + public void testRenameTable() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + String fromTableName = "fromTable"; + String toTableName = "toTable"; + GetTableResponse response = MockRESTMessage.getTableResponse(); + mockResponse(mapper.writeValueAsString(response), 200); + assertDoesNotThrow( + () -> + mockRestCatalog.renameTable( + Identifier.create(databaseName, fromTableName), + Identifier.create(databaseName, toTableName), + true)); + verify(mockRestCatalog, times(1)).updateTable(any(), any(), anyList()); + } + + @Test + public void testRenameTableWhenTableNotExistAndIgnoreIfNotExistsIsFalse() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + String fromTableName = "fromTable"; + String toTableName = "toTable"; + mockResponse("", 404); + assertThrows( + Catalog.TableNotExistException.class, + () -> + mockRestCatalog.renameTable( + Identifier.create(databaseName, fromTableName), + Identifier.create(databaseName, toTableName), + false)); + } + + @Test + public void testRenameTableWhenToTableAlreadyExist() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + String fromTableName = "fromTable"; + String toTableName = "toTable"; + mockResponse("", 409); + assertThrows( + Catalog.TableAlreadyExistException.class, + () -> + mockRestCatalog.renameTable( + Identifier.create(databaseName, fromTableName), + Identifier.create(databaseName, toTableName), + false)); + } + + @Test + public void testAlterTable() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + List changes = MockRESTMessage.getChanges(); + GetTableResponse response = MockRESTMessage.getTableResponse(); + mockResponse(mapper.writeValueAsString(response), 200); + assertDoesNotThrow( + () -> + mockRestCatalog.alterTable( + Identifier.create(databaseName, "t1"), changes, true)); + verify(mockRestCatalog, times(1)).updateTable(any(), any(), anyList()); + } + + @Test + public void testAlterTableWhenTableNotExistAndIgnoreIfNotExistsIsFalse() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + List changes = MockRESTMessage.getChanges(); + mockResponse("", 404); + assertThrows( + Catalog.TableNotExistException.class, + () -> + mockRestCatalog.alterTable( + Identifier.create(databaseName, "t1"), changes, false)); + } + + @Test + public void testDropTable() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + String tableName = "table"; + mockResponse("", 200); + assertDoesNotThrow( + () -> restCatalog.dropTable(Identifier.create(databaseName, tableName), true)); + } + + @Test + public void testDropTableWhenTableNotExistAndIgnoreIfNotExistsIsFalse() throws Exception { + String databaseName = MockRESTMessage.databaseName(); + String tableName = "table"; + mockResponse("", 404); + assertThrows( + Catalog.TableNotExistException.class, + () -> restCatalog.dropTable(Identifier.create(databaseName, tableName), false)); + } + private void mockResponse(String mockResponse, int httpCode) { MockResponse mockResponseObj = new 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 987bb1cd4c5f..9cc362881f1c 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 @@ -27,7 +27,9 @@ import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.ErrorResponse; import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.GetTableResponse; import org.apache.paimon.rest.responses.ListDatabasesResponse; +import org.apache.paimon.rest.responses.ListTablesResponse; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.IntType; @@ -160,10 +162,27 @@ public void dataFieldParseTest() throws Exception { @Test public void updateTableRequestParseTest() throws Exception { - UpdateTableRequest request = MockRESTMessage.updateTableRequest("t1", "t2"); + UpdateTableRequest request = MockRESTMessage.updateTableRequest("t2"); String requestStr = mapper.writeValueAsString(request); UpdateTableRequest parseData = mapper.readValue(requestStr, UpdateTableRequest.class); - assertEquals(request.getFromIdentifier(), parseData.getFromIdentifier()); - assertEquals(request.getToIdentifier(), parseData.getToIdentifier()); + assertEquals(request.getIdentifierChange(), parseData.getIdentifierChange()); + assertEquals(request.getChanges(), parseData.getChanges()); + } + + @Test + public void getTableResponseParseTest() throws Exception { + GetTableResponse response = MockRESTMessage.getTableResponse(); + String responseStr = mapper.writeValueAsString(response); + GetTableResponse parseData = mapper.readValue(responseStr, GetTableResponse.class); + assertEquals(response.getLocation(), parseData.getLocation()); + assertEquals(response.getSchema(), parseData.getSchema()); + } + + @Test + public void listTablesResponseParseTest() throws Exception { + ListTablesResponse response = MockRESTMessage.listTablesResponse(); + String responseStr = mapper.writeValueAsString(response); + ListTablesResponse parseData = mapper.readValue(responseStr, ListTablesResponse.class); + assertEquals(response.getTables(), parseData.getTables()); } } diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index d7669d085291..f3f3f33349b7 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -164,7 +164,7 @@ paths: content: application/json: schema: - $ref: '#/components/schemas/ListTablesResponse' + $ref: '#/components/schemas/GetTableResponse' "404": description: Resource not found content: @@ -436,6 +436,33 @@ components: $ref: '#/components/schemas/TableSchema' TableSchema: type: object + properties: + version: + type: integer + format: int32 + id: + type: integer + format: int64 + highestFieldId: + type: integer + format: int32 + partitionKeys: + type: array + items: + type: string + primaryKeys: + type: array + items: + type: string + options: + type: object + additionalProperties: + type: string + comment: + type: string + timeMillis: + type: integer + format: int64 AddColumn: type: object properties: @@ -545,11 +572,9 @@ components: UpdateTableRequest: type: object properties: - from: - $ref: '#/components/schemas/Identifier' - to: + identifier-change: $ref: '#/components/schemas/Identifier' - changes: + schema-changes: $ref: '#/components/schemas/SchemaChanges' AlterDatabaseRequest: type: object From bbdfa9618abc6ae307fa21c01f6cffd6d1c2ddcc Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 24 Dec 2024 16:36:51 +0800 Subject: [PATCH 20/23] delete no need change --- .../src/main/java/org/apache/paimon/types/DataType.java | 3 --- .../main/java/org/apache/paimon/rest/RESTObjectMapper.java | 7 ++----- .../{TableSchemaSerializer.java => SchemaSerializer.java} | 4 ++-- .../main/java/org/apache/paimon/utils/JsonSerdeUtil.java | 7 ++----- 4 files changed, 6 insertions(+), 15 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/schema/{TableSchemaSerializer.java => SchemaSerializer.java} (97%) diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java index 8f84009a6a63..ac1e5b2a645e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java @@ -21,7 +21,6 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.utils.Preconditions; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; import java.io.IOException; @@ -55,7 +54,6 @@ public DataType(boolean isNullable, DataTypeRoot typeRoot) { } /** Returns whether a value of this type can be {@code null}. */ - @JsonIgnore public boolean isNullable() { return isNullable; } @@ -63,7 +61,6 @@ public boolean isNullable() { /** * Returns the root of this type. It is an essential description without additional parameters. */ - @JsonIgnore public DataTypeRoot getTypeRoot() { return typeRoot; } 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 d9220bceb4dd..ce20158d0b3f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -18,8 +18,8 @@ package org.apache.paimon.rest; +import org.apache.paimon.schema.SchemaSerializer; import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.schema.TableSchemaSerializer; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeJsonParser; @@ -47,10 +47,7 @@ public static ObjectMapper create() { public static Module createPaimonRestJacksonModule() { SimpleModule module = new SimpleModule("Paimon_REST"); registerJsonObjects( - module, - TableSchema.class, - TableSchemaSerializer.INSTANCE, - TableSchemaSerializer.INSTANCE); + module, TableSchema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); registerJsonObjects( module, DataField.class, diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchemaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaSerializer.java similarity index 97% rename from paimon-core/src/main/java/org/apache/paimon/schema/TableSchemaSerializer.java rename to paimon-core/src/main/java/org/apache/paimon/schema/SchemaSerializer.java index f6b6c7302f2c..4fb28359c92f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchemaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaSerializer.java @@ -39,10 +39,10 @@ import static org.apache.paimon.schema.TableSchema.PAIMON_08_VERSION; /** A {@link JsonSerializer} for {@link TableSchema}. */ -public class TableSchemaSerializer +public class SchemaSerializer implements JsonSerializer, JsonDeserializer { - public static final TableSchemaSerializer INSTANCE = new TableSchemaSerializer(); + public static final SchemaSerializer INSTANCE = new SchemaSerializer(); @Override public void serialize(TableSchema tableSchema, JsonGenerator generator) throws IOException { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java index 7c36f4261347..edc6dac5f992 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java @@ -18,8 +18,8 @@ package org.apache.paimon.utils; +import org.apache.paimon.schema.SchemaSerializer; import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.schema.TableSchemaSerializer; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeJsonParser; @@ -181,10 +181,7 @@ public T deserialize(JsonParser parser, DeserializationContext context) private static Module createPaimonJacksonModule() { SimpleModule module = new SimpleModule("Paimon"); registerJsonObjects( - module, - TableSchema.class, - TableSchemaSerializer.INSTANCE, - TableSchemaSerializer.INSTANCE); + module, TableSchema.class, SchemaSerializer.INSTANCE, SchemaSerializer.INSTANCE); registerJsonObjects( module, DataField.class, From eacf66ecfaa68bc4ac06e1d3b8ce96b9136580aa Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 24 Dec 2024 17:57:21 +0800 Subject: [PATCH 21/23] delete no need get and split NoPermissionException to DatabaseNoPermissionException and TableNoPermissionException --- .../org/apache/paimon/types/DataField.java | 19 ---- .../org/apache/paimon/catalog/Catalog.java | 58 +++++----- .../org/apache/paimon/rest/RESTCatalog.java | 16 +-- .../apache/paimon/schema/SchemaChange.java | 104 +++--------------- 4 files changed, 58 insertions(+), 139 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java index a7fccd63c1e5..209118023ba5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java @@ -20,7 +20,6 @@ import org.apache.paimon.annotation.Public; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonGenerator; import javax.annotation.Nullable; @@ -38,7 +37,6 @@ * @since 0.4.0 */ @Public -@JsonIgnoreProperties(ignoreUnknown = true) public final class DataField implements Serializable { private static final long serialVersionUID = 1L; @@ -78,23 +76,6 @@ public DataType type() { return type; } - public int getId() { - return id; - } - - public String getName() { - return name; - } - - public DataType getType() { - return type; - } - - @Nullable - public String getDescription() { - return description; - } - public DataField newId(int newid) { return new DataField(newid, name, type, description); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index e1d044e95b3d..904c96910746 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -364,31 +364,6 @@ default void repairTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); } - /** - * Exception for trying to operate on a resource that doesn't have permission. Define as a - * runtime exception: 1. Other engine has no this type exception. 2. It wouldn't bring api break - * change. - */ - class NoPermissionException extends RuntimeException { - private static final String MSG = "No permission for %s %s."; - private static final String DATABASE_TYPE_NAME = "database"; - private static final String TABLE_TYPE_NAME = "table"; - - public static NoPermissionException createDatabaseNoPermissionException( - String databaseName, Throwable cause) { - return new NoPermissionException(DATABASE_TYPE_NAME, databaseName, cause); - } - - public static NoPermissionException createTableNoPermissionException( - Identifier identifier, Throwable cause) { - return new NoPermissionException(TABLE_TYPE_NAME, identifier.getFullName(), cause); - } - - public NoPermissionException(String resourceType, String resourceName, Throwable cause) { - super(String.format(MSG, resourceType, resourceName), cause); - } - } - /** Exception for trying to drop on a database that is not empty. */ class DatabaseNotEmptyException extends Exception { private static final String MSG = "Database %s is not empty."; @@ -458,6 +433,22 @@ public ProcessSystemDatabaseException() { } } + /** Exception for trying to operate on the database that doesn't have permission. */ + class DatabaseNoPermissionException extends RuntimeException { + private static final String MSG = "Database %s has no permission."; + + private final String database; + + public DatabaseNoPermissionException(String database, Throwable cause) { + super(String.format(MSG, database), cause); + this.database = database; + } + + public String database() { + return database; + } + } + /** Exception for trying to create a table that already exists. */ class TableAlreadyExistException extends Exception { @@ -500,6 +491,23 @@ public Identifier identifier() { } } + /** Exception for trying to operate on the table that doesn't have permission. */ + class TableNoPermissionException extends RuntimeException { + + private static final String MSG = "Table %s has no permission."; + + private final Identifier identifier; + + public TableNoPermissionException(Identifier identifier, Throwable cause) { + super(String.format(MSG, identifier.getFullName()), cause); + this.identifier = identifier; + } + + public Identifier identifier() { + return identifier; + } + } + /** Exception for trying to operate on a partition that doesn't exist. */ class PartitionNotExistException extends 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 4b36c3fc6e37..48367106011a 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 @@ -202,7 +202,7 @@ public void createDatabase(String name, boolean ignoreIfExists, Map changes, boolean ign throw new DatabaseNotExistException(name); } } catch (ForbiddenException e) { - throw NoPermissionException.createDatabaseNoPermissionException(name, e); + throw new DatabaseNoPermissionException(name, e); } } @@ -318,7 +318,7 @@ public void renameTable(Identifier fromTable, Identifier toTable, boolean ignore throw new TableNotExistException(fromTable); } } catch (ForbiddenException e) { - throw NoPermissionException.createTableNoPermissionException(fromTable, e); + throw new TableNoPermissionException(fromTable, e); } catch (AlreadyExistsException e) { throw new TableAlreadyExistException(toTable); } @@ -335,7 +335,7 @@ public void alterTable( throw new TableNotExistException(identifier); } } catch (ForbiddenException e) { - throw NoPermissionException.createTableNoPermissionException(identifier, e); + throw new TableNoPermissionException(identifier, e); } } @@ -351,7 +351,7 @@ public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throw new TableNotExistException(identifier); } } catch (ForbiddenException e) { - throw NoPermissionException.createTableNoPermissionException(identifier, e); + throw new TableNoPermissionException(identifier, e); } } @@ -448,7 +448,7 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } catch (NoSuchResourceException e) { throw new TableNotExistException(identifier); } catch (ForbiddenException e) { - throw NoPermissionException.createTableNoPermissionException(identifier, e); + throw new TableNoPermissionException(identifier, e); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java index ec8f09e84734..a600b089c52a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java @@ -260,43 +260,25 @@ private AddColumn( this.move = move; } + @JsonGetter(FIELD_FILED_NAMES) public String[] fieldNames() { return fieldNames; } + @JsonGetter(FIELD_DATA_TYPE) public DataType dataType() { return dataType; } @Nullable + @JsonGetter(FIELD_COMMENT) public String description() { return description; } @Nullable - public Move move() { - return move; - } - - @JsonGetter(FIELD_FILED_NAMES) - public String[] getFieldNames() { - return fieldNames; - } - - @JsonGetter(FIELD_DATA_TYPE) - public DataType getDataType() { - return dataType; - } - - @JsonGetter(FIELD_COMMENT) - @Nullable - public String getDescription() { - return description; - } - @JsonGetter(FIELD_MOVE) - @Nullable - public Move getMove() { + public Move move() { return move; } @@ -347,21 +329,13 @@ private RenameColumn( this.newName = newName; } - public String[] fieldNames() { - return fieldNames; - } - - public String newName() { - return newName; - } - @JsonGetter(FIELD_FILED_NAMES) - public String[] getFieldNames() { + public String[] fieldNames() { return fieldNames; } @JsonGetter(FIELD_NEW_NAME) - public String getNewName() { + public String newName() { return newName; } @@ -402,12 +376,8 @@ private DropColumn(@JsonProperty(FIELD_FILED_NAMES) String[] fieldNames) { this.fieldNames = fieldNames; } - public String[] fieldNames() { - return fieldNames; - } - @JsonGetter(FIELD_FILED_NAMES) - public String[] getFieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -457,30 +427,18 @@ private UpdateColumnType( this.keepNullability = keepNullability; } - public String[] fieldNames() { - return fieldNames; - } - - public DataType newDataType() { - return newDataType; - } - - public boolean keepNullability() { - return keepNullability; - } - @JsonGetter(FIELD_FILED_NAMES) - public String[] getFieldNames() { + public String[] fieldNames() { return fieldNames; } @JsonGetter(FIELD_NEW_DATA_TYPE) - public DataType getNewDataType() { + public DataType newDataType() { return newDataType; } @JsonGetter(FIELD_KEEP_NULLABILITY) - public boolean isKeepNullability() { + public boolean keepNullability() { return keepNullability; } @@ -590,30 +548,18 @@ public Move( this.type = type; } - public String fieldName() { - return fieldName; - } - - public String referenceFieldName() { - return referenceFieldName; - } - - public MoveType type() { - return type; - } - @JsonGetter(FIELD_FILED_NAMES) - public String getFieldName() { + public String fieldName() { return fieldName; } @JsonGetter(FIELD_REFERENCE_FIELD_NAME) - public String getReferenceFieldName() { + public String referenceFieldName() { return referenceFieldName; } @JsonGetter(FIELD_TYPE) - public MoveType getType() { + public MoveType type() { return type; } @@ -660,21 +606,13 @@ public UpdateColumnNullability( this.newNullability = newNullability; } - public String[] fieldNames() { - return fieldNames; - } - - public boolean newNullability() { - return newNullability; - } - @JsonGetter(FIELD_FILED_NAMES) - public String[] getFieldNames() { + public String[] fieldNames() { return fieldNames; } @JsonGetter(FIELD_NEW_NULLABILITY) - public boolean isNewNullability() { + public boolean newNullability() { return newNullability; } @@ -722,21 +660,13 @@ public UpdateColumnComment( this.newDescription = newDescription; } - public String[] fieldNames() { - return fieldNames; - } - - public String newDescription() { - return newDescription; - } - @JsonGetter(FIELD_FILED_NAMES) - public String[] getFieldNames() { + public String[] fieldNames() { return fieldNames; } @JsonGetter(FIELD_NEW_COMMENT) - public String getNewDescription() { + public String newDescription() { return newDescription; } From e8626e2718fe14e40544a605686b9888c289c8d3 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 24 Dec 2024 18:03:17 +0800 Subject: [PATCH 22/23] delete no need get method --- .../java/org/apache/paimon/schema/Schema.java | 30 ++++--------------- 1 file changed, 5 insertions(+), 25 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java index 59f5af6c870b..ee098415887a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java @@ -93,48 +93,28 @@ public RowType rowType() { return new RowType(false, fields); } - public List fields() { - return fields; - } - - public List partitionKeys() { - return partitionKeys; - } - - public List primaryKeys() { - return primaryKeys; - } - - public Map options() { - return options; - } - - public String comment() { - return comment; - } - @JsonGetter(FIELD_FIELDS) - public List getFields() { + public List fields() { return fields; } @JsonGetter(FIELD_PARTITION_KEYS) - public List getPartitionKeys() { + public List partitionKeys() { return partitionKeys; } @JsonGetter(FIELD_PRIMARY_KEYS) - public List getPrimaryKeys() { + public List primaryKeys() { return primaryKeys; } @JsonGetter(FIELD_OPTIONS) - public Map getOptions() { + public Map options() { return options; } @JsonGetter(FIELD_COMMENT) - public String getComment() { + public String comment() { return comment; } From 1f94a154ef77cb9e8ee94aa35fa416d0c4083ef4 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Tue, 24 Dec 2024 18:27:16 +0800 Subject: [PATCH 23/23] fix --- .../org/apache/paimon/catalog/Identifier.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java index f193929cef11..ac6996821b03 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java @@ -52,8 +52,7 @@ public class Identifier implements Serializable { private static final long serialVersionUID = 1L; private static final String FIELD_DATABASE_NAME = "database"; - private static final String FIELD_TABLE_NAME = "table"; - private static final String FIELD_BRANCH_NAME = "branch"; + private static final String FIELD_OBJECT_NAME = "object"; public static final RowType SCHEMA = new RowType( @@ -67,26 +66,24 @@ public class Identifier implements Serializable { @JsonProperty(FIELD_DATABASE_NAME) private final String database; + @JsonProperty(FIELD_OBJECT_NAME) private final String object; - @JsonProperty(FIELD_TABLE_NAME) private transient String table; - @JsonProperty(FIELD_BRANCH_NAME) private transient String branch; private transient String systemTable; - public Identifier(String database, String object) { + @JsonCreator + public Identifier( + @JsonProperty(FIELD_DATABASE_NAME) String database, + @JsonProperty(FIELD_OBJECT_NAME) String object) { this.database = database; this.object = object; } - @JsonCreator - public Identifier( - @JsonProperty(FIELD_DATABASE_NAME) String database, - @JsonProperty(FIELD_TABLE_NAME) String table, - @JsonProperty(FIELD_BRANCH_NAME) @Nullable String branch) { + public Identifier(String database, String table, @Nullable String branch) { this(database, table, branch, null); } @@ -115,7 +112,7 @@ public String getDatabaseName() { return database; } - @JsonIgnore + @JsonGetter(FIELD_OBJECT_NAME) public String getObjectName() { return object; } @@ -127,13 +124,13 @@ public String getFullName() { : String.format("%s.%s", database, object); } - @JsonGetter(FIELD_TABLE_NAME) + @JsonIgnore public String getTableName() { splitObjectName(); return table; } - @JsonGetter(FIELD_BRANCH_NAME) + @JsonIgnore public @Nullable String getBranchName() { splitObjectName(); return branch;