This is an automated email from the ASF dual-hosted git repository.

snuyanzin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9bc742d38ee20cbc0974e2642ebf03947138c807
Author: Sergey Nuyanzin <[email protected]>
AuthorDate: Mon Dec 1 18:44:10 2025 +0100

    [hotfix][table] Rename `SqlAlterDistribution` to 
`SqlAlterTableDistribution` and `SqlAlterMaterializedTableAddDistribution` to 
`SqlAlterMaterializedTableDistribution`
---
 .../src/main/codegen/data/Parser.tdd               |  8 +--
 .../src/main/codegen/includes/parserImpls.ftl      |  4 +-
 .../SqlAlterMaterializedTableAddDistribution.java  | 62 ----------------------
 ... => SqlAlterMaterializedTableDistribution.java} | 50 ++++++++++++-----
 ...ibution.java => SqlAlterTableDistribution.java} | 17 +++---
 .../MaterializedTableStatementParserTest.java      | 26 +++++++++
 ...rMaterializedTableAddDistributionConverter.java |  2 +-
 ...terializedTableModifyDistributionConverter.java |  2 +-
 .../SqlAlterTableAddDistributionConverter.java     | 12 ++---
 .../SqlAlterTableModifyDistributionConverter.java  | 12 ++---
 10 files changed, 94 insertions(+), 101 deletions(-)

diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd 
b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
index 6e93241af40..42acdb21f70 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
+++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd
@@ -31,8 +31,8 @@
     "org.apache.flink.sql.parser.ddl.resource.SqlResource"
     "org.apache.flink.sql.parser.ddl.resource.SqlResourceType"
     "org.apache.flink.sql.parser.ddl.SqlAddJar"
-    "org.apache.flink.sql.parser.ddl.SqlAlterDistribution.SqlAddDistribution"
-    
"org.apache.flink.sql.parser.ddl.SqlAlterDistribution.SqlModifyDistribution"
+    
"org.apache.flink.sql.parser.ddl.SqlAlterTableDistribution.SqlAlterTableAddDistribution"
+    
"org.apache.flink.sql.parser.ddl.SqlAlterTableDistribution.SqlAlterTableModifyDistribution"
     "org.apache.flink.sql.parser.ddl.SqlAddPartitions"
     
"org.apache.flink.sql.parser.ddl.SqlAddPartitions.AlterTableAddPartitionContext"
     "org.apache.flink.sql.parser.ddl.SqlAlterCatalog"
@@ -42,8 +42,8 @@
     "org.apache.flink.sql.parser.ddl.SqlAlterDatabase"
     "org.apache.flink.sql.parser.ddl.SqlAlterFunction"
     "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTable"
-    "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableAddDistribution"
-    
"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableModifyDistribution"
+    
"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableAddDistribution"
+    
"org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableModifyDistribution"
     "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableDropDistribution"
     "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableAsQuery"
     "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableFreshness"
diff --git 
a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl 
b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
index bf2f07ce060..d1f6d2eba50 100644
--- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
+++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl
@@ -924,7 +924,7 @@ SqlAlterTable SqlAlterTable() :
         |
         (
             <DISTRIBUTION>
-            {return new SqlAddDistribution(getPos(), tableIdentifier, 
SqlDistribution(getPos()));}
+            {return new SqlAlterTableAddDistribution(getPos(), 
tableIdentifier, SqlDistribution(getPos()));}
         |
             AlterTableAddOrModify(ctx)
         |
@@ -949,7 +949,7 @@ SqlAlterTable SqlAlterTable() :
         <MODIFY>
         (
             <DISTRIBUTION>
-            {return new SqlModifyDistribution(getPos(), tableIdentifier, 
SqlDistribution(getPos()));}
+            {return new SqlAlterTableModifyDistribution(getPos(), 
tableIdentifier, SqlDistribution(getPos()));}
         |
             AlterTableAddOrModify(ctx)
         |
diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableAddDistribution.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableAddDistribution.java
deleted file mode 100644
index 2130247c8a6..00000000000
--- 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableAddDistribution.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.sql.parser.ddl;
-
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.ImmutableNullableList;
-
-import javax.annotation.Nullable;
-
-import java.util.List;
-import java.util.Optional;
-
-/**
- * SqlNode to describe the ALTER MATERIALIZED TABLE 
[catalogName.][dataBasesName.]tableName ADD
- * DISTRIBUTION statement.
- */
-public class SqlAlterMaterializedTableAddDistribution extends 
SqlAlterMaterializedTable {
-    protected final @Nullable SqlDistribution distribution;
-
-    public SqlAlterMaterializedTableAddDistribution(
-            SqlParserPos pos, SqlIdentifier tableName, @Nullable 
SqlDistribution distribution) {
-        super(pos, tableName);
-        this.distribution = distribution;
-    }
-
-    @Override
-    public List<SqlNode> getOperandList() {
-        return ImmutableNullableList.of(name, distribution);
-    }
-
-    @Override
-    public void unparseAlterOperation(SqlWriter writer, int leftPrec, int 
rightPrec) {
-        super.unparseAlterOperation(writer, leftPrec, rightPrec);
-        writer.keyword("ADD");
-        if (distribution != null) {
-            distribution.unparseAlter(writer, leftPrec, rightPrec);
-        }
-    }
-
-    public Optional<SqlDistribution> getDistribution() {
-        return Optional.ofNullable(distribution);
-    }
-}
diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableModifyDistribution.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableDistribution.java
similarity index 55%
rename from 
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableModifyDistribution.java
rename to 
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableDistribution.java
index b11acab0bf8..5bacc292c33 100644
--- 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableModifyDistribution.java
+++ 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableDistribution.java
@@ -24,24 +24,24 @@ import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.ImmutableNullableList;
 
-import javax.annotation.Nullable;
-
 import java.util.List;
 import java.util.Optional;
 
 /**
- * SqlNode to describe the ALTER MATERIALIZED TABLE 
[catalogName.][dataBasesName.]tableName MODIFY
- * DISTRIBUTION statement.
+ * SqlNode to describe the ALTER MATERIALIZED TABLE 
[catalogName.][dataBasesName.]tableName
+ * (ADD|MODIFY) DISTRIBUTION statement.
  */
-public class SqlAlterMaterializedTableModifyDistribution extends 
SqlAlterMaterializedTable {
-    protected final @Nullable SqlDistribution distribution;
+public abstract class SqlAlterMaterializedTableDistribution extends 
SqlAlterMaterializedTable {
+    protected final SqlDistribution distribution;
 
-    public SqlAlterMaterializedTableModifyDistribution(
-            SqlParserPos pos, SqlIdentifier tableName, @Nullable 
SqlDistribution distribution) {
+    public SqlAlterMaterializedTableDistribution(
+            SqlParserPos pos, SqlIdentifier tableName, SqlDistribution 
distribution) {
         super(pos, tableName);
         this.distribution = distribution;
     }
 
+    protected abstract String getAlterOperation();
+
     @Override
     public List<SqlNode> getOperandList() {
         return ImmutableNullableList.of(name, distribution);
@@ -50,13 +50,39 @@ public class SqlAlterMaterializedTableModifyDistribution 
extends SqlAlterMateria
     @Override
     public void unparseAlterOperation(SqlWriter writer, int leftPrec, int 
rightPrec) {
         super.unparseAlterOperation(writer, leftPrec, rightPrec);
-        writer.keyword("MODIFY");
-        if (distribution != null) {
-            distribution.unparseAlter(writer, leftPrec, rightPrec);
-        }
+        writer.keyword(getAlterOperation());
+        distribution.unparseAlter(writer, leftPrec, rightPrec);
     }
 
     public Optional<SqlDistribution> getDistribution() {
         return Optional.ofNullable(distribution);
     }
+
+    public static class SqlAlterMaterializedTableAddDistribution
+            extends SqlAlterMaterializedTableDistribution {
+
+        public SqlAlterMaterializedTableAddDistribution(
+                SqlParserPos pos, SqlIdentifier tableName, SqlDistribution 
distribution) {
+            super(pos, tableName, distribution);
+        }
+
+        @Override
+        protected String getAlterOperation() {
+            return "ADD";
+        }
+    }
+
+    public static class SqlAlterMaterializedTableModifyDistribution
+            extends SqlAlterMaterializedTableDistribution {
+
+        public SqlAlterMaterializedTableModifyDistribution(
+                SqlParserPos pos, SqlIdentifier tableName, SqlDistribution 
distribution) {
+            super(pos, tableName, distribution);
+        }
+
+        @Override
+        protected String getAlterOperation() {
+            return "MODIFY";
+        }
+    }
 }
diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDistribution.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterTableDistribution.java
similarity index 83%
rename from 
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDistribution.java
rename to 
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterTableDistribution.java
index e8696f8b1c5..91b0b2ade20 100644
--- 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDistribution.java
+++ 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterTableDistribution.java
@@ -28,12 +28,15 @@ import javax.annotation.Nonnull;
 import java.util.ArrayList;
 import java.util.List;
 
-/** ALTER Distribution DDL sql call for tables and materialized tables. */
-public abstract class SqlAlterDistribution extends SqlAlterTable {
+/**
+ * SqlNode to describe the ALTER TABLE [catalogName.][dataBasesName.]tableName 
(ADD|MODIFY)
+ * DISTRIBUTION statement.
+ */
+public abstract class SqlAlterTableDistribution extends SqlAlterTable {
 
     private final SqlDistribution distribution;
 
-    public SqlAlterDistribution(
+    public SqlAlterTableDistribution(
             SqlParserPos pos, SqlIdentifier tableName, SqlDistribution 
distribution) {
         super(pos, tableName, false);
         this.distribution = distribution;
@@ -61,9 +64,9 @@ public abstract class SqlAlterDistribution extends 
SqlAlterTable {
         distribution.unparseAlter(writer, leftPrec, rightPrec);
     }
 
-    public static class SqlAddDistribution extends SqlAlterDistribution {
+    public static class SqlAlterTableAddDistribution extends 
SqlAlterTableDistribution {
 
-        public SqlAddDistribution(
+        public SqlAlterTableAddDistribution(
                 SqlParserPos pos, SqlIdentifier tableName, SqlDistribution 
distribution) {
             super(pos, tableName, distribution);
         }
@@ -74,9 +77,9 @@ public abstract class SqlAlterDistribution extends 
SqlAlterTable {
         }
     }
 
-    public static class SqlModifyDistribution extends SqlAlterDistribution {
+    public static class SqlAlterTableModifyDistribution extends 
SqlAlterTableDistribution {
 
-        public SqlModifyDistribution(
+        public SqlAlterTableModifyDistribution(
                 SqlParserPos pos, SqlIdentifier tableName, SqlDistribution 
distribution) {
             super(pos, tableName, distribution);
         }
diff --git 
a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java
 
b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java
index 9758aa9122c..f62935ee80c 100644
--- 
a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java
+++ 
b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java
@@ -350,6 +350,32 @@ class MaterializedTableStatementParserTest {
                                 + "    ");
     }
 
+    @Test
+    void testAlterMaterializedTableAddDistribution() {
+        sql("alter materialized table mt1 add distribution by hash(a) into 6 
buckets")
+                .ok("ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY 
HASH(`A`) INTO 6 BUCKETS");
+
+        sql("alter materialized table mt1 add distribution by hash(a, h) into 
6 buckets")
+                .ok(
+                        "ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY 
HASH(`A`, `H`) INTO 6 BUCKETS");
+
+        sql("alter materialized table mt1 add distribution by range(a, h) into 
6 buckets")
+                .ok(
+                        "ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY 
RANGE(`A`, `H`) INTO 6 BUCKETS");
+
+        sql("alter materialized table mt1 add distribution by ^RANDOM^(a, h) 
into 6 buckets")
+                .fails("(?s).*Encountered \"RANDOM\" at line 1, column.*");
+
+        sql("alter materialized table mt1 add distribution by (a, h) into 6 
buckets")
+                .ok("ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY (`A`, 
`H`) INTO 6 BUCKETS");
+
+        sql("alter materialized table mt1 add distribution by range(a, h)")
+                .ok("ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY 
RANGE(`A`, `H`)");
+
+        sql("alter materialized table mt1 add distribution by (a, h)")
+                .ok("ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY (`A`, 
`H`)");
+    }
+
     @Test
     void testDropMaterializedTable() {
         final String sql = "DROP MATERIALIZED TABLE tbl1";
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java
index cd4877626b3..78c86edb089 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.planner.operations.converters;
 
-import 
org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableAddDistribution;
+import 
org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableAddDistribution;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
 import org.apache.flink.table.catalog.ObjectIdentifier;
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java
index 861b21416a4..a97d6db6d76 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.planner.operations.converters;
 
-import 
org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableModifyDistribution;
+import 
org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableModifyDistribution;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.CatalogMaterializedTable;
 import org.apache.flink.table.catalog.ObjectIdentifier;
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableAddDistributionConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableAddDistributionConverter.java
index 3a40b8cfd31..bc9607543be 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableAddDistributionConverter.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableAddDistributionConverter.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.table.planner.operations.converters.table;
 
-import org.apache.flink.sql.parser.ddl.SqlAlterDistribution;
-import org.apache.flink.sql.parser.ddl.SqlAlterDistribution.SqlAddDistribution;
 import org.apache.flink.sql.parser.ddl.SqlAlterTable;
+import org.apache.flink.sql.parser.ddl.SqlAlterTableDistribution;
+import 
org.apache.flink.sql.parser.ddl.SqlAlterTableDistribution.SqlAlterTableAddDistribution;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.ResolvedCatalogTable;
 import org.apache.flink.table.catalog.TableChange;
@@ -31,13 +31,13 @@ import 
org.apache.flink.table.planner.utils.OperationConverterUtils;
 import java.util.List;
 import java.util.Optional;
 
-/** A converter for {@link SqlAlterDistribution} for ADD call. */
+/** A converter for {@link SqlAlterTableDistribution} for ADD call. */
 public class SqlAlterTableAddDistributionConverter
-        extends AbstractAlterTableConverter<SqlAddDistribution> {
+        extends AbstractAlterTableConverter<SqlAlterTableAddDistribution> {
 
     @Override
     protected Operation convertToOperation(
-            SqlAddDistribution sqlAddDistribution,
+            SqlAlterTableAddDistribution sqlAddDistribution,
             ResolvedCatalogTable oldTable,
             ConvertContext context) {
         return buildAlterTableChangeOperation(
@@ -62,6 +62,6 @@ public class SqlAlterTableAddDistributionConverter
                             EX_MSG_PREFIX, oldDistribution.get()));
         }
         return OperationConverterUtils.getDistributionFromSqlDistribution(
-                ((SqlAlterDistribution) alterTable).getDistribution());
+                ((SqlAlterTableDistribution) alterTable).getDistribution());
     }
 }
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableModifyDistributionConverter.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableModifyDistributionConverter.java
index 0afa279df2e..479b332f8be 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableModifyDistributionConverter.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableModifyDistributionConverter.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.table.planner.operations.converters.table;
 
-import org.apache.flink.sql.parser.ddl.SqlAlterDistribution;
-import 
org.apache.flink.sql.parser.ddl.SqlAlterDistribution.SqlModifyDistribution;
 import org.apache.flink.sql.parser.ddl.SqlAlterTable;
+import org.apache.flink.sql.parser.ddl.SqlAlterTableDistribution;
+import 
org.apache.flink.sql.parser.ddl.SqlAlterTableDistribution.SqlAlterTableModifyDistribution;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.ResolvedCatalogTable;
 import org.apache.flink.table.catalog.TableChange;
@@ -31,13 +31,13 @@ import 
org.apache.flink.table.planner.utils.OperationConverterUtils;
 import java.util.List;
 import java.util.Optional;
 
-/** A converter for {@link SqlAlterDistribution} for MODIFY call. */
+/** A converter for {@link SqlAlterTableDistribution} for MODIFY call. */
 public class SqlAlterTableModifyDistributionConverter
-        extends AbstractAlterTableConverter<SqlModifyDistribution> {
+        extends AbstractAlterTableConverter<SqlAlterTableModifyDistribution> {
 
     @Override
     protected Operation convertToOperation(
-            SqlModifyDistribution sqlModifyDistribution,
+            SqlAlterTableModifyDistribution sqlModifyDistribution,
             ResolvedCatalogTable oldTable,
             ConvertContext context) {
         return buildAlterTableChangeOperation(
@@ -61,6 +61,6 @@ public class SqlAlterTableModifyDistributionConverter
                             EX_MSG_PREFIX));
         }
         return OperationConverterUtils.getDistributionFromSqlDistribution(
-                ((SqlAlterDistribution) alterTable).getDistribution());
+                ((SqlAlterTableDistribution) alterTable).getDistribution());
     }
 }

Reply via email to