From 030f8c9966753be7ac1aa0b719dac4d283332a63 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 12 Dec 2024 17:15:49 +0800 Subject: [PATCH 01/19] support alter database --- .../paimon/catalog/AbstractCatalog.java | 16 ++++ .../apache/paimon/catalog/CachingCatalog.java | 7 ++ .../org/apache/paimon/catalog/Catalog.java | 13 ++++ .../apache/paimon/catalog/DatabaseChange.java | 73 +++++++++++++++++++ .../paimon/catalog/DelegateCatalog.java | 6 ++ .../paimon/catalog/FileSystemCatalog.java | 5 ++ .../org/apache/paimon/jdbc/JdbcCatalog.java | 26 +++++++ .../org/apache/paimon/jdbc/JdbcUtils.java | 33 +++++++++ .../privilege/AllGrantedPrivilegeChecker.java | 3 + .../paimon/privilege/PrivilegeChecker.java | 2 + .../privilege/PrivilegeCheckerImpl.java | 8 ++ .../paimon/privilege/PrivilegeType.java | 1 + .../paimon/privilege/PrivilegedCatalog.java | 8 ++ .../org/apache/paimon/rest/RESTCatalog.java | 7 ++ .../org/apache/paimon/hive/HiveCatalog.java | 21 ++++++ 15 files changed, 229 insertions(+) create mode 100644 paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.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 b56fec279ab1..d6e07c03d8b3 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 @@ -229,6 +229,22 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade protected abstract void dropDatabaseImpl(String name); + @Override + public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + checkNotSystemDatabase(name); + try { + getDatabase(name); + } catch (DatabaseNotExistException e) { + if (ignoreIfNotExists) { + return; + } + throw new DatabaseNotExistException(name); + } + } + + protected abstract void alertDatabaseImpl(String name, List changes); + @Override public List listTables(String databaseName) throws DatabaseNotExistException { if (isSystemDatabase(databaseName)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 82d503b7a272..4f4b90c945b7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -187,6 +187,13 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } } + @Override + public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + super.alertDatabase(name, changes, ignoreIfNotExists); + databaseCache.invalidate(name); + } + @Override public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throws TableNotExistException { 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 d919c5978297..698aaf3e69e1 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 @@ -125,6 +125,19 @@ void createDatabase(String name, boolean ignoreIfExists, Map pro void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException; + /** + * Alert a database. + * + * @param name Name of the database to alert. + * @param changes a collection of changes to apply to the database. + * @param ignoreIfNotExists Flag to specify behavior when the database does not exist: if set to + * false, throw an exception, if set to true, do nothing. + * @throws DatabaseNotExistException if the given database is not exist and ignoreIfNotExists is + * false + */ + void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException; + /** * Return a {@link Table} identified by the given {@link Identifier}. * diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java new file mode 100644 index 000000000000..5cb6bbfcff3a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java @@ -0,0 +1,73 @@ +/* + * 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.catalog; + +import java.util.Map; + +public interface DatabaseChange { + static DatabaseChange setProperty(String property, String value) { + return new SetProperty(property, value); + } + + static DatabaseChange removeProperty(String property) { + return new RemoveProperty(property); + } + + String apply(Map parameter); + + public static final class SetProperty implements DatabaseChange { + private final String property; + private final String value; + + private SetProperty(String property, String value) { + this.property = property; + this.value = value; + } + + public String property() { + return this.property; + } + + public String value() { + return this.value; + } + + @Override + public String apply(Map parameter) { + return parameter.put(property, value); + } + } + + public static final class RemoveProperty implements DatabaseChange { + private final String property; + + private RemoveProperty(String property) { + this.property = property; + } + + public String property() { + return this.property; + } + + @Override + public String apply(Map parameter) { + return parameter.remove(property); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index ec14d53a2b03..a44eaef1c955 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -84,6 +84,12 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade wrapped.dropDatabase(name, ignoreIfNotExists, cascade); } + @Override + public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + wrapped.alertDatabase(name, changes, ignoreIfNotExists); + } + @Override public List listTables(String databaseName) throws DatabaseNotExistException { return wrapped.listTables(databaseName); 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 9264a54647b1..f68df4d99ced 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 @@ -92,6 +92,11 @@ protected void dropDatabaseImpl(String name) { uncheck(() -> fileIO.delete(newDatabasePath(name), true)); } + @Override + protected void alertDatabaseImpl(String name, List changes) { + throw new UnsupportedOperationException("Alert database is not supported."); + } + @Override protected List listTablesImpl(String databaseName) { return uncheck(() -> listTablesInFileSystem(newDatabasePath(databaseName))); diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 778bc591fe89..60a0a1babcd8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -23,6 +23,7 @@ import org.apache.paimon.catalog.CatalogLockContext; import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.DatabaseChange; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -38,6 +39,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,10 +54,12 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.paimon.jdbc.JdbcCatalogLock.acquireTimeout; import static org.apache.paimon.jdbc.JdbcCatalogLock.checkMaxSleep; +import static org.apache.paimon.jdbc.JdbcUtils.deleteProperties; import static org.apache.paimon.jdbc.JdbcUtils.execute; import static org.apache.paimon.jdbc.JdbcUtils.insertProperties; import static org.apache.paimon.jdbc.JdbcUtils.updateTable; @@ -197,6 +201,28 @@ protected void dropDatabaseImpl(String name) { execute(connections, JdbcUtils.DELETE_ALL_DATABASE_PROPERTIES_SQL, catalogKey, name); } + @Override + protected void alertDatabaseImpl(String name, List changes) { + Map insertProperties = Maps.newHashMap(); + Set removeProperties = Sets.newHashSet(); + changes.forEach( + change -> { + if (change instanceof DatabaseChange.SetProperty) { + DatabaseChange.SetProperty setProperty = + (DatabaseChange.SetProperty) change; + insertProperties.put(setProperty.property(), setProperty.value()); + } else { + removeProperties.add(((DatabaseChange.RemoveProperty) change).property()); + } + }); + if (!insertProperties.isEmpty()) { + insertProperties(connections, catalogKey, name, insertProperties); + } + if (!removeProperties.isEmpty()) { + deleteProperties(connections, catalogKey, name, removeProperties); + } + } + @Override protected List listTablesImpl(String databaseName) { return fetch( diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java index 4acb0f25aa91..d729a1bfbaef 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java @@ -30,8 +30,10 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.SQLIntegrityConstraintViolationException; +import java.util.Collections; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.function.Consumer; import java.util.stream.Stream; @@ -202,6 +204,16 @@ public class JdbcUtils { + " = ? AND " + DATABASE_NAME + " = ? "; + static final String DELETE_DATABASE_PROPERTIES_SQL = + "DELETE FROM " + + DATABASE_PROPERTIES_TABLE_NAME + + " WHERE " + + CATALOG_KEY + + " = ? AND " + + DATABASE_NAME + + " = ? AND " + + DATABASE_PROPERTY_KEY + + " IN "; static final String DELETE_ALL_DATABASE_PROPERTIES_SQL = "DELETE FROM " + DATABASE_PROPERTIES_TABLE_NAME @@ -403,6 +415,27 @@ private static String insertPropertiesStatement(int size) { return sqlStatement.toString(); } + public static boolean deleteProperties( + JdbcClientPool connections, + String storeKey, + String databaseName, + Set properties) { + String[] args = + Stream.concat(Stream.of(storeKey, databaseName), properties.stream()) + .toArray(String[]::new); + + return execute(connections, JdbcUtils.deletePropertiesStatement(properties), args) > 0; + } + + private static String deletePropertiesStatement(Set properties) { + StringBuilder sqlStatement = new StringBuilder(JdbcUtils.DELETE_DATABASE_PROPERTIES_SQL); + String values = + String.join(",", Collections.nCopies(properties.size(), String.valueOf('?'))); + sqlStatement.append("(").append(values).append(")"); + + return sqlStatement.toString(); + } + public static void createDistributedLockTable(JdbcClientPool connections, Options options) throws SQLException, InterruptedException { DistributedLockDialectFactory.create(connections.getProtocol()) diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java b/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java index 09944681a2e7..c33a13f69275 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java @@ -41,6 +41,9 @@ public void assertCanCreateTable(String databaseName) {} @Override public void assertCanDropDatabase(String databaseName) {} + @Override + public void assertCanAlertDatabase(String databaseName) {} + @Override public void assertCanCreateDatabase() {} diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java index 1771d40f4028..ad904f74964a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java @@ -53,6 +53,8 @@ default void assertCanSelectOrInsert(Identifier identifier) { void assertCanDropDatabase(String databaseName); + void assertCanAlertDatabase(String databaseName); + void assertCanCreateDatabase(); void assertCanCreateUser(); diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java index 19c1813ee852..ec13446b2afb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java @@ -85,6 +85,14 @@ public void assertCanDropDatabase(String databaseName) { } } + @Override + public void assertCanAlertDatabase(String databaseName) { + if (!check(databaseName, PrivilegeType.ALERT_DATABASE)) { + throw new NoPrivilegeException( + user, "database", databaseName, PrivilegeType.ALERT_DATABASE); + } + } + @Override public void assertCanCreateDatabase() { if (!check( diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java index 375f5030d557..0b0aa93281cc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java @@ -34,6 +34,7 @@ public enum PrivilegeType { CREATE_TABLE(PrivilegeTarget.DATABASE), DROP_DATABASE(PrivilegeTarget.DATABASE), + ALERT_DATABASE(PrivilegeTarget.DATABASE), CREATE_DATABASE(PrivilegeTarget.CATALOG), // you can create and drop users, grant and revoke any privileges to or from others diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java index 2e88213a24b9..bc4eb8b66c5b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java @@ -19,6 +19,7 @@ package org.apache.paimon.privilege; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.DatabaseChange; import org.apache.paimon.catalog.DelegateCatalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.options.ConfigOption; @@ -82,6 +83,13 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade privilegeManager.objectDropped(name); } + @Override + public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + privilegeManager.getPrivilegeChecker().assertCanAlertDatabase(name); + super.alertDatabase(name, changes, ignoreIfNotExists); + } + @Override public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throws TableNotExistException { 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 03b257efbf86..405b1629366c 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,6 +20,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.DatabaseChange; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -172,6 +173,12 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } } + @Override + public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + @Override public Table getTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); 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 c74ede981546..88d1e493d919 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 @@ -25,6 +25,7 @@ import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogLockContext; import org.apache.paimon.catalog.CatalogLockFactory; +import org.apache.paimon.catalog.DatabaseChange; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.client.ClientPool; import org.apache.paimon.data.BinaryRow; @@ -53,6 +54,8 @@ import org.apache.paimon.view.View; import org.apache.paimon.view.ViewImpl; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + import org.apache.flink.table.hive.LegacyHiveClasses; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -405,6 +408,24 @@ protected void dropDatabaseImpl(String name) { } } + @Override + protected void alertDatabaseImpl(String name, List changes) { + try { + Database database = clients.run(client -> client.getDatabase(name)); + Map parameter = Maps.newHashMap(); + parameter.putAll(database.getParameters()); + changes.forEach(change -> change.apply(parameter)); + Map newProperties = Collections.unmodifiableMap(parameter); + Database alertDatabase = convertToHiveDatabase(name, newProperties); + clients.execute(client -> client.alterDatabase(name, alertDatabase)); + } catch (TException e) { + throw new RuntimeException("Failed to alert database " + name, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to alertDatabase " + name, e); + } + } + @Override protected List listTablesImpl(String databaseName) { try { From 1e66c0ae4617b8670b7e49d9ba7eb41a5e2d9e3d Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 09:22:39 +0800 Subject: [PATCH 02/19] add AlterDatabase request and response, and remove ignoreIfExists in CreateDatabaseReques fix RESTCatalog fix checkstyle fail fix error change in RESTCatalog --- .../apache/paimon/catalog/DatabaseChange.java | 7 ++- .../org/apache/paimon/jdbc/JdbcUtils.java | 27 +++++---- .../org/apache/paimon/rest/RESTCatalog.java | 41 +++++++++++-- .../rest/requests/AlertDatabaseRequest.java | 59 +++++++++++++++++++ .../rest/requests/CreateDatabaseRequest.java | 11 ---- .../rest/responses/AlertDatabaseResponse.java | 53 +++++++++++++++++ .../apache/paimon/rest/MockRESTMessage.java | 3 +- .../paimon/rest/RESTObjectMapperTest.java | 1 - 8 files changed, 172 insertions(+), 30 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java index 5cb6bbfcff3a..8503163ae762 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java @@ -20,6 +20,7 @@ import java.util.Map; +/** define change to the database property. */ public interface DatabaseChange { static DatabaseChange setProperty(String property, String value) { return new SetProperty(property, value); @@ -31,7 +32,8 @@ static DatabaseChange removeProperty(String property) { String apply(Map parameter); - public static final class SetProperty implements DatabaseChange { + /** Set property for database change. */ + final class SetProperty implements DatabaseChange { private final String property; private final String value; @@ -54,7 +56,8 @@ public String apply(Map parameter) { } } - public static final class RemoveProperty implements DatabaseChange { + /** Remove property for database change. */ + final class RemoveProperty implements DatabaseChange { private final String property; private RemoveProperty(String property) { diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java index d729a1bfbaef..78796b826cb9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java @@ -424,16 +424,14 @@ public static boolean deleteProperties( Stream.concat(Stream.of(storeKey, databaseName), properties.stream()) .toArray(String[]::new); - return execute(connections, JdbcUtils.deletePropertiesStatement(properties), args) > 0; - } - - private static String deletePropertiesStatement(Set properties) { - StringBuilder sqlStatement = new StringBuilder(JdbcUtils.DELETE_DATABASE_PROPERTIES_SQL); - String values = - String.join(",", Collections.nCopies(properties.size(), String.valueOf('?'))); - sqlStatement.append("(").append(values).append(")"); - - return sqlStatement.toString(); + int deleteRecords = + execute(connections, JdbcUtils.deletePropertiesStatement(properties), args); + if (deleteRecords > 0) { + return true; + } + throw new IllegalStateException( + String.format( + "Failed to delete: %d of %d succeeded", deleteRecords, properties.size())); } public static void createDistributedLockTable(JdbcClientPool connections, Options options) @@ -460,4 +458,13 @@ public static void release(JdbcClientPool connections, String lockId) DistributedLockDialectFactory.create(connections.getProtocol()) .releaseLock(connections, lockId); } + + private static String deletePropertiesStatement(Set properties) { + StringBuilder sqlStatement = new StringBuilder(JdbcUtils.DELETE_DATABASE_PROPERTIES_SQL); + String values = + String.join(",", Collections.nCopies(properties.size(), String.valueOf('?'))); + sqlStatement.append("(").append(values).append(")"); + + return sqlStatement.toString(); + } } 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 405b1629366c..5ee2b33d7ef2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -18,6 +18,7 @@ package org.apache.paimon.rest; +import java.util.ArrayList; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.DatabaseChange; @@ -32,7 +33,9 @@ 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.AlertDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.AlertDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; import org.apache.paimon.rest.responses.CreateDatabaseResponse; import org.apache.paimon.rest.responses.DatabaseName; @@ -44,10 +47,11 @@ import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.time.Duration; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -137,12 +141,14 @@ public List listDatabases() { @Override public void createDatabase(String name, boolean ignoreIfExists, Map properties) throws DatabaseAlreadyExistException { - CreateDatabaseRequest request = new CreateDatabaseRequest(name, ignoreIfExists, properties); + CreateDatabaseRequest request = new CreateDatabaseRequest(name, properties); try { client.post( resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); } catch (AlreadyExistsException e) { - throw new DatabaseAlreadyExistException(name); + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(name); + } } } @@ -176,7 +182,34 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade @Override public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { - throw new UnsupportedOperationException(); + try { + Map insertProperties = Maps.newHashMap(); + List removeProperties = Lists.newArrayList(); + changes.forEach( + change -> { + if (change instanceof DatabaseChange.SetProperty) { + DatabaseChange.SetProperty setProperty = + (DatabaseChange.SetProperty) change; + insertProperties.put(setProperty.property(), setProperty.value()); + } else { + removeProperties.add( + ((DatabaseChange.RemoveProperty) change).property()); + } + }); + AlertDatabaseRequest request = + new AlertDatabaseRequest(removeProperties, insertProperties); + AlertDatabaseResponse response = + client.post( + resourcePaths.database(name), + request, + AlertDatabaseResponse.class, + headers()); + if (response.getUpdated().isEmpty()) { + throw new IllegalStateException("Failed to update properties"); + } + } catch (NoSuchResourceException e) { + throw new DatabaseNotExistException(name); + } } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java new file mode 100644 index 000000000000..e1b3ac22245f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.rest.RESTRequest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Map; + +/** Request for alerting database. */ +public class AlertDatabaseRequest implements RESTRequest { + + private static final String FIELD_REMOVALS = "removals"; + private static final String FIELD_UPDATES = "updates"; + + @JsonProperty(FIELD_REMOVALS) + private List removals; + + @JsonProperty(FIELD_UPDATES) + private Map updates; + + @JsonCreator + public AlertDatabaseRequest( + @JsonProperty(FIELD_REMOVALS) List removals, + @JsonProperty(FIELD_UPDATES) Map updates) { + this.removals = removals; + this.updates = updates; + } + + @JsonGetter(FIELD_REMOVALS) + public List getRemovals() { + return removals; + } + + @JsonGetter(FIELD_UPDATES) + public Map getUpdates() { + return updates; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java index 6067bf544b87..07e5cf2462f2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -30,25 +30,19 @@ public class CreateDatabaseRequest implements RESTRequest { private static final String FIELD_NAME = "name"; - private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; private static final String FIELD_OPTIONS = "options"; @JsonProperty(FIELD_NAME) private String name; - @JsonProperty(FIELD_IGNORE_IF_EXISTS) - private boolean ignoreIfExists; - @JsonProperty(FIELD_OPTIONS) private Map options; @JsonCreator public CreateDatabaseRequest( @JsonProperty(FIELD_NAME) String name, - @JsonProperty(FIELD_IGNORE_IF_EXISTS) boolean ignoreIfExists, @JsonProperty(FIELD_OPTIONS) Map options) { this.name = name; - this.ignoreIfExists = ignoreIfExists; this.options = options; } @@ -57,11 +51,6 @@ public String getName() { return name; } - @JsonGetter(FIELD_IGNORE_IF_EXISTS) - public boolean getIgnoreIfExists() { - return ignoreIfExists; - } - @JsonGetter(FIELD_OPTIONS) public Map getOptions() { return options; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java new file mode 100644 index 000000000000..88125f64adbb --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.responses; + +import org.apache.paimon.rest.RESTResponse; + +import java.util.List; + +/** Response for alerting database. */ +public class AlertDatabaseResponse implements RESTResponse { + + // List of namespace property keys that were removed + private List removed; + // List of namespace property keys that were added or updated + private List updated; + // List of properties that were requested for removal that were not found in the namespace's + // properties + private List missing; + + public AlertDatabaseResponse(List removed, List updated, List missing) { + this.removed = removed; + this.updated = updated; + this.missing = missing; + } + + public List getRemoved() { + return removed; + } + + public List getUpdated() { + return updated; + } + + public List getMissing() { + return missing; + } +} 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 a605e5e77c2a..489f9b94759e 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 @@ -40,10 +40,9 @@ public static String databaseName() { } public static CreateDatabaseRequest createDatabaseRequest(String name) { - boolean ignoreIfExists = true; Map options = new HashMap<>(); options.put("a", "b"); - return new CreateDatabaseRequest(name, ignoreIfExists, options); + return new CreateDatabaseRequest(name, options); } public static CreateDatabaseResponse createDatabaseResponse(String name) { 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 7fee81ef1024..3c9c29af1100 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 @@ -68,7 +68,6 @@ public void createDatabaseRequestParseTest() throws Exception { String requestStr = mapper.writeValueAsString(request); CreateDatabaseRequest parseData = mapper.readValue(requestStr, CreateDatabaseRequest.class); assertEquals(request.getName(), parseData.getName()); - assertEquals(request.getIgnoreIfExists(), parseData.getIgnoreIfExists()); assertEquals(request.getOptions().size(), parseData.getOptions().size()); } From ecd6a90adf583a89eee2bbca53130f68f071b4d9 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 11:04:38 +0800 Subject: [PATCH 03/19] fix compile fail --- .../src/main/java/org/apache/paimon/rest/RESTCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index 5ee2b33d7ef2..4944248349f1 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,7 +18,6 @@ package org.apache.paimon.rest; -import java.util.ArrayList; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.DatabaseChange; @@ -52,6 +51,7 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.time.Duration; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; From 95c88c2c11460409e9e41d7392941cc405161c42 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 11:23:23 +0800 Subject: [PATCH 04/19] add getAddAndRemovePropertiesFromDatabaseChanges in AbstractCatalog to support get set and remove from change --- .../paimon/catalog/AbstractCatalog.java | 27 +++++++++++++++++-- .../apache/paimon/catalog/DatabaseChange.java | 17 +++--------- .../org/apache/paimon/jdbc/JdbcCatalog.java | 18 ++++--------- .../org/apache/paimon/hive/HiveCatalog.java | 11 +++++++- 4 files changed, 43 insertions(+), 30 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 d6e07c03d8b3..5b42466aa7cc 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 @@ -42,8 +42,12 @@ import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.system.SystemTableLoader; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; + import javax.annotation.Nullable; import java.io.IOException; @@ -56,6 +60,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.paimon.CoreOptions.TYPE; @@ -234,7 +239,7 @@ public void alertDatabase(String name, List changes, boolean ign throws DatabaseNotExistException { checkNotSystemDatabase(name); try { - getDatabase(name); + alertDatabaseImpl(name, changes); } catch (DatabaseNotExistException e) { if (ignoreIfNotExists) { return; @@ -243,7 +248,25 @@ public void alertDatabase(String name, List changes, boolean ign } } - protected abstract void alertDatabaseImpl(String name, List changes); + protected abstract void alertDatabaseImpl(String name, List changes) + throws DatabaseNotExistException; + + protected Pair, Set> getAddAndRemovePropertiesFromDatabaseChanges( + List changes) { + Map insertProperties = Maps.newHashMap(); + Set removeProperties = Sets.newHashSet(); + changes.forEach( + change -> { + if (change instanceof DatabaseChange.SetProperty) { + DatabaseChange.SetProperty setProperty = + (DatabaseChange.SetProperty) change; + insertProperties.put(setProperty.property(), setProperty.value()); + } else { + removeProperties.add(((DatabaseChange.RemoveProperty) change).property()); + } + }); + return Pair.of(insertProperties, removeProperties); + } @Override public List listTables(String databaseName) throws DatabaseNotExistException { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java index 8503163ae762..c972d7351727 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java @@ -18,10 +18,9 @@ package org.apache.paimon.catalog; -import java.util.Map; - /** define change to the database property. */ public interface DatabaseChange { + static DatabaseChange setProperty(String property, String value) { return new SetProperty(property, value); } @@ -30,10 +29,9 @@ static DatabaseChange removeProperty(String property) { return new RemoveProperty(property); } - String apply(Map parameter); - /** Set property for database change. */ final class SetProperty implements DatabaseChange { + private final String property; private final String value; @@ -49,15 +47,11 @@ public String property() { public String value() { return this.value; } - - @Override - public String apply(Map parameter) { - return parameter.put(property, value); - } } /** Remove property for database change. */ final class RemoveProperty implements DatabaseChange { + private final String property; private RemoveProperty(String property) { @@ -67,10 +61,5 @@ private RemoveProperty(String property) { public String property() { return this.property; } - - @Override - public String apply(Map parameter) { - return parameter.remove(property); - } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 60a0a1babcd8..9d125fe1f32b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -34,12 +34,12 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; -import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -203,18 +203,10 @@ protected void dropDatabaseImpl(String name) { @Override protected void alertDatabaseImpl(String name, List changes) { - Map insertProperties = Maps.newHashMap(); - Set removeProperties = Sets.newHashSet(); - changes.forEach( - change -> { - if (change instanceof DatabaseChange.SetProperty) { - DatabaseChange.SetProperty setProperty = - (DatabaseChange.SetProperty) change; - insertProperties.put(setProperty.property(), setProperty.value()); - } else { - removeProperties.add(((DatabaseChange.RemoveProperty) change).property()); - } - }); + Pair, Set> insertProperties2removeProperties = + getAddAndRemovePropertiesFromDatabaseChanges(changes); + Map insertProperties = insertProperties2removeProperties.getLeft(); + Set removeProperties = insertProperties2removeProperties.getRight(); if (!insertProperties.isEmpty()) { insertProperties(connections, catalogKey, name, insertProperties); } 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 88d1e493d919..399ddabcf68b 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 @@ -414,7 +414,16 @@ protected void alertDatabaseImpl(String name, List changes) { Database database = clients.run(client -> client.getDatabase(name)); Map parameter = Maps.newHashMap(); parameter.putAll(database.getParameters()); - changes.forEach(change -> change.apply(parameter)); + Pair, Set> insertProperties2removeProperties = + getAddAndRemovePropertiesFromDatabaseChanges(changes); + Map insertProperties = insertProperties2removeProperties.getLeft(); + Set removeProperties = insertProperties2removeProperties.getRight(); + if (insertProperties.size() > 0) { + parameter.putAll(insertProperties); + } + if (removeProperties.size() > 0) { + parameter.keySet().removeAll(removeProperties); + } Map newProperties = Collections.unmodifiableMap(parameter); Database alertDatabase = convertToHiveDatabase(name, newProperties); clients.execute(client -> client.alterDatabase(name, alertDatabase)); From 8b97cf712810885aa0381dbc8a51ce92b22269d0 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 11:46:27 +0800 Subject: [PATCH 05/19] fix typo --- .../paimon/catalog/AbstractCatalog.java | 28 ++---------------- .../apache/paimon/catalog/CachingCatalog.java | 4 +-- .../org/apache/paimon/catalog/Catalog.java | 6 ++-- .../apache/paimon/catalog/DatabaseChange.java | 26 +++++++++++++++++ .../paimon/catalog/DelegateCatalog.java | 4 +-- .../paimon/catalog/FileSystemCatalog.java | 2 +- .../org/apache/paimon/jdbc/JdbcCatalog.java | 4 +-- .../paimon/privilege/PrivilegedCatalog.java | 4 +-- .../org/apache/paimon/rest/RESTCatalog.java | 29 +++++++------------ ...Request.java => AlterDatabaseRequest.java} | 14 ++++----- .../rest/responses/AlertDatabaseResponse.java | 2 +- .../org/apache/paimon/hive/HiveCatalog.java | 12 ++++---- 12 files changed, 65 insertions(+), 70 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/rest/requests/{AlertDatabaseRequest.java => AlterDatabaseRequest.java} (85%) 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 5b42466aa7cc..2f1d35ec1bdc 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 @@ -42,12 +42,8 @@ import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.system.SystemTableLoader; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; -import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; -import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; - import javax.annotation.Nullable; import java.io.IOException; @@ -60,7 +56,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.stream.Collectors; import static org.apache.paimon.CoreOptions.TYPE; @@ -235,11 +230,11 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade protected abstract void dropDatabaseImpl(String name); @Override - public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { checkNotSystemDatabase(name); try { - alertDatabaseImpl(name, changes); + alterDatabaseImpl(name, changes); } catch (DatabaseNotExistException e) { if (ignoreIfNotExists) { return; @@ -248,26 +243,9 @@ public void alertDatabase(String name, List changes, boolean ign } } - protected abstract void alertDatabaseImpl(String name, List changes) + protected abstract void alterDatabaseImpl(String name, List changes) throws DatabaseNotExistException; - protected Pair, Set> getAddAndRemovePropertiesFromDatabaseChanges( - List changes) { - Map insertProperties = Maps.newHashMap(); - Set removeProperties = Sets.newHashSet(); - changes.forEach( - change -> { - if (change instanceof DatabaseChange.SetProperty) { - DatabaseChange.SetProperty setProperty = - (DatabaseChange.SetProperty) change; - insertProperties.put(setProperty.property(), setProperty.value()); - } else { - removeProperties.add(((DatabaseChange.RemoveProperty) change).property()); - } - }); - return Pair.of(insertProperties, removeProperties); - } - @Override public List listTables(String databaseName) throws DatabaseNotExistException { if (isSystemDatabase(databaseName)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 4f4b90c945b7..a16a009deb61 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -188,9 +188,9 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { - super.alertDatabase(name, changes, ignoreIfNotExists); + super.alterDatabase(name, changes, ignoreIfNotExists); databaseCache.invalidate(name); } 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 698aaf3e69e1..5965f206f424 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 @@ -126,16 +126,16 @@ void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException; /** - * Alert a database. + * Alter a database. * - * @param name Name of the database to alert. + * @param name Name of the database to alter. * @param changes a collection of changes to apply to the database. * @param ignoreIfNotExists Flag to specify behavior when the database does not exist: if set to * false, throw an exception, if set to true, do nothing. * @throws DatabaseNotExistException if the given database is not exist and ignoreIfNotExists is * false */ - void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException; /** diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java index c972d7351727..ffddf115c0b8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java @@ -18,6 +18,15 @@ package org.apache.paimon.catalog; +import org.apache.paimon.utils.Pair; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; + +import java.util.List; +import java.util.Map; +import java.util.Set; + /** define change to the database property. */ public interface DatabaseChange { @@ -29,6 +38,23 @@ static DatabaseChange removeProperty(String property) { return new RemoveProperty(property); } + static Pair, Set> getAddAndRemoveProperties( + List changes) { + Map insertProperties = Maps.newHashMap(); + Set removeProperties = Sets.newHashSet(); + changes.forEach( + change -> { + if (change instanceof DatabaseChange.SetProperty) { + DatabaseChange.SetProperty setProperty = + (DatabaseChange.SetProperty) change; + insertProperties.put(setProperty.property(), setProperty.value()); + } else { + removeProperties.add(((DatabaseChange.RemoveProperty) change).property()); + } + }); + return Pair.of(insertProperties, removeProperties); + } + /** Set property for database change. */ final class SetProperty implements DatabaseChange { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index a44eaef1c955..537f9c20b22e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -85,9 +85,9 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { - wrapped.alertDatabase(name, changes, ignoreIfNotExists); + wrapped.alterDatabase(name, changes, ignoreIfNotExists); } @Override 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 f68df4d99ced..acaee0da97e6 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 @@ -93,7 +93,7 @@ protected void dropDatabaseImpl(String name) { } @Override - protected void alertDatabaseImpl(String name, List changes) { + protected void alterDatabaseImpl(String name, List changes) { throw new UnsupportedOperationException("Alert database is not supported."); } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 9d125fe1f32b..53c55d77b088 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -202,9 +202,9 @@ protected void dropDatabaseImpl(String name) { } @Override - protected void alertDatabaseImpl(String name, List changes) { + protected void alterDatabaseImpl(String name, List changes) { Pair, Set> insertProperties2removeProperties = - getAddAndRemovePropertiesFromDatabaseChanges(changes); + DatabaseChange.getAddAndRemoveProperties(changes); Map insertProperties = insertProperties2removeProperties.getLeft(); Set removeProperties = insertProperties2removeProperties.getRight(); if (!insertProperties.isEmpty()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java index bc4eb8b66c5b..af67fd5e3c6f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java @@ -84,10 +84,10 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { privilegeManager.getPrivilegeChecker().assertCanAlertDatabase(name); - super.alertDatabase(name, changes, ignoreIfNotExists); + super.alterDatabase(name, changes, ignoreIfNotExists); } @Override 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 4944248349f1..d888e1edd1e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -32,7 +32,7 @@ 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.AlertDatabaseRequest; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; import org.apache.paimon.rest.responses.AlertDatabaseResponse; import org.apache.paimon.rest.responses.ConfigResponse; @@ -43,11 +43,10 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; -import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; -import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.time.Duration; @@ -55,6 +54,7 @@ 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; @@ -180,24 +180,15 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alertDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { try { - Map insertProperties = Maps.newHashMap(); - List removeProperties = Lists.newArrayList(); - changes.forEach( - change -> { - if (change instanceof DatabaseChange.SetProperty) { - DatabaseChange.SetProperty setProperty = - (DatabaseChange.SetProperty) change; - insertProperties.put(setProperty.property(), setProperty.value()); - } else { - removeProperties.add( - ((DatabaseChange.RemoveProperty) change).property()); - } - }); - AlertDatabaseRequest request = - new AlertDatabaseRequest(removeProperties, insertProperties); + Pair, Set> insertProperties2removeProperties = + DatabaseChange.getAddAndRemoveProperties(changes); + Map insertProperties = insertProperties2removeProperties.getLeft(); + Set removeProperties = insertProperties2removeProperties.getRight(); + AlterDatabaseRequest request = + new AlterDatabaseRequest(removeProperties, insertProperties); AlertDatabaseResponse response = client.post( resourcePaths.database(name), diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java similarity index 85% rename from paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java rename to paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java index e1b3ac22245f..6a9d058a5a5e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlertDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java @@ -24,31 +24,31 @@ 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.Set; -/** Request for alerting database. */ -public class AlertDatabaseRequest implements RESTRequest { +/** Request for altering database. */ +public class AlterDatabaseRequest implements RESTRequest { private static final String FIELD_REMOVALS = "removals"; private static final String FIELD_UPDATES = "updates"; @JsonProperty(FIELD_REMOVALS) - private List removals; + private Set removals; @JsonProperty(FIELD_UPDATES) private Map updates; @JsonCreator - public AlertDatabaseRequest( - @JsonProperty(FIELD_REMOVALS) List removals, + public AlterDatabaseRequest( + @JsonProperty(FIELD_REMOVALS) Set removals, @JsonProperty(FIELD_UPDATES) Map updates) { this.removals = removals; this.updates = updates; } @JsonGetter(FIELD_REMOVALS) - public List getRemovals() { + public Set getRemovals() { return removals; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java index 88125f64adbb..d59ffaed7b9b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java @@ -22,7 +22,7 @@ import java.util.List; -/** Response for alerting database. */ +/** Response for altering database. */ public class AlertDatabaseResponse implements RESTResponse { // List of namespace property keys that were removed 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 399ddabcf68b..cba6a2348619 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 @@ -409,13 +409,13 @@ protected void dropDatabaseImpl(String name) { } @Override - protected void alertDatabaseImpl(String name, List changes) { + protected void alterDatabaseImpl(String name, List changes) { try { Database database = clients.run(client -> client.getDatabase(name)); Map parameter = Maps.newHashMap(); parameter.putAll(database.getParameters()); Pair, Set> insertProperties2removeProperties = - getAddAndRemovePropertiesFromDatabaseChanges(changes); + DatabaseChange.getAddAndRemoveProperties(changes); Map insertProperties = insertProperties2removeProperties.getLeft(); Set removeProperties = insertProperties2removeProperties.getRight(); if (insertProperties.size() > 0) { @@ -425,13 +425,13 @@ protected void alertDatabaseImpl(String name, List changes) { parameter.keySet().removeAll(removeProperties); } Map newProperties = Collections.unmodifiableMap(parameter); - Database alertDatabase = convertToHiveDatabase(name, newProperties); - clients.execute(client -> client.alterDatabase(name, alertDatabase)); + Database alterDatabase = convertToHiveDatabase(name, newProperties); + clients.execute(client -> client.alterDatabase(name, alterDatabase)); } catch (TException e) { - throw new RuntimeException("Failed to alert database " + name, e); + throw new RuntimeException("Failed to alter database " + name, e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted in call to alertDatabase " + name, e); + throw new RuntimeException("Interrupted in call to alterDatabase " + name, e); } } From 9edbd55e8af3a470a1acc8d10288c0a297b56d62 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 14:59:25 +0800 Subject: [PATCH 06/19] add ut for alter database --- .../org/apache/paimon/rest/RESTCatalog.java | 12 ++++---- .../rest/requests/AlterDatabaseRequest.java | 8 ++--- ...sponse.java => AlterDatabaseResponse.java} | 28 ++++++++++++++---- .../paimon/catalog/CachingCatalogTest.java | 23 +++++++++++++++ .../paimon/catalog/CatalogTestBase.java | 29 +++++++++++++++++++ .../apache/paimon/jdbc/JdbcCatalogTest.java | 10 +++++++ .../apache/paimon/rest/MockRESTMessage.java | 15 ++++++++++ .../apache/paimon/rest/RESTCatalogTest.java | 28 ++++++++++++++++++ .../paimon/rest/RESTObjectMapperTest.java | 23 +++++++++++++++ .../apache/paimon/hive/HiveCatalogTest.java | 10 +++++++ 10 files changed, 171 insertions(+), 15 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/rest/responses/{AlertDatabaseResponse.java => AlterDatabaseResponse.java} (59%) 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 d888e1edd1e2..11a23b3c668b 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 @@ -34,7 +34,7 @@ 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.responses.AlertDatabaseResponse; +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; @@ -188,18 +188,20 @@ public void alterDatabase(String name, List changes, boolean ign Map insertProperties = insertProperties2removeProperties.getLeft(); Set removeProperties = insertProperties2removeProperties.getRight(); AlterDatabaseRequest request = - new AlterDatabaseRequest(removeProperties, insertProperties); - AlertDatabaseResponse response = + new AlterDatabaseRequest(new ArrayList<>(removeProperties), insertProperties); + AlterDatabaseResponse response = client.post( resourcePaths.database(name), request, - AlertDatabaseResponse.class, + AlterDatabaseResponse.class, headers()); if (response.getUpdated().isEmpty()) { throw new IllegalStateException("Failed to update properties"); } } catch (NoSuchResourceException e) { - throw new DatabaseNotExistException(name); + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(name); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java index 6a9d058a5a5e..c1330142bb7e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/AlterDatabaseRequest.java @@ -24,8 +24,8 @@ 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.Set; /** Request for altering database. */ public class AlterDatabaseRequest implements RESTRequest { @@ -34,21 +34,21 @@ public class AlterDatabaseRequest implements RESTRequest { private static final String FIELD_UPDATES = "updates"; @JsonProperty(FIELD_REMOVALS) - private Set removals; + private List removals; @JsonProperty(FIELD_UPDATES) private Map updates; @JsonCreator public AlterDatabaseRequest( - @JsonProperty(FIELD_REMOVALS) Set removals, + @JsonProperty(FIELD_REMOVALS) List removals, @JsonProperty(FIELD_UPDATES) Map updates) { this.removals = removals; this.updates = updates; } @JsonGetter(FIELD_REMOVALS) - public Set getRemovals() { + public List getRemovals() { return removals; } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java similarity index 59% rename from paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java rename to paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java index d59ffaed7b9b..08d751dc595c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlertDatabaseResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/AlterDatabaseResponse.java @@ -20,33 +20,49 @@ import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + import java.util.List; /** Response for altering database. */ -public class AlertDatabaseResponse implements RESTResponse { +public class AlterDatabaseResponse implements RESTResponse { + + private static final String FIELD_REMOVED = "removed"; + private static final String FIELD_UPDATED = "updated"; + private static final String FIELD_MISSING = "missing"; - // List of namespace property keys that were removed + @JsonProperty(FIELD_REMOVED) private List removed; - // List of namespace property keys that were added or updated + + @JsonProperty(FIELD_UPDATED) private List updated; - // List of properties that were requested for removal that were not found in the namespace's - // properties + + @JsonProperty(FIELD_MISSING) private List missing; - public AlertDatabaseResponse(List removed, List updated, List missing) { + @JsonCreator + public AlterDatabaseResponse( + @JsonProperty(FIELD_REMOVED) List removed, + @JsonProperty(FIELD_UPDATED) List updated, + @JsonProperty(FIELD_MISSING) List missing) { this.removed = removed; this.updated = updated; this.missing = missing; } + @JsonGetter(FIELD_REMOVED) public List getRemoved() { return removed; } + @JsonGetter(FIELD_UPDATED) public List getUpdated() { return updated; } + @JsonGetter(FIELD_MISSING) public List getMissing() { return missing; } diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java index 4792e33c932b..fee6d1433143 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java @@ -44,6 +44,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.io.FileNotFoundException; import java.time.Duration; @@ -63,6 +64,8 @@ import static org.apache.paimon.options.CatalogOptions.CACHE_MANIFEST_SMALL_FILE_THRESHOLD; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.when; class CachingCatalogTest extends CatalogTestBase { @@ -86,6 +89,26 @@ public void testListDatabasesWhenNoDatabases() { assertThat(databases).contains("db"); } + @Test + public void testInvalidateWhenDatabaseIsAltered() throws Exception { + Catalog mockcatalog = Mockito.mock(Catalog.class); + Catalog catalog = new CachingCatalog(mockcatalog); + String databaseName = "db"; + boolean ignoreIfExists = false; + Database database = Database.of(databaseName); + Database secondDatabase = Database.of(databaseName); + when(mockcatalog.getDatabase(databaseName)).thenReturn(database, secondDatabase); + doNothing().when(mockcatalog).alterDatabase(databaseName, emptyList(), ignoreIfExists); + Database cachingDatabase = catalog.getDatabase(databaseName); + assertThat(cachingDatabase.name()).isEqualTo(databaseName); + catalog.alterDatabase(databaseName, emptyList(), ignoreIfExists); + Database newCachingDatabase = catalog.getDatabase(databaseName); + // same as secondDatabase means cache is invalidated, so call getDatabase again then return + // secondDatabase + assertThat(newCachingDatabase).isNotSameAs(database); + assertThat(newCachingDatabase).isSameAs(secondDatabase); + } + @Test public void testInvalidateSystemTablesIfBaseTableIsModified() throws Exception { Catalog catalog = new CachingCatalog(this.catalog); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 98a9b92c5c38..077d9a8fc7f2 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -54,6 +54,7 @@ import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; /** Base test class of paimon catalog in {@link Catalog}. */ public abstract class CatalogTestBase { @@ -960,4 +961,32 @@ public void testTableUUID() throws Exception { assertThat(Long.parseLong(uuid.substring((identifier.getFullName() + ".").length()))) .isGreaterThan(0); } + + protected void alterDatabaseAddPropertyWhenSupport() throws Exception { + // Alter database + String databaseName = "db_to_alter_add"; + catalog.createDatabase(databaseName, false); + catalog.alterDatabase( + databaseName, + Lists.newArrayList(DatabaseChange.setProperty("key", "value")), + false); + Database db = catalog.getDatabase(databaseName); + assertEquals("value", db.options().get("key")); + } + + protected void alterDatabaseRemovePropertyWhenSupport() throws Exception { + // Alter database + String databaseName = "db_to_alter_remove"; + String key = "key"; + String value = "value"; + catalog.createDatabase(databaseName, false); + catalog.alterDatabase( + databaseName, Lists.newArrayList(DatabaseChange.setProperty(key, value)), false); + Database db = catalog.getDatabase(databaseName); + assertEquals(value, db.options().get(key)); + catalog.alterDatabase( + databaseName, Lists.newArrayList(DatabaseChange.removeProperty(key)), false); + db = catalog.getDatabase(databaseName); + assertEquals(false, db.options().containsKey(key)); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index f5befc724f8b..13d8ba67a55b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -122,4 +122,14 @@ public void testSerializeTable() throws Exception { } }); } + + @Test + public void testAlterDatabaseAddProperty() throws Exception { + this.alterDatabaseAddPropertyWhenSupport(); + } + + @Test + public void testAlterDatabaseRemoveProperty() throws Exception { + this.alterDatabaseRemovePropertyWhenSupport(); + } } 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 489f9b94759e..821257a0e10e 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,13 +18,17 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; 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; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -68,4 +72,15 @@ public static ListDatabasesResponse listDatabasesResponse(String name) { public static ErrorResponse noSuchResourceExceptionErrorResponse() { return new ErrorResponse("message", 404, new ArrayList<>()); } + + public static AlterDatabaseRequest alterDatabaseRequest() { + Map add = new HashMap<>(); + add.put("add", "value"); + return new AlterDatabaseRequest(Lists.newArrayList("remove"), add); + } + + public static AlterDatabaseResponse alterDatabaseResponse() { + return new AlterDatabaseResponse( + Lists.newArrayList("remove"), Lists.newArrayList("add"), new ArrayList<>()); + } } 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 0fff81afdcde..9b1582929560 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 @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Database; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; +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; @@ -185,6 +186,33 @@ public void testDropDatabaseWhenCascadeIsFalseAndTablesExist() throws Exception 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.alterDatabase(name, new ArrayList<>(), true)); + } + + @Test + public void testAlterDatabaseWhenDatabaseNotExistAndIgnoreIfNotExistsIsFalse() + 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)); + } + 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 3c9c29af1100..e930daea5c92 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -18,7 +18,9 @@ package org.apache.paimon.rest; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +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; @@ -103,4 +105,25 @@ public void listDatabaseResponseParseTest() throws Exception { assertEquals(response.getDatabases().size(), parseData.getDatabases().size()); assertEquals(name, parseData.getDatabases().get(0).getName()); } + + @Test + public void alterDatabaseRequestParseTest() throws Exception { + AlterDatabaseRequest request = MockRESTMessage.alterDatabaseRequest(); + String requestStr = mapper.writeValueAsString(request); + AlterDatabaseRequest parseData = mapper.readValue(requestStr, AlterDatabaseRequest.class); + assertEquals(request.getRemovals().size(), parseData.getRemovals().size()); + assertEquals(request.getUpdates().size(), parseData.getUpdates().size()); + } + + @Test + public void alertDatabaseResponseParseTest() throws Exception { + String name = MockRESTMessage.databaseName(); + AlterDatabaseResponse response = MockRESTMessage.alterDatabaseResponse(); + String responseStr = mapper.writeValueAsString(response); + AlterDatabaseResponse parseData = + mapper.readValue(responseStr, AlterDatabaseResponse.class); + assertEquals(response.getRemoved().size(), parseData.getRemoved().size()); + assertEquals(response.getUpdated().size(), parseData.getUpdated().size()); + assertEquals(response.getMissing().size(), parseData.getMissing().size()); + } } diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index 267bdf0c7100..ceadc9c86ed3 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -173,6 +173,16 @@ private void testHiveConfDirFromEnvImpl() { assertThat(hiveConf.get("hive.metastore.uris")).isEqualTo("dummy-hms"); } + @Test + public void testAlterDatabaseAddProperty() throws Exception { + this.alterDatabaseAddPropertyWhenSupport(); + } + + @Test + public void testAlterDatabaseRemoveProperty() throws Exception { + this.alterDatabaseRemovePropertyWhenSupport(); + } + @Test public void testAddHiveTableParameters() { try { From ba12048474ac01106c0b9d0dc0143f7b0e2d50b5 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 15:16:06 +0800 Subject: [PATCH 07/19] fix typo --- .../apache/paimon/privilege/AllGrantedPrivilegeChecker.java | 2 +- .../java/org/apache/paimon/privilege/PrivilegeChecker.java | 2 +- .../org/apache/paimon/privilege/PrivilegeCheckerImpl.java | 6 +++--- .../java/org/apache/paimon/privilege/PrivilegeType.java | 2 +- .../java/org/apache/paimon/privilege/PrivilegedCatalog.java | 2 +- .../java/org/apache/paimon/rest/RESTObjectMapperTest.java | 3 +-- 6 files changed, 8 insertions(+), 9 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java b/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java index c33a13f69275..8e8e4cd53d04 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/AllGrantedPrivilegeChecker.java @@ -42,7 +42,7 @@ public void assertCanCreateTable(String databaseName) {} public void assertCanDropDatabase(String databaseName) {} @Override - public void assertCanAlertDatabase(String databaseName) {} + public void assertCanAlterDatabase(String databaseName) {} @Override public void assertCanCreateDatabase() {} diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java index ad904f74964a..14cbbc6f36a3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeChecker.java @@ -53,7 +53,7 @@ default void assertCanSelectOrInsert(Identifier identifier) { void assertCanDropDatabase(String databaseName); - void assertCanAlertDatabase(String databaseName); + void assertCanAlterDatabase(String databaseName); void assertCanCreateDatabase(); diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java index ec13446b2afb..7e7876fa4e44 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeCheckerImpl.java @@ -86,10 +86,10 @@ public void assertCanDropDatabase(String databaseName) { } @Override - public void assertCanAlertDatabase(String databaseName) { - if (!check(databaseName, PrivilegeType.ALERT_DATABASE)) { + public void assertCanAlterDatabase(String databaseName) { + if (!check(databaseName, PrivilegeType.ALTER_DATABASE)) { throw new NoPrivilegeException( - user, "database", databaseName, PrivilegeType.ALERT_DATABASE); + user, "database", databaseName, PrivilegeType.ALTER_DATABASE); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java index 0b0aa93281cc..00b3a50596cb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegeType.java @@ -34,7 +34,7 @@ public enum PrivilegeType { CREATE_TABLE(PrivilegeTarget.DATABASE), DROP_DATABASE(PrivilegeTarget.DATABASE), - ALERT_DATABASE(PrivilegeTarget.DATABASE), + ALTER_DATABASE(PrivilegeTarget.DATABASE), CREATE_DATABASE(PrivilegeTarget.CATALOG), // you can create and drop users, grant and revoke any privileges to or from others diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java index af67fd5e3c6f..2209e96742b0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java @@ -86,7 +86,7 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade @Override public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { - privilegeManager.getPrivilegeChecker().assertCanAlertDatabase(name); + privilegeManager.getPrivilegeChecker().assertCanAlterDatabase(name); super.alterDatabase(name, changes, ignoreIfNotExists); } 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 e930daea5c92..0e5a71be39c0 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 @@ -116,8 +116,7 @@ public void alterDatabaseRequestParseTest() throws Exception { } @Test - public void alertDatabaseResponseParseTest() throws Exception { - String name = MockRESTMessage.databaseName(); + public void alterDatabaseResponseParseTest() throws Exception { AlterDatabaseResponse response = MockRESTMessage.alterDatabaseResponse(); String responseStr = mapper.writeValueAsString(response); AlterDatabaseResponse parseData = From 6c77dca2aa90ba83c0cb23bf703135da5abe6092 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 15:47:04 +0800 Subject: [PATCH 08/19] add alter database open api define --- .../org/apache/paimon/rest/RESTCatalog.java | 2 +- .../org/apache/paimon/rest/ResourcePaths.java | 9 +++ paimon-open-api/rest-catalog-open-api.yaml | 79 ++++++++++++++++--- .../open/api/RESTCatalogController.java | 31 ++++++++ 4 files changed, 111 insertions(+), 10 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 11a23b3c668b..b473ea506117 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 @@ -191,7 +191,7 @@ public void alterDatabase(String name, List changes, boolean ign new AlterDatabaseRequest(new ArrayList<>(removeProperties), insertProperties); AlterDatabaseResponse response = client.post( - resourcePaths.database(name), + resourcePaths.databaseProperties(name), request, AlterDatabaseResponse.class, headers()); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java index b58053374daa..51277454ffb0 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 @@ -43,4 +43,13 @@ public String databases() { public String database(String databaseName) { return SLASH.add("v1").add(prefix).add("databases").add(databaseName).toString(); } + + public String databaseProperties(String databaseName) { + return SLASH.add("v1") + .add(prefix) + .add("databases") + .add(databaseName) + .add("properties") + .toString(); + } } diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index 9b69b3de2776..a2d5c6b5508a 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -80,6 +80,43 @@ paths: $ref: '#/components/schemas/ErrorResponse' "500": description: Internal Server Error + /v1/{prefix}/databases/{database}/properties: + post: + tags: + - database + summary: Alter Database + operationId: alterDatabase + 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/AlterDatabaseRequest' + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/AlterDatabaseResponse' + "404": + description: Resource not found + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error /v1/{prefix}/databases/{database}: get: tags: @@ -159,12 +196,22 @@ components: properties: name: type: string - ignoreIfExists: - type: boolean options: type: object additionalProperties: type: string + ErrorResponse: + type: object + properties: + message: + type: string + code: + type: integer + format: int32 + stack: + type: array + items: + type: string CreateDatabaseResponse: type: object properties: @@ -174,15 +221,29 @@ components: type: object additionalProperties: type: string - ErrorResponse: + AlterDatabaseRequest: type: object properties: - message: - type: string - code: - type: integer - format: int32 - stack: + removals: + type: array + items: + type: string + updates: + type: object + additionalProperties: + type: string + AlterDatabaseResponse: + type: object + properties: + removed: + type: array + items: + type: string + updated: + type: array + items: + type: string + missing: type: array items: type: string 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 19f6f8cdf673..d85f059bc588 100644 --- a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -19,7 +19,9 @@ package org.apache.paimon.open.api; import org.apache.paimon.rest.ResourcePaths; +import org.apache.paimon.rest.requests.AlterDatabaseRequest; import org.apache.paimon.rest.requests.CreateDatabaseRequest; +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; @@ -28,6 +30,7 @@ import org.apache.paimon.rest.responses.ListDatabasesResponse; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import io.swagger.v3.oas.annotations.Operation; import io.swagger.v3.oas.annotations.media.Content; @@ -150,4 +153,32 @@ public GetDatabaseResponse getDatabases( }) @DeleteMapping("/v1/{prefix}/databases/{database}") public void dropDatabases(@PathVariable String prefix, @PathVariable String database) {} + + @Operation( + summary = "Alter Database", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = { + @Content(schema = @Schema(implementation = AlterDatabaseResponse.class)) + }), + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/v1/{prefix}/databases/{database}/properties") + public AlterDatabaseResponse alterDatabase( + @PathVariable String prefix, + @PathVariable String database, + @RequestBody AlterDatabaseRequest request) { + return new AlterDatabaseResponse( + Lists.newArrayList("remove"), + Lists.newArrayList("add"), + Lists.newArrayList("missing")); + } } From 5cbb9dbfe0ed0a091ea29e60551743b1024649c5 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 15:57:29 +0800 Subject: [PATCH 09/19] fix typo and update remove properties to remove keys --- .../org/apache/paimon/catalog/DatabaseChange.java | 8 ++++---- .../org/apache/paimon/catalog/FileSystemCatalog.java | 2 +- .../java/org/apache/paimon/jdbc/JdbcCatalog.java | 12 ++++++------ .../main/java/org/apache/paimon/jdbc/JdbcUtils.java | 8 ++++---- .../java/org/apache/paimon/rest/RESTCatalog.java | 10 +++++----- .../java/org/apache/paimon/hive/HiveCatalog.java | 8 ++++---- 6 files changed, 24 insertions(+), 24 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java index ffddf115c0b8..59b015dfc12c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java @@ -38,10 +38,10 @@ static DatabaseChange removeProperty(String property) { return new RemoveProperty(property); } - static Pair, Set> getAddAndRemoveProperties( + static Pair, Set> getAddPropertiesAndRemoveKeys( List changes) { Map insertProperties = Maps.newHashMap(); - Set removeProperties = Sets.newHashSet(); + Set removeKeys = Sets.newHashSet(); changes.forEach( change -> { if (change instanceof DatabaseChange.SetProperty) { @@ -49,10 +49,10 @@ static Pair, Set> getAddAndRemoveProperties( (DatabaseChange.SetProperty) change; insertProperties.put(setProperty.property(), setProperty.value()); } else { - removeProperties.add(((DatabaseChange.RemoveProperty) change).property()); + removeKeys.add(((DatabaseChange.RemoveProperty) change).property()); } }); - return Pair.of(insertProperties, removeProperties); + return Pair.of(insertProperties, removeKeys); } /** Set property for database change. */ 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 acaee0da97e6..4d303cf7d1fc 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 @@ -94,7 +94,7 @@ protected void dropDatabaseImpl(String name) { @Override protected void alterDatabaseImpl(String name, List changes) { - throw new UnsupportedOperationException("Alert database is not supported."); + throw new UnsupportedOperationException("Alter database is not supported."); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 53c55d77b088..7a9b421d464c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -203,15 +203,15 @@ protected void dropDatabaseImpl(String name) { @Override protected void alterDatabaseImpl(String name, List changes) { - Pair, Set> insertProperties2removeProperties = - DatabaseChange.getAddAndRemoveProperties(changes); - Map insertProperties = insertProperties2removeProperties.getLeft(); - Set removeProperties = insertProperties2removeProperties.getRight(); + Pair, Set> insertProperties2removeKeys = + DatabaseChange.getAddPropertiesAndRemoveKeys(changes); + Map insertProperties = insertProperties2removeKeys.getLeft(); + Set removeKeys = insertProperties2removeKeys.getRight(); if (!insertProperties.isEmpty()) { insertProperties(connections, catalogKey, name, insertProperties); } - if (!removeProperties.isEmpty()) { - deleteProperties(connections, catalogKey, name, removeProperties); + if (!removeKeys.isEmpty()) { + deleteProperties(connections, catalogKey, name, removeKeys); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java index 78796b826cb9..936703e83a96 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java @@ -419,19 +419,19 @@ public static boolean deleteProperties( JdbcClientPool connections, String storeKey, String databaseName, - Set properties) { + Set removeKeys) { String[] args = - Stream.concat(Stream.of(storeKey, databaseName), properties.stream()) + Stream.concat(Stream.of(storeKey, databaseName), removeKeys.stream()) .toArray(String[]::new); int deleteRecords = - execute(connections, JdbcUtils.deletePropertiesStatement(properties), args); + execute(connections, JdbcUtils.deletePropertiesStatement(removeKeys), args); if (deleteRecords > 0) { return true; } throw new IllegalStateException( String.format( - "Failed to delete: %d of %d succeeded", deleteRecords, properties.size())); + "Failed to delete: %d of %d succeeded", deleteRecords, removeKeys.size())); } public static void createDistributedLockTable(JdbcClientPool connections, Options options) 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 b473ea506117..9fdb48aee9e4 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 @@ -183,12 +183,12 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { try { - Pair, Set> insertProperties2removeProperties = - DatabaseChange.getAddAndRemoveProperties(changes); - Map insertProperties = insertProperties2removeProperties.getLeft(); - Set removeProperties = insertProperties2removeProperties.getRight(); + Pair, Set> insertProperties2removeKeys = + DatabaseChange.getAddPropertiesAndRemoveKeys(changes); + Map insertProperties = insertProperties2removeKeys.getLeft(); + Set removeKeys = insertProperties2removeKeys.getRight(); AlterDatabaseRequest request = - new AlterDatabaseRequest(new ArrayList<>(removeProperties), insertProperties); + new AlterDatabaseRequest(new ArrayList<>(removeKeys), insertProperties); AlterDatabaseResponse response = client.post( resourcePaths.databaseProperties(name), 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 cba6a2348619..b735abb13726 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 @@ -415,14 +415,14 @@ protected void alterDatabaseImpl(String name, List changes) { Map parameter = Maps.newHashMap(); parameter.putAll(database.getParameters()); Pair, Set> insertProperties2removeProperties = - DatabaseChange.getAddAndRemoveProperties(changes); + DatabaseChange.getAddPropertiesAndRemoveKeys(changes); Map insertProperties = insertProperties2removeProperties.getLeft(); - Set removeProperties = insertProperties2removeProperties.getRight(); + Set removeKeys = insertProperties2removeProperties.getRight(); if (insertProperties.size() > 0) { parameter.putAll(insertProperties); } - if (removeProperties.size() > 0) { - parameter.keySet().removeAll(removeProperties); + if (removeKeys.size() > 0) { + parameter.keySet().removeAll(removeKeys); } Map newProperties = Collections.unmodifiableMap(parameter); Database alterDatabase = convertToHiveDatabase(name, newProperties); From a4d3c4621b060321b7453783654d5533f30fe41e Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 16:54:46 +0800 Subject: [PATCH 10/19] fix add when keys has exist in jdbc catalog --- .../org/apache/paimon/jdbc/JdbcCatalog.java | 31 +++++++++++- .../org/apache/paimon/jdbc/JdbcUtils.java | 50 +++++++++++++++++++ .../paimon/catalog/CatalogTestBase.java | 17 ++++++- .../apache/paimon/jdbc/JdbcCatalogTest.java | 4 +- .../apache/paimon/hive/HiveCatalogTest.java | 4 +- 5 files changed, 98 insertions(+), 8 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 7a9b421d464c..e8f9a14090c2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -40,6 +40,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.guava30.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +63,7 @@ import static org.apache.paimon.jdbc.JdbcUtils.deleteProperties; import static org.apache.paimon.jdbc.JdbcUtils.execute; import static org.apache.paimon.jdbc.JdbcUtils.insertProperties; +import static org.apache.paimon.jdbc.JdbcUtils.updateProperties; import static org.apache.paimon.jdbc.JdbcUtils.updateTable; /* This file is based on source code from the Iceberg Project (http://iceberg.apache.org/), licensed by the Apache @@ -207,11 +209,36 @@ protected void alterDatabaseImpl(String name, List changes) { DatabaseChange.getAddPropertiesAndRemoveKeys(changes); Map insertProperties = insertProperties2removeKeys.getLeft(); Set removeKeys = insertProperties2removeKeys.getRight(); + Map startingProperties = fetchProperties(name); + Map inserts = Maps.newHashMap(); + Map updates = Maps.newHashMap(); + Set removes = Sets.newHashSet(); if (!insertProperties.isEmpty()) { - insertProperties(connections, catalogKey, name, insertProperties); + insertProperties.forEach( + (k, v) -> { + if (!startingProperties.containsKey(k)) { + inserts.put(k, v); + } else { + updates.put(k, v); + } + }); } if (!removeKeys.isEmpty()) { - deleteProperties(connections, catalogKey, name, removeKeys); + removeKeys.forEach( + k -> { + if (startingProperties.containsKey(k)) { + removes.add(k); + } + }); + } + if (!inserts.isEmpty()) { + insertProperties(connections, catalogKey, name, inserts); + } + if (!updates.isEmpty()) { + updateProperties(connections, catalogKey, name, updates); + } + if (!removes.isEmpty()) { + deleteProperties(connections, catalogKey, name, removes); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java index 936703e83a96..1b9e599d72bc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java @@ -415,6 +415,56 @@ private static String insertPropertiesStatement(int size) { return sqlStatement.toString(); } + public static boolean updateProperties( + JdbcClientPool connections, + String storeKey, + String databaseName, + Map properties) { + Stream caseArgs = + properties.entrySet().stream() + .flatMap(entry -> Stream.of(entry.getKey(), entry.getValue())); + Stream whereArgs = + Stream.concat(Stream.of(storeKey, databaseName), properties.keySet().stream()); + + String[] args = Stream.concat(caseArgs, whereArgs).toArray(String[]::new); + + int updatedRecords = + execute(connections, JdbcUtils.updatePropertiesStatement(properties.size()), args); + if (updatedRecords == properties.size()) { + return true; + } + throw new IllegalStateException( + String.format( + "Failed to update: %d of %d succeeded", updatedRecords, properties.size())); + } + + private static String updatePropertiesStatement(int size) { + StringBuilder sqlStatement = + new StringBuilder( + "UPDATE " + + DATABASE_PROPERTIES_TABLE_NAME + + " SET " + + DATABASE_PROPERTY_VALUE + + " = CASE"); + for (int i = 0; i < size; i += 1) { + sqlStatement.append(" WHEN " + DATABASE_PROPERTY_KEY + " = ? THEN ?"); + } + + sqlStatement.append( + " END WHERE " + + CATALOG_KEY + + " = ? AND " + + DATABASE_NAME + + " = ? AND " + + DATABASE_PROPERTY_KEY + + " IN "); + + String values = String.join(",", Collections.nCopies(size, String.valueOf('?'))); + sqlStatement.append("(").append(values).append(")"); + + return sqlStatement.toString(); + } + public static boolean deleteProperties( JdbcClientPool connections, String storeKey, diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 077d9a8fc7f2..51b8342c96e3 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -962,19 +962,27 @@ public void testTableUUID() throws Exception { .isGreaterThan(0); } - protected void alterDatabaseAddPropertyWhenSupport() throws Exception { + protected void alterDatabaseAddPropertyWhenSupportAlter() throws Exception { // Alter database String databaseName = "db_to_alter_add"; catalog.createDatabase(databaseName, false); + // Add property catalog.alterDatabase( databaseName, Lists.newArrayList(DatabaseChange.setProperty("key", "value")), false); Database db = catalog.getDatabase(databaseName); assertEquals("value", db.options().get("key")); + // Update property + catalog.alterDatabase( + databaseName, + Lists.newArrayList(DatabaseChange.setProperty("key", "value1")), + false); + db = catalog.getDatabase(databaseName); + assertEquals("value1", db.options().get("key")); } - protected void alterDatabaseRemovePropertyWhenSupport() throws Exception { + protected void alterDatabaseRemovePropertyWhenSupportAlter() throws Exception { // Alter database String databaseName = "db_to_alter_remove"; String key = "key"; @@ -988,5 +996,10 @@ protected void alterDatabaseRemovePropertyWhenSupport() throws Exception { databaseName, Lists.newArrayList(DatabaseChange.removeProperty(key)), false); db = catalog.getDatabase(databaseName); assertEquals(false, db.options().containsKey(key)); + // Remove non-existent property + catalog.alterDatabase( + databaseName, Lists.newArrayList(DatabaseChange.removeProperty(key)), false); + db = catalog.getDatabase(databaseName); + assertEquals(false, db.options().containsKey(key)); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index 13d8ba67a55b..7a0df7d3d633 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -125,11 +125,11 @@ public void testSerializeTable() throws Exception { @Test public void testAlterDatabaseAddProperty() throws Exception { - this.alterDatabaseAddPropertyWhenSupport(); + this.alterDatabaseAddPropertyWhenSupportAlter(); } @Test public void testAlterDatabaseRemoveProperty() throws Exception { - this.alterDatabaseRemovePropertyWhenSupport(); + this.alterDatabaseRemovePropertyWhenSupportAlter(); } } diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index ceadc9c86ed3..249ca68d9044 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -175,12 +175,12 @@ private void testHiveConfDirFromEnvImpl() { @Test public void testAlterDatabaseAddProperty() throws Exception { - this.alterDatabaseAddPropertyWhenSupport(); + this.alterDatabaseAddPropertyWhenSupportAlter(); } @Test public void testAlterDatabaseRemoveProperty() throws Exception { - this.alterDatabaseRemovePropertyWhenSupport(); + this.alterDatabaseRemovePropertyWhenSupportAlter(); } @Test From 4bb3e5261d2cf89908a624205ee936b8a23bc562 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 17:07:38 +0800 Subject: [PATCH 11/19] update name for getSetPropertiesToRemoveKeys --- .../org/apache/paimon/catalog/DatabaseChange.java | 8 ++++---- .../java/org/apache/paimon/jdbc/JdbcCatalog.java | 12 ++++++------ .../java/org/apache/paimon/rest/RESTCatalog.java | 10 +++++----- .../java/org/apache/paimon/hive/HiveCatalog.java | 12 ++++++------ 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java index 59b015dfc12c..a6166e0971c8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java @@ -38,21 +38,21 @@ static DatabaseChange removeProperty(String property) { return new RemoveProperty(property); } - static Pair, Set> getAddPropertiesAndRemoveKeys( + static Pair, Set> getSetPropertiesToRemoveKeys( List changes) { - Map insertProperties = Maps.newHashMap(); + Map setProperties = Maps.newHashMap(); Set removeKeys = Sets.newHashSet(); changes.forEach( change -> { if (change instanceof DatabaseChange.SetProperty) { DatabaseChange.SetProperty setProperty = (DatabaseChange.SetProperty) change; - insertProperties.put(setProperty.property(), setProperty.value()); + setProperties.put(setProperty.property(), setProperty.value()); } else { removeKeys.add(((DatabaseChange.RemoveProperty) change).property()); } }); - return Pair.of(insertProperties, removeKeys); + return Pair.of(setProperties, removeKeys); } /** Set property for database change. */ diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index e8f9a14090c2..428bb3d0c93c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -205,16 +205,16 @@ protected void dropDatabaseImpl(String name) { @Override protected void alterDatabaseImpl(String name, List changes) { - Pair, Set> insertProperties2removeKeys = - DatabaseChange.getAddPropertiesAndRemoveKeys(changes); - Map insertProperties = insertProperties2removeKeys.getLeft(); - Set removeKeys = insertProperties2removeKeys.getRight(); + Pair, Set> setPropertiesToRemoveKeys = + DatabaseChange.getSetPropertiesToRemoveKeys(changes); + Map setProperties = setPropertiesToRemoveKeys.getLeft(); + Set removeKeys = setPropertiesToRemoveKeys.getRight(); Map startingProperties = fetchProperties(name); Map inserts = Maps.newHashMap(); Map updates = Maps.newHashMap(); Set removes = Sets.newHashSet(); - if (!insertProperties.isEmpty()) { - insertProperties.forEach( + if (!setProperties.isEmpty()) { + setProperties.forEach( (k, v) -> { if (!startingProperties.containsKey(k)) { inserts.put(k, v); 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 9fdb48aee9e4..8ec589e952d2 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 @@ -183,12 +183,12 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { try { - Pair, Set> insertProperties2removeKeys = - DatabaseChange.getAddPropertiesAndRemoveKeys(changes); - Map insertProperties = insertProperties2removeKeys.getLeft(); - Set removeKeys = insertProperties2removeKeys.getRight(); + Pair, Set> setPropertiesToRemoveKeys = + DatabaseChange.getSetPropertiesToRemoveKeys(changes); + Map updateProperties = setPropertiesToRemoveKeys.getLeft(); + Set removeKeys = setPropertiesToRemoveKeys.getRight(); AlterDatabaseRequest request = - new AlterDatabaseRequest(new ArrayList<>(removeKeys), insertProperties); + new AlterDatabaseRequest(new ArrayList<>(removeKeys), updateProperties); AlterDatabaseResponse response = client.post( resourcePaths.databaseProperties(name), 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 b735abb13726..81c2ce5cfa46 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 @@ -414,12 +414,12 @@ protected void alterDatabaseImpl(String name, List changes) { Database database = clients.run(client -> client.getDatabase(name)); Map parameter = Maps.newHashMap(); parameter.putAll(database.getParameters()); - Pair, Set> insertProperties2removeProperties = - DatabaseChange.getAddPropertiesAndRemoveKeys(changes); - Map insertProperties = insertProperties2removeProperties.getLeft(); - Set removeKeys = insertProperties2removeProperties.getRight(); - if (insertProperties.size() > 0) { - parameter.putAll(insertProperties); + Pair, Set> setPropertiesToRemoveKeys = + DatabaseChange.getSetPropertiesToRemoveKeys(changes); + Map setProperties = setPropertiesToRemoveKeys.getLeft(); + Set removeKeys = setPropertiesToRemoveKeys.getRight(); + if (setProperties.size() > 0) { + parameter.putAll(setProperties); } if (removeKeys.size() > 0) { parameter.keySet().removeAll(removeKeys); From 29da0ecd2f8b1149271dc8932b27b3ac3e10be7d Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 17:29:58 +0800 Subject: [PATCH 12/19] add alter database ut for FileSystemCatalog --- .../paimon/catalog/CatalogTestBase.java | 30 +++++-------------- .../paimon/catalog/FileSystemCatalogTest.java | 11 +++++++ .../apache/paimon/jdbc/JdbcCatalogTest.java | 9 ++---- .../apache/paimon/hive/HiveCatalogTest.java | 9 ++---- 4 files changed, 23 insertions(+), 36 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 51b8342c96e3..029fbb82b6f4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -962,36 +962,22 @@ public void testTableUUID() throws Exception { .isGreaterThan(0); } - protected void alterDatabaseAddPropertyWhenSupportAlter() throws Exception { + protected void alterDatabaseWhenSupportAlter() throws Exception { // Alter database - String databaseName = "db_to_alter_add"; + String databaseName = "db_to_alter_alert"; catalog.createDatabase(databaseName, false); + String key = "key"; // Add property catalog.alterDatabase( - databaseName, - Lists.newArrayList(DatabaseChange.setProperty("key", "value")), - false); + databaseName, Lists.newArrayList(DatabaseChange.setProperty(key, "value")), false); Database db = catalog.getDatabase(databaseName); - assertEquals("value", db.options().get("key")); + assertEquals("value", db.options().get(key)); // Update property catalog.alterDatabase( - databaseName, - Lists.newArrayList(DatabaseChange.setProperty("key", "value1")), - false); + databaseName, Lists.newArrayList(DatabaseChange.setProperty(key, "value1")), false); db = catalog.getDatabase(databaseName); - assertEquals("value1", db.options().get("key")); - } - - protected void alterDatabaseRemovePropertyWhenSupportAlter() throws Exception { - // Alter database - String databaseName = "db_to_alter_remove"; - String key = "key"; - String value = "value"; - catalog.createDatabase(databaseName, false); - catalog.alterDatabase( - databaseName, Lists.newArrayList(DatabaseChange.setProperty(key, value)), false); - Database db = catalog.getDatabase(databaseName); - assertEquals(value, db.options().get(key)); + assertEquals("value1", db.options().get(key)); + // remove property catalog.alterDatabase( databaseName, Lists.newArrayList(DatabaseChange.removeProperty(key)), false); db = catalog.getDatabase(databaseName); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index 303a9d8733d4..aa2ddaf4f7a7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -27,7 +27,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.ArrayList; + import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link FileSystemCatalog}. */ public class FileSystemCatalogTest extends CatalogTestBase { @@ -67,4 +70,12 @@ public void testCreateTableAllowUpperCase() throws Exception { .isThrownBy(() -> catalog.createTable(identifier, schema, false)) .withMessage("Field name [Pk1, Col1] cannot contain upper case in the catalog."); } + + @Test + public void testAlterDatabase() throws Exception { + String databaseName = "test_alter_db"; + catalog.createDatabase(databaseName, false); + assertThatThrownBy(() -> catalog.alterDatabase(databaseName, new ArrayList<>(), false)) + .isInstanceOf(UnsupportedOperationException.class); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index 7a0df7d3d633..f01a46fd6bb4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -124,12 +124,7 @@ public void testSerializeTable() throws Exception { } @Test - public void testAlterDatabaseAddProperty() throws Exception { - this.alterDatabaseAddPropertyWhenSupportAlter(); - } - - @Test - public void testAlterDatabaseRemoveProperty() throws Exception { - this.alterDatabaseRemovePropertyWhenSupportAlter(); + public void testAlterDatabase() throws Exception { + this.alterDatabaseWhenSupportAlter(); } } diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index 249ca68d9044..e3b48f02a696 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -174,13 +174,8 @@ private void testHiveConfDirFromEnvImpl() { } @Test - public void testAlterDatabaseAddProperty() throws Exception { - this.alterDatabaseAddPropertyWhenSupportAlter(); - } - - @Test - public void testAlterDatabaseRemoveProperty() throws Exception { - this.alterDatabaseRemovePropertyWhenSupportAlter(); + public void testAlterDatabase() throws Exception { + this.alterDatabaseWhenSupportAlter(); } @Test From c2cfcbe39c87ca1ea92f1bfdc96673efd681f0d7 Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 17:42:32 +0800 Subject: [PATCH 13/19] add alter database in catalog-api doc --- docs/content/program-api/catalog-api.md | 24 +++++++++++++++++++ paimon-open-api/rest-catalog-open-api.yaml | 2 +- .../open/api/RESTCatalogController.java | 2 +- 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/docs/content/program-api/catalog-api.md b/docs/content/program-api/catalog-api.md index 570577437d86..6dbe8d766878 100644 --- a/docs/content/program-api/catalog-api.md +++ b/docs/content/program-api/catalog-api.md @@ -102,6 +102,30 @@ public class DropDatabase { } ``` +## Alter Database + +You can use the catalog to alter databases.(ps: only support hive and jdbc catalog) + +```java +import java.util.ArrayList; +import org.apache.paimon.catalog.Catalog; + +public class AlterDatabase { + + public static void main(String[] args) { + try { + Catalog catalog = CreateCatalog.createHiveCatalog(); + List changes = new ArrayList<>(); + changes.add(DatabaseChange.setProperty("k1", "v1")); + changes.add(DatabaseChange.removeProperty("k2")); + catalog.alterDatabase("my_db", changes, true); + } catch (Catalog.DatabaseNotExistException e) { + // do something + } + } +} +``` + ## Determine Whether Table Exists You can use the catalog to determine whether the table exists diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml index a2d5c6b5508a..f7f9529f53dd 100644 --- a/paimon-open-api/rest-catalog-open-api.yaml +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -153,7 +153,7 @@ paths: tags: - database summary: Drop Database - operationId: dropDatabases + operationId: dropDatabase parameters: - name: prefix in: path 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 d85f059bc588..5331b65d71b6 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 @@ -152,7 +152,7 @@ public GetDatabaseResponse getDatabases( content = {@Content(schema = @Schema())}) }) @DeleteMapping("/v1/{prefix}/databases/{database}") - public void dropDatabases(@PathVariable String prefix, @PathVariable String database) {} + public void dropDatabase(@PathVariable String prefix, @PathVariable String database) {} @Operation( summary = "Alter Database", From 240c30ba678b99b8f01b70748de8533058ab5bfa Mon Sep 17 00:00:00 2001 From: yantian Date: Fri, 13 Dec 2024 18:00:47 +0800 Subject: [PATCH 14/19] update alert database add multiply properties and update databases to database when operation on a database fix typo --- docs/content/program-api/catalog-api.md | 4 +-- .../paimon/catalog/CatalogTestBase.java | 31 +++++++++++++++---- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/docs/content/program-api/catalog-api.md b/docs/content/program-api/catalog-api.md index 6dbe8d766878..7e716aad15bb 100644 --- a/docs/content/program-api/catalog-api.md +++ b/docs/content/program-api/catalog-api.md @@ -82,7 +82,7 @@ public class ListDatabases { ## Drop Database -You can use the catalog to drop databases. +You can use the catalog to drop database. ```java import org.apache.paimon.catalog.Catalog; @@ -104,7 +104,7 @@ public class DropDatabase { ## Alter Database -You can use the catalog to alter databases.(ps: only support hive and jdbc catalog) +You can use the catalog to alter database's properties.(ps: only support hive and jdbc catalog) ```java import java.util.ArrayList; diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 029fbb82b6f4..986b4b83c7cb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -964,28 +964,47 @@ public void testTableUUID() throws Exception { protected void alterDatabaseWhenSupportAlter() throws Exception { // Alter database - String databaseName = "db_to_alter_alert"; + String databaseName = "db_to_alter"; catalog.createDatabase(databaseName, false); - String key = "key"; + String key = "key1"; + String key2 = "key2"; // Add property catalog.alterDatabase( - databaseName, Lists.newArrayList(DatabaseChange.setProperty(key, "value")), false); + databaseName, + Lists.newArrayList( + DatabaseChange.setProperty(key, "value"), + DatabaseChange.setProperty(key2, "value")), + false); Database db = catalog.getDatabase(databaseName); assertEquals("value", db.options().get(key)); + assertEquals("value", db.options().get(key2)); // Update property catalog.alterDatabase( - databaseName, Lists.newArrayList(DatabaseChange.setProperty(key, "value1")), false); + databaseName, + Lists.newArrayList( + DatabaseChange.setProperty(key, "value1"), + DatabaseChange.setProperty(key2, "value1")), + false); db = catalog.getDatabase(databaseName); assertEquals("value1", db.options().get(key)); + assertEquals("value1", db.options().get(key2)); // remove property catalog.alterDatabase( - databaseName, Lists.newArrayList(DatabaseChange.removeProperty(key)), false); + databaseName, + Lists.newArrayList( + DatabaseChange.removeProperty(key), DatabaseChange.removeProperty(key2)), + false); db = catalog.getDatabase(databaseName); assertEquals(false, db.options().containsKey(key)); + assertEquals(false, db.options().containsKey(key2)); // Remove non-existent property catalog.alterDatabase( - databaseName, Lists.newArrayList(DatabaseChange.removeProperty(key)), false); + databaseName, + Lists.newArrayList( + DatabaseChange.removeProperty(key), DatabaseChange.removeProperty(key2)), + false); db = catalog.getDatabase(databaseName); assertEquals(false, db.options().containsKey(key)); + assertEquals(false, db.options().containsKey(key2)); } } From c0e45e5677b93d4ef792ef17b268477d6ddb845f Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 16 Dec 2024 13:41:58 +0800 Subject: [PATCH 15/19] rename DatabaseChange to PropertyChange --- .../paimon/catalog/AbstractCatalog.java | 4 ++-- .../apache/paimon/catalog/CachingCatalog.java | 2 +- .../org/apache/paimon/catalog/Catalog.java | 4 ++-- .../paimon/catalog/DelegateCatalog.java | 2 +- .../paimon/catalog/FileSystemCatalog.java | 2 +- ...atabaseChange.java => PropertyChange.java} | 20 +++++++++---------- .../org/apache/paimon/jdbc/JdbcCatalog.java | 6 +++--- .../paimon/privilege/PrivilegedCatalog.java | 4 ++-- .../org/apache/paimon/rest/RESTCatalog.java | 6 +++--- .../paimon/catalog/CatalogTestBase.java | 12 +++++------ .../org/apache/paimon/hive/HiveCatalog.java | 6 +++--- 11 files changed, 34 insertions(+), 34 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/catalog/{DatabaseChange.java => PropertyChange.java} (81%) 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 2f1d35ec1bdc..2ae256b91517 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 @@ -230,7 +230,7 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade protected abstract void dropDatabaseImpl(String name); @Override - public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { checkNotSystemDatabase(name); try { @@ -243,7 +243,7 @@ public void alterDatabase(String name, List changes, boolean ign } } - protected abstract void alterDatabaseImpl(String name, List changes) + protected abstract void alterDatabaseImpl(String name, List changes) throws DatabaseNotExistException; @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index a16a009deb61..99540cf0cea5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -188,7 +188,7 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { super.alterDatabase(name, changes, ignoreIfNotExists); databaseCache.invalidate(name); 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 5965f206f424..a90fb86ac2d3 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 @@ -129,13 +129,13 @@ void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) * Alter a database. * * @param name Name of the database to alter. - * @param changes a collection of changes to apply to the database. + * @param changes the property changes * @param ignoreIfNotExists Flag to specify behavior when the database does not exist: if set to * false, throw an exception, if set to true, do nothing. * @throws DatabaseNotExistException if the given database is not exist and ignoreIfNotExists is * false */ - void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException; /** diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index 537f9c20b22e..8a80e020e8b8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -85,7 +85,7 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { wrapped.alterDatabase(name, changes, ignoreIfNotExists); } 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 4d303cf7d1fc..0d1a2c4c6621 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 @@ -93,7 +93,7 @@ protected void dropDatabaseImpl(String name) { } @Override - protected void alterDatabaseImpl(String name, List changes) { + protected void alterDatabaseImpl(String name, List changes) { throw new UnsupportedOperationException("Alter database is not supported."); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java b/paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java similarity index 81% rename from paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java rename to paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java index a6166e0971c8..c3423efd081e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DatabaseChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/PropertyChange.java @@ -28,35 +28,35 @@ import java.util.Set; /** define change to the database property. */ -public interface DatabaseChange { +public interface PropertyChange { - static DatabaseChange setProperty(String property, String value) { + static PropertyChange setProperty(String property, String value) { return new SetProperty(property, value); } - static DatabaseChange removeProperty(String property) { + static PropertyChange removeProperty(String property) { return new RemoveProperty(property); } static Pair, Set> getSetPropertiesToRemoveKeys( - List changes) { + List changes) { Map setProperties = Maps.newHashMap(); Set removeKeys = Sets.newHashSet(); changes.forEach( change -> { - if (change instanceof DatabaseChange.SetProperty) { - DatabaseChange.SetProperty setProperty = - (DatabaseChange.SetProperty) change; + if (change instanceof PropertyChange.SetProperty) { + PropertyChange.SetProperty setProperty = + (PropertyChange.SetProperty) change; setProperties.put(setProperty.property(), setProperty.value()); } else { - removeKeys.add(((DatabaseChange.RemoveProperty) change).property()); + removeKeys.add(((PropertyChange.RemoveProperty) change).property()); } }); return Pair.of(setProperties, removeKeys); } /** Set property for database change. */ - final class SetProperty implements DatabaseChange { + final class SetProperty implements PropertyChange { private final String property; private final String value; @@ -76,7 +76,7 @@ public String value() { } /** Remove property for database change. */ - final class RemoveProperty implements DatabaseChange { + final class RemoveProperty implements PropertyChange { private final String property; diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 428bb3d0c93c..5543f9c3e230 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -23,8 +23,8 @@ import org.apache.paimon.catalog.CatalogLockContext; import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Database; -import org.apache.paimon.catalog.DatabaseChange; 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.operation.Lock; @@ -204,9 +204,9 @@ protected void dropDatabaseImpl(String name) { } @Override - protected void alterDatabaseImpl(String name, List changes) { + protected void alterDatabaseImpl(String name, List changes) { Pair, Set> setPropertiesToRemoveKeys = - DatabaseChange.getSetPropertiesToRemoveKeys(changes); + PropertyChange.getSetPropertiesToRemoveKeys(changes); Map setProperties = setPropertiesToRemoveKeys.getLeft(); Set removeKeys = setPropertiesToRemoveKeys.getRight(); Map startingProperties = fetchProperties(name); diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java index 2209e96742b0..35822471a2d6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java @@ -19,9 +19,9 @@ package org.apache.paimon.privilege; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.DatabaseChange; import org.apache.paimon.catalog.DelegateCatalog; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; import org.apache.paimon.options.Options; @@ -84,7 +84,7 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { privilegeManager.getPrivilegeChecker().assertCanAlterDatabase(name); super.alterDatabase(name, changes, 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 8ec589e952d2..c7204f484058 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,8 +20,8 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Database; -import org.apache.paimon.catalog.DatabaseChange; 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; @@ -180,11 +180,11 @@ public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade } @Override - public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) + public void alterDatabase(String name, List changes, boolean ignoreIfNotExists) throws DatabaseNotExistException { try { Pair, Set> setPropertiesToRemoveKeys = - DatabaseChange.getSetPropertiesToRemoveKeys(changes); + PropertyChange.getSetPropertiesToRemoveKeys(changes); Map updateProperties = setPropertiesToRemoveKeys.getLeft(); Set removeKeys = setPropertiesToRemoveKeys.getRight(); AlterDatabaseRequest request = diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 986b4b83c7cb..31c4c8e682b8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -972,8 +972,8 @@ protected void alterDatabaseWhenSupportAlter() throws Exception { catalog.alterDatabase( databaseName, Lists.newArrayList( - DatabaseChange.setProperty(key, "value"), - DatabaseChange.setProperty(key2, "value")), + PropertyChange.setProperty(key, "value"), + PropertyChange.setProperty(key2, "value")), false); Database db = catalog.getDatabase(databaseName); assertEquals("value", db.options().get(key)); @@ -982,8 +982,8 @@ protected void alterDatabaseWhenSupportAlter() throws Exception { catalog.alterDatabase( databaseName, Lists.newArrayList( - DatabaseChange.setProperty(key, "value1"), - DatabaseChange.setProperty(key2, "value1")), + PropertyChange.setProperty(key, "value1"), + PropertyChange.setProperty(key2, "value1")), false); db = catalog.getDatabase(databaseName); assertEquals("value1", db.options().get(key)); @@ -992,7 +992,7 @@ protected void alterDatabaseWhenSupportAlter() throws Exception { catalog.alterDatabase( databaseName, Lists.newArrayList( - DatabaseChange.removeProperty(key), DatabaseChange.removeProperty(key2)), + PropertyChange.removeProperty(key), PropertyChange.removeProperty(key2)), false); db = catalog.getDatabase(databaseName); assertEquals(false, db.options().containsKey(key)); @@ -1001,7 +1001,7 @@ protected void alterDatabaseWhenSupportAlter() throws Exception { catalog.alterDatabase( databaseName, Lists.newArrayList( - DatabaseChange.removeProperty(key), DatabaseChange.removeProperty(key2)), + PropertyChange.removeProperty(key), PropertyChange.removeProperty(key2)), false); db = catalog.getDatabase(databaseName); assertEquals(false, db.options().containsKey(key)); 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 81c2ce5cfa46..f1e01209dfa4 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 @@ -25,8 +25,8 @@ import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogLockContext; import org.apache.paimon.catalog.CatalogLockFactory; -import org.apache.paimon.catalog.DatabaseChange; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.client.ClientPool; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.fs.FileIO; @@ -409,13 +409,13 @@ protected void dropDatabaseImpl(String name) { } @Override - protected void alterDatabaseImpl(String name, List changes) { + protected void alterDatabaseImpl(String name, List changes) { try { Database database = clients.run(client -> client.getDatabase(name)); Map parameter = Maps.newHashMap(); parameter.putAll(database.getParameters()); Pair, Set> setPropertiesToRemoveKeys = - DatabaseChange.getSetPropertiesToRemoveKeys(changes); + PropertyChange.getSetPropertiesToRemoveKeys(changes); Map setProperties = setPropertiesToRemoveKeys.getLeft(); Set removeKeys = setPropertiesToRemoveKeys.getRight(); if (setProperties.size() > 0) { From 36583eac87fb3a35d890f27461bf9b1836e60072 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 16 Dec 2024 16:31:14 +0800 Subject: [PATCH 16/19] support ALTER DATABASE for Flink SQL and Spark SQL --- .../paimon/catalog/AbstractCatalog.java | 3 ++ paimon-flink/paimon-flink-common/pom.xml | 9 ++++ .../org/apache/paimon/flink/FlinkCatalog.java | 39 +++++++++++++-- .../apache/paimon/flink/FlinkCatalogTest.java | 43 ++++++++++++++++ .../org/apache/paimon/spark/SparkCatalog.java | 50 +++++++++++++++---- 5 files changed, 130 insertions(+), 14 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 2ae256b91517..c76d0286b595 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 @@ -234,6 +234,9 @@ public void alterDatabase(String name, List changes, boolean ign throws DatabaseNotExistException { checkNotSystemDatabase(name); try { + if (changes == null || changes.isEmpty()) { + return; + } alterDatabaseImpl(name, changes); } catch (DatabaseNotExistException e) { if (ignoreIfNotExists) { diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index 91222983bf6b..7388f8944109 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -169,8 +169,17 @@ under the License. + + + org.mockito + mockito-inline + ${mockito.version} + jar + test + + diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 3a7f9790ccca..7a2e996d44a1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -20,8 +20,11 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TableType; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.paimon.flink.utils.FlinkDescriptorProperties; @@ -1223,13 +1226,20 @@ public static Identifier toIdentifier(ObjectPath path) { return new Identifier(path.getDatabaseName(), path.getObjectName()); } - // --------------------- unsupported methods ---------------------------- - @Override public final void alterDatabase( String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); + throws CatalogException, DatabaseNotExistException { + try { + Database oldDatabase = catalog.getDatabase(name); + List changes = + getPropertyChanges(oldDatabase.options(), newDatabase.getProperties()); + catalog.alterDatabase(name, changes, ignoreIfNotExists); + } catch (Catalog.DatabaseNotExistException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), e.database()); + } + } } @Override @@ -1278,6 +1288,27 @@ public final List listPartitions(ObjectPath tablePath) return getPartitionSpecs(tablePath, null); } + @VisibleForTesting + static List getPropertyChanges( + Map oldOptions, Map newOptions) { + List changes = new ArrayList<>(); + newOptions.forEach( + (k, v) -> { + if (!oldOptions.containsKey(k) || !oldOptions.get(k).equals(v)) { + changes.add(PropertyChange.setProperty(k, v)); + } + }); + oldOptions + .keySet() + .forEach( + (k) -> { + if (!newOptions.containsKey(k)) { + changes.add(PropertyChange.removeProperty(k)); + } + }); + return changes; + } + private Table getPaimonTable(ObjectPath tablePath) throws TableNotExistException { try { Identifier identifier = toIdentifier(tablePath); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index e4286eb18172..c0f81828e1d3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.TableType; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.flink.log.LogSinkProvider; import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.log.LogStoreRegister; @@ -99,6 +100,11 @@ import static org.assertj.core.api.Assertions.assertThatCollection; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.doNothing; +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 {@link FlinkCatalog}. */ public class FlinkCatalogTest { @@ -600,6 +606,43 @@ public void testDropDb_DatabaseNotExistException() { .hasMessage("Database db1 does not exist in Catalog test-catalog."); } + @Test + public void testAlterDb() throws DatabaseAlreadyExistException, DatabaseNotExistException { + CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); + catalog.createDatabase(path1.getDatabaseName(), database, false); + Map properties = Collections.singletonMap("haa", "ccc"); + CatalogDatabaseImpl newDatabase = new CatalogDatabaseImpl(properties, "haha"); + Catalog mockCatalog = spy(catalog); + doNothing().when(mockCatalog).alterDatabase(path1.getDatabaseName(), newDatabase, false); + when(mockCatalog.getDatabase(path1.getDatabaseName())).thenReturn(database); + mockCatalog.alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).getDatabase(path1.getDatabaseName()); + } + + @Test + public void testAlterDb_DatabaseNotExistException() { + CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); + assertThatThrownBy(() -> catalog.alterDatabase(path1.getDatabaseName(), database, false)) + .isInstanceOf(DatabaseNotExistException.class) + .hasMessage("Database db1 does not exist in Catalog test-catalog."); + } + + @Test + public void testGetProperties() throws Exception { + Map oldProperties = Collections.emptyMap(); + Map newProperties = Collections.singletonMap("haa", "ccc"); + List propertyChanges = + FlinkCatalog.getPropertyChanges(oldProperties, newProperties); + assertThat(propertyChanges.size()).isEqualTo(1); + oldProperties = newProperties; + propertyChanges = FlinkCatalog.getPropertyChanges(oldProperties, newProperties); + assertThat(propertyChanges.size()).isEqualTo(0); + oldProperties = Collections.singletonMap("aa", "ccc"); + propertyChanges = FlinkCatalog.getPropertyChanges(oldProperties, newProperties); + assertThat(propertyChanges.size()).isEqualTo(2); + } + @Test public void testCreateTableWithColumnOptions() throws Exception { ResolvedExpression expression = diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index d6318c723fe0..fad3a383b1c8 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -130,7 +131,8 @@ public void createNamespace(String[] namespace, Map metadata) throws NamespaceAlreadyExistsException { checkNamespace(namespace); try { - catalog.createDatabase(namespace[0], false, metadata); + String databaseName = getDatabaseNameFromNamespace(namespace); + catalog.createDatabase(databaseName, false, metadata); } catch (Catalog.DatabaseAlreadyExistException e) { throw new NamespaceAlreadyExistsException(namespace); } @@ -153,7 +155,8 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep } checkNamespace(namespace); try { - catalog.getDatabase(namespace[0]); + String databaseName = getDatabaseNameFromNamespace(namespace); + catalog.getDatabase(databaseName); return new String[0][]; } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); @@ -164,9 +167,9 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep public Map loadNamespaceMetadata(String[] namespace) throws NoSuchNamespaceException { checkNamespace(namespace); - String dataBaseName = namespace[0]; try { - return catalog.getDatabase(dataBaseName).options(); + String databaseName = getDatabaseNameFromNamespace(namespace); + return catalog.getDatabase(databaseName).options(); } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); } @@ -203,7 +206,8 @@ public boolean dropNamespace(String[] namespace, boolean cascade) throws NoSuchNamespaceException { checkNamespace(namespace); try { - catalog.dropDatabase(namespace[0], false, cascade); + String databaseName = getDatabaseNameFromNamespace(namespace); + catalog.dropDatabase(databaseName, false, cascade); return true; } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); @@ -217,7 +221,8 @@ public boolean dropNamespace(String[] namespace, boolean cascade) public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { checkNamespace(namespace); try { - return catalog.listTables(namespace[0]).stream() + String databaseName = getDatabaseNameFromNamespace(namespace); + return catalog.listTables(databaseName).stream() .map(table -> Identifier.of(namespace, table)) .toArray(Identifier[]::new); } catch (Catalog.DatabaseNotExistException e) { @@ -516,10 +521,35 @@ protected List convertPartitionTransforms(Transform[] transforms) { return partitionColNames; } - // --------------------- unsupported methods ---------------------------- - @Override - public void alterNamespace(String[] namespace, NamespaceChange... changes) { - throw new UnsupportedOperationException("Alter namespace in Spark is not supported yet."); + public void alterNamespace(String[] namespace, NamespaceChange... changes) + throws NoSuchNamespaceException { + checkNamespace(namespace); + try { + String databaseName = getDatabaseNameFromNamespace(namespace); + List propertyChanges = + Arrays.stream(changes).map(this::toPropertyChange).collect(Collectors.toList()); + catalog.alterDatabase(databaseName, propertyChanges, false); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + private PropertyChange toPropertyChange(NamespaceChange change) { + if (change instanceof NamespaceChange.SetProperty) { + NamespaceChange.SetProperty set = (NamespaceChange.SetProperty) change; + return PropertyChange.setProperty(set.property(), set.value()); + } else if (change instanceof NamespaceChange.RemoveProperty) { + NamespaceChange.RemoveProperty remove = (NamespaceChange.RemoveProperty) change; + return PropertyChange.removeProperty(remove.property()); + + } else { + throw new UnsupportedOperationException( + "Change is not supported: " + change.getClass()); + } + } + + private String getDatabaseNameFromNamespace(String[] namespace) { + return namespace[0]; } } From f7a2e150a1881593c5e668fc6b98353991576ec6 Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 16 Dec 2024 16:51:28 +0800 Subject: [PATCH 17/19] fix ut fail --- .../apache/paimon/catalog/FileSystemCatalogTest.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index aa2ddaf4f7a7..aef2f8f485ce 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -24,11 +24,11 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.util.ArrayList; - import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -75,7 +75,12 @@ public void testCreateTableAllowUpperCase() throws Exception { public void testAlterDatabase() throws Exception { String databaseName = "test_alter_db"; catalog.createDatabase(databaseName, false); - assertThatThrownBy(() -> catalog.alterDatabase(databaseName, new ArrayList<>(), false)) + assertThatThrownBy( + () -> + catalog.alterDatabase( + databaseName, + Lists.newArrayList(PropertyChange.removeProperty("a")), + false)) .isInstanceOf(UnsupportedOperationException.class); } } From 6b77e1338ef4f11bdb584dbecb9814f57f3924ef Mon Sep 17 00:00:00 2001 From: yantian Date: Mon, 16 Dec 2024 18:02:46 +0800 Subject: [PATCH 18/19] add alter database doc for flink and spark and support comment when create database in flink --- docs/content/flink/sql-alter.md | 8 ++++++++ docs/content/spark/sql-alter.md | 10 ++++++++++ .../java/org/apache/paimon/flink/FlinkCatalog.java | 12 ++++++------ .../org/apache/paimon/flink/FlinkCatalogTest.java | 9 +++++---- 4 files changed, 29 insertions(+), 10 deletions(-) diff --git a/docs/content/flink/sql-alter.md b/docs/content/flink/sql-alter.md index 877995cc631b..645cb06ef365 100644 --- a/docs/content/flink/sql-alter.md +++ b/docs/content/flink/sql-alter.md @@ -227,3 +227,11 @@ The following SQL modifies the watermark strategy to `ts - INTERVAL '2' HOUR`. ```sql ALTER TABLE my_table MODIFY WATERMARK FOR ts AS ts - INTERVAL '2' HOUR ``` + +# ALTER DATABASE + +The following SQL sets one or more properties in the specified database. If a particular property is already set in the database, override the old value with the new one. + +```sql +ALTER DATABASE [catalog_name.]db_name SET (key1=val1, key2=val2, ...) +``` diff --git a/docs/content/spark/sql-alter.md b/docs/content/spark/sql-alter.md index 3ad72048029b..839f0d2f8ce4 100644 --- a/docs/content/spark/sql-alter.md +++ b/docs/content/spark/sql-alter.md @@ -240,3 +240,13 @@ The following SQL changes the type of a nested column `f2` to `BIGINT` in a stru -- column v previously has type MAP> ALTER TABLE my_table ALTER COLUMN v.value.f2 TYPE BIGINT; ``` + + +# ALTER DATABASE + +The following SQL sets one or more properties in the specified database. If a particular property is already set in the database, override the old value with the new one. + +```sql +ALTER { DATABASE | SCHEMA | NAMESPACE } my_database + SET { DBPROPERTIES | PROPERTIES } ( property_name = property_value [ , ... ] ) +``` \ No newline at end of file diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 7a2e996d44a1..2d57b2ef59ca 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -239,19 +239,19 @@ public CatalogDatabase getDatabase(String databaseName) @Override public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) throws DatabaseAlreadyExistException, CatalogException { + Map properties; if (database != null) { + properties = new HashMap<>(database.getProperties()); if (database.getDescription().isPresent() && !database.getDescription().get().equals("")) { - throw new UnsupportedOperationException( - "Create database with description is unsupported."); + properties.put(Catalog.COMMENT_PROP, database.getDescription().get()); } + } else { + properties = Collections.emptyMap(); } try { - catalog.createDatabase( - name, - ignoreIfExists, - database == null ? Collections.emptyMap() : database.getProperties()); + catalog.createDatabase(name, ignoreIfExists, properties); } catch (Catalog.DatabaseAlreadyExistException e) { throw new DatabaseAlreadyExistException(getName(), e.database()); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index c0f81828e1d3..fb36f7ad77b1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -99,6 +99,7 @@ import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatCollection; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; @@ -581,11 +582,9 @@ public void testCreateDb_DatabaseWithProperties() throws Exception { } @Test - public void testCreateDb_DatabaseWithCommentException() { + public void testCreateDb_DatabaseWithCommentSuccessful() throws DatabaseAlreadyExistException { CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), "haha"); - assertThatThrownBy(() -> catalog.createDatabase(path1.getDatabaseName(), database, false)) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Create database with description is unsupported."); + assertDoesNotThrow(() -> catalog.createDatabase(path1.getDatabaseName(), database, false)); } @ParameterizedTest @@ -612,6 +611,8 @@ public void testAlterDb() throws DatabaseAlreadyExistException, DatabaseNotExist catalog.createDatabase(path1.getDatabaseName(), database, false); Map properties = Collections.singletonMap("haa", "ccc"); CatalogDatabaseImpl newDatabase = new CatalogDatabaseImpl(properties, "haha"); + // as file system catalog don't support alter database, so we have to use mock to overview + // this method to test Catalog mockCatalog = spy(catalog); doNothing().when(mockCatalog).alterDatabase(path1.getDatabaseName(), newDatabase, false); when(mockCatalog.getDatabase(path1.getDatabaseName())).thenReturn(database); From 91f3408d862e4f4dce4be4682f41f0f90e027f31 Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 17 Dec 2024 14:35:53 +0800 Subject: [PATCH 19/19] fix flink alter database change comment and add test, doc for location --- docs/content/flink/sql-alter.md | 8 ++++ docs/content/spark/sql-alter.md | 8 ++++ .../org/apache/paimon/flink/FlinkCatalog.java | 18 ++++++-- .../apache/paimon/flink/FlinkCatalogTest.java | 31 ++++++++++++++ .../sql/DDLWithHiveCatalogTestBase.scala | 42 ++++++++++++++++++- 5 files changed, 103 insertions(+), 4 deletions(-) diff --git a/docs/content/flink/sql-alter.md b/docs/content/flink/sql-alter.md index 645cb06ef365..6c3186b4af75 100644 --- a/docs/content/flink/sql-alter.md +++ b/docs/content/flink/sql-alter.md @@ -235,3 +235,11 @@ The following SQL sets one or more properties in the specified database. If a pa ```sql ALTER DATABASE [catalog_name.]db_name SET (key1=val1, key2=val2, ...) ``` + +## Altering Database Location + +The following SQL changes location of database `my_database` to `file:/temp/my_database`. + +```sql +ALTER DATABASE my_database SET ('location' = 'file:/temp/my_database') +``` diff --git a/docs/content/spark/sql-alter.md b/docs/content/spark/sql-alter.md index 839f0d2f8ce4..359b1187292d 100644 --- a/docs/content/spark/sql-alter.md +++ b/docs/content/spark/sql-alter.md @@ -249,4 +249,12 @@ The following SQL sets one or more properties in the specified database. If a pa ```sql ALTER { DATABASE | SCHEMA | NAMESPACE } my_database SET { DBPROPERTIES | PROPERTIES } ( property_name = property_value [ , ... ] ) +``` + +## Altering Database Location + +The following SQL sets the location of the specified database to `file:/temp/my_database.db`. + +```sql +ALTER DATABASE my_database SET LOCATION 'file:/temp/my_database.db' ``` \ No newline at end of file diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 510d7c59a8b7..ec3c4a47a69d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -136,6 +136,7 @@ import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_MODE; import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_STATUS; import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.catalog.Catalog.COMMENT_PROP; import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; @@ -244,7 +245,7 @@ public void createDatabase(String name, CatalogDatabase database, boolean ignore properties = new HashMap<>(database.getProperties()); if (database.getDescription().isPresent() && !database.getDescription().get().equals("")) { - properties.put(Catalog.COMMENT_PROP, database.getDescription().get()); + properties.put(COMMENT_PROP, database.getDescription().get()); } } else { properties = Collections.emptyMap(); @@ -623,7 +624,7 @@ private List toSchemaChange( SchemaManager.checkAlterTablePath(key); - if (Catalog.COMMENT_PROP.equals(key)) { + if (COMMENT_PROP.equals(key)) { schemaChanges.add(SchemaChange.updateComment(value)); } else { schemaChanges.add(SchemaChange.setOption(key, value)); @@ -632,7 +633,7 @@ private List toSchemaChange( } else if (change instanceof ResetOption) { ResetOption resetOption = (ResetOption) change; String key = resetOption.getKey(); - if (Catalog.COMMENT_PROP.equals(key)) { + if (COMMENT_PROP.equals(key)) { schemaChanges.add(SchemaChange.updateComment(null)); } else { schemaChanges.add(SchemaChange.removeOption(resetOption.getKey())); @@ -1220,6 +1221,8 @@ public final void alterDatabase( Database oldDatabase = catalog.getDatabase(name); List changes = getPropertyChanges(oldDatabase.options(), newDatabase.getProperties()); + getPropertyChangeFromComment(oldDatabase.comment(), newDatabase.getDescription()) + .ifPresent(changes::add); catalog.alterDatabase(name, changes, ignoreIfNotExists); } catch (Catalog.DatabaseNotExistException e) { if (!ignoreIfNotExists) { @@ -1295,6 +1298,15 @@ static List getPropertyChanges( return changes; } + @VisibleForTesting + static Optional getPropertyChangeFromComment( + Optional oldComment, Optional newComment) { + if (newComment.isPresent() && !oldComment.equals(newComment)) { + return Optional.of(PropertyChange.setProperty(COMMENT_PROP, newComment.get())); + } + return Optional.empty(); + } + private Table getPaimonTable(ObjectPath tablePath) throws TableNotExistException { try { Identifier identifier = toIdentifier(tablePath); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index 0b68d9ecf6ff..4b8cf7912192 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -84,6 +84,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.stream.Stream; @@ -617,6 +618,20 @@ public void testAlterDb() throws DatabaseAlreadyExistException, DatabaseNotExist verify(mockCatalog, times(1)).getDatabase(path1.getDatabaseName()); } + @Test + public void testAlterDbComment() + throws DatabaseAlreadyExistException, DatabaseNotExistException { + CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); + catalog.createDatabase(path1.getDatabaseName(), database, false); + Catalog mockCatalog = spy(catalog); + when(mockCatalog.getDatabase(path1.getDatabaseName())).thenReturn(database); + CatalogDatabaseImpl newDatabase = new CatalogDatabaseImpl(Collections.emptyMap(), "aa"); + doNothing().when(mockCatalog).alterDatabase(path1.getDatabaseName(), newDatabase, false); + mockCatalog.alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).alterDatabase(path1.getDatabaseName(), newDatabase, false); + verify(mockCatalog, times(1)).getDatabase(path1.getDatabaseName()); + } + @Test public void testAlterDb_DatabaseNotExistException() { CatalogDatabaseImpl database = new CatalogDatabaseImpl(Collections.emptyMap(), null); @@ -640,6 +655,22 @@ public void testGetProperties() throws Exception { assertThat(propertyChanges.size()).isEqualTo(2); } + @Test + public void testGetPropertyChangeFromComment() { + Optional commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.empty(), Optional.empty()); + assertThat(commentChange.isPresent()).isFalse(); + commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.of("aa"), Optional.of("bb")); + assertThat(commentChange.isPresent()).isTrue(); + commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.of("aa"), Optional.empty()); + assertThat(commentChange.isPresent()).isFalse(); + commentChange = + FlinkCatalog.getPropertyChangeFromComment(Optional.empty(), Optional.of("bb")); + assertThat(commentChange.isPresent()).isTrue(); + } + @Test public void testCreateTableWithColumnOptions() throws Exception { ResolvedExpression expression = diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 4ba079ea0bb2..526e24250751 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -22,7 +22,7 @@ import org.apache.paimon.hive.HiveMetastoreClient import org.apache.paimon.spark.PaimonHiveTestBase import org.apache.paimon.table.FileStoreTable -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.junit.jupiter.api.Assertions abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { @@ -194,6 +194,46 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: alter database's properties") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + val databaseName = "paimon_db" + withDatabase(databaseName) { + spark.sql(s"CREATE DATABASE $databaseName WITH DBPROPERTIES ('k1' = 'v1', 'k2' = 'v2')") + var props = getDatabaseProps(databaseName) + Assertions.assertEquals(props("k1"), "v1") + Assertions.assertEquals(props("k2"), "v2") + spark.sql(s"ALTER DATABASE $databaseName SET DBPROPERTIES ('k1' = 'v11', 'k2' = 'v22')") + props = getDatabaseProps(databaseName) + Assertions.assertEquals(props("k1"), "v11") + Assertions.assertEquals(props("k2"), "v22") + } + } + } + + test("Paimon DDL with hive catalog: alter database location") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + val databaseName = "paimon_db" + withDatabase(databaseName) { + spark.sql(s"CREATE DATABASE $databaseName WITH DBPROPERTIES ('k1' = 'v1', 'k2' = 'v2')") + withTempDir { + dBLocation => + try { + spark.sql( + s"ALTER DATABASE $databaseName SET LOCATION '${dBLocation.getCanonicalPath}'") + } catch { + case e: AnalysisException => + Assertions.assertTrue( + e.getMessage.contains("does not support altering database location")) + } + } + } + } + } + test("Paimon DDL with hive catalog: set default database") { var reusedSpark = spark