LB-Yu commented on code in PR #1509:
URL: https://github.com/apache/fluss/pull/1509#discussion_r2265560292
##########
fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/MetadataManager.java:
##########
@@ -300,6 +303,65 @@ public long createTable(
"Fail to create table " + tablePath);
}
+ public void alterTable(
+ TablePath tablePath, TableDescriptor tableDescriptor, boolean
ignoreIfNotExists) {
+ // validate table properties before creating table
+ validateTableDescriptor(tableDescriptor, maxBucketNum);
Review Comment:
Currently we only support alter table properties. Should I fill the `Schema`
here?
##########
fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/MetadataManager.java:
##########
@@ -300,6 +303,65 @@ public long createTable(
"Fail to create table " + tablePath);
}
+ public void alterTable(
+ TablePath tablePath, TableDescriptor tableDescriptor, boolean
ignoreIfNotExists) {
+ // validate table properties before creating table
Review Comment:
// validate table properties before altering table
##########
fluss-rpc/src/main/proto/FlussApi.proto:
##########
@@ -108,6 +108,17 @@ message CreateTableRequest {
message CreateTableResponse {
}
+// alter table request and response
+message AlterTableRequest {
+ required PbTablePath table_path = 1;
+ required bytes table_json = 2;
+ required bool ignore_if_not_exists = 3;
+}
+
+message AlterTableResponse {
+
Review Comment:
Redundant blank line.
##########
fluss-client/src/main/java/com/alibaba/fluss/client/admin/Admin.java:
##########
@@ -234,6 +234,23 @@ CompletableFuture<Void> createTable(
*/
CompletableFuture<List<String>> listTables(String databaseName);
+ /**
+ * Alter a table.
+ *
+ * <p>The following exceptions can be anticipated when calling {@code
get()} on returned future.
+ *
+ * <ul>
+ * <li>{@link DatabaseNotExistException} if the database does not exist.
+ * </ul>
+ *
+ * @param tablePath The table path of the table.
+ * @param tableDescriptor The table descriptor.
+ * @param ignoreIfNotExists if it is true, do nothing if table does not
exist. If false, throw a
+ * TableNotExistException.
+ */
+ CompletableFuture<Void> alterTable(
Review Comment:
We should add some IT case for this.
##########
fluss-flink/fluss-flink-common/src/main/java/com/alibaba/fluss/flink/catalog/FlinkCatalog.java:
##########
@@ -389,9 +389,25 @@ public void createTable(ObjectPath objectPath,
CatalogBaseTable table, boolean i
}
@Override
- public void alterTable(ObjectPath objectPath, CatalogBaseTable
catalogBaseTable, boolean b)
+ public void alterTable(
+ ObjectPath objectPath, CatalogBaseTable newTable, boolean
ignoreIfNotExist)
throws TableNotExistException, CatalogException {
- throw new UnsupportedOperationException();
+ TablePath tablePath = toTablePath(objectPath);
+ TableDescriptor tableDescriptor =
+ FlinkConversions.toFlussTable((ResolvedCatalogTable) newTable);
+ try {
+ admin.alterTable(tablePath, tableDescriptor,
ignoreIfNotExist).get();
+ } catch (Exception e) {
+ Throwable t = ExceptionUtils.stripExecutionException(e);
+ if (CatalogExceptionUtils.isTableAlreadyExist(t)) {
+ throw new TableNotExistException(getName(), objectPath);
Review Comment:
I'm confused here. Why throw `TableNotExistException` when table already
exists?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]