You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pp...@apache.org on 2023/06/07 09:50:18 UTC

[ignite-3] branch main updated: IGNITE-19407 Sql. Introduce DDL command for ALTER COLUMN TYPE (#2105)

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

ppa pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 703d6c4cac IGNITE-19407 Sql. Introduce DDL command for ALTER COLUMN TYPE (#2105)
703d6c4cac is described below

commit 703d6c4cac3aa005f6586663b939315c5b51ba30
Author: Pavel Pereslegin <xx...@gmail.com>
AuthorDate: Wed Jun 7 12:50:13 2023 +0300

    IGNITE-19407 Sql. Introduce DDL command for ALTER COLUMN TYPE (#2105)
---
 .../ignite/internal/catalog/CatalogManager.java    |   9 +
 .../internal/catalog/CatalogServiceImpl.java       | 124 ++++++
 .../catalog/commands/AlterColumnParams.java        | 139 ++++++
 .../internal/catalog/commands/CatalogUtils.java    |  27 ++
 .../internal/catalog/commands/ColumnParams.java    |  48 ++-
 .../internal/catalog/commands/DefaultValue.java    |  44 ++
 .../catalog/events/AlterColumnEventParameters.java |  54 +++
 .../internal/catalog/events/CatalogEvent.java      |   2 +-
 .../internal/catalog/storage/AlterColumnEntry.java |  59 +++
 .../internal/catalog/CatalogServiceSelfTest.java   | 469 ++++++++++++++++++++-
 modules/sql-engine/src/main/codegen/config.fmpp    |   4 +-
 .../src/main/codegen/includes/parserImpls.ftl      |  82 ++--
 .../sql/engine/exec/ddl/DdlCommandHandler.java     |   3 +
 .../engine/exec/ddl/DdlCommandHandlerWrapper.java  |   6 +
 .../exec/ddl/DdlToCatalogCommandConverter.java     |  33 ++
 .../sql/engine/prepare/ddl/AlterColumnCommand.java |  69 +++
 .../prepare/ddl/DdlSqlToCommandConverter.java      |  91 ++++
 .../sql/engine/sql/IgniteSqlAlterColumn.java       | 108 ++++-
 .../engine/sql/IgniteSqlAlterColumnDefault.java    |  75 ----
 .../engine/sql/IgniteSqlAlterColumnNotNull.java    |  70 ---
 .../sql/engine/sql/IgniteSqlAlterColumnType.java   |  96 -----
 .../engine/sql/SqlAlterColumnDdlParserTest.java    |  85 ++--
 22 files changed, 1363 insertions(+), 334 deletions(-)

diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManager.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManager.java
index 2c9e0f5ae5..a92360410e 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManager.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManager.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.catalog;
 
 import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.catalog.commands.AlterColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableAddColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableDropColumnParams;
 import org.apache.ignite.internal.catalog.commands.CreateHashIndexParams;
@@ -63,6 +64,14 @@ public interface CatalogManager extends IgniteComponent, CatalogService {
      */
     CompletableFuture<Void> dropColumn(AlterTableDropColumnParams params);
 
+    /**
+     * Changes a table column.
+     *
+     * @param params Parameters.
+     * @return Operation future.
+     */
+    CompletableFuture<Void> alterColumn(AlterColumnParams params);
+
     /**
      * Creates new sorted index.
      *
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogServiceImpl.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogServiceImpl.java
index 22d5cbe311..736545ae8a 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogServiceImpl.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogServiceImpl.java
@@ -19,10 +19,12 @@ package org.apache.ignite.internal.catalog;
 
 import static java.util.concurrent.CompletableFuture.completedFuture;
 import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.apache.ignite.lang.ErrorGroups.Sql.UNSUPPORTED_DDL_OPERATION_ERR;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map.Entry;
@@ -33,6 +35,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
+import org.apache.ignite.internal.catalog.commands.AlterColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableAddColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableDropColumnParams;
 import org.apache.ignite.internal.catalog.commands.CatalogUtils;
@@ -47,6 +50,7 @@ import org.apache.ignite.internal.catalog.descriptors.SchemaDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.TableDescriptor;
 import org.apache.ignite.internal.catalog.events.AddColumnEventParameters;
+import org.apache.ignite.internal.catalog.events.AlterColumnEventParameters;
 import org.apache.ignite.internal.catalog.events.CatalogEvent;
 import org.apache.ignite.internal.catalog.events.CatalogEventParameters;
 import org.apache.ignite.internal.catalog.events.CreateIndexEventParameters;
@@ -54,6 +58,7 @@ import org.apache.ignite.internal.catalog.events.CreateTableEventParameters;
 import org.apache.ignite.internal.catalog.events.DropColumnEventParameters;
 import org.apache.ignite.internal.catalog.events.DropIndexEventParameters;
 import org.apache.ignite.internal.catalog.events.DropTableEventParameters;
+import org.apache.ignite.internal.catalog.storage.AlterColumnEntry;
 import org.apache.ignite.internal.catalog.storage.DropColumnsEntry;
 import org.apache.ignite.internal.catalog.storage.DropIndexEntry;
 import org.apache.ignite.internal.catalog.storage.DropTableEntry;
@@ -77,10 +82,12 @@ import org.apache.ignite.lang.ErrorGroups;
 import org.apache.ignite.lang.ErrorGroups.Common;
 import org.apache.ignite.lang.ErrorGroups.Sql;
 import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
 import org.apache.ignite.lang.IndexAlreadyExistsException;
 import org.apache.ignite.lang.IndexNotFoundException;
 import org.apache.ignite.lang.TableAlreadyExistsException;
 import org.apache.ignite.lang.TableNotFoundException;
+import org.apache.ignite.sql.ColumnType;
 import org.apache.ignite.sql.SqlException;
 import org.jetbrains.annotations.Nullable;
 
@@ -339,6 +346,85 @@ public class CatalogServiceImpl extends Producer<CatalogEvent, CatalogEventParam
         });
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public CompletableFuture<Void> alterColumn(AlterColumnParams params) {
+        return saveUpdate(catalog -> {
+            String schemaName = Objects.requireNonNullElse(params.schemaName(), CatalogService.PUBLIC);
+
+            SchemaDescriptor schema = Objects.requireNonNull(catalog.schema(schemaName), "No schema found: " + schemaName);
+
+            TableDescriptor table = schema.table(params.tableName());
+
+            if (table == null) {
+                throw new TableNotFoundException(schemaName, params.tableName());
+            }
+
+            String columnName = params.columnName();
+
+            TableColumnDescriptor origin = table.columns().stream()
+                    .filter(desc -> desc.name().equals(columnName))
+                    .findFirst()
+                    .orElseThrow(() ->  new ColumnNotFoundException(columnName));
+
+            TableColumnDescriptor target = new TableColumnDescriptor(
+                    origin.name(),
+                    Objects.requireNonNullElse(params.type(), origin.type()),
+                    !Objects.requireNonNullElse(params.notNull(), !origin.nullable()),
+                    Objects.requireNonNullElse(params.precision(), origin.precision()),
+                    Objects.requireNonNullElse(params.scale(), origin.scale()),
+                    Objects.requireNonNullElse(params.length(), origin.length()),
+                    Objects.requireNonNullElse(params.defaultValue(origin.type()), origin.defaultValue())
+            );
+
+            if (origin.equals(target)) {
+                // No modifications required.
+                return Collections.emptyList();
+            }
+
+            boolean isPkColumn = table.isPrimaryKeyColumn(origin.name());
+
+            if (origin.nullable() != target.nullable()) {
+                if (isPkColumn) {
+                    throwUnsupportedDdl("Cannot change NOT NULL for the primary key column '{}'.", origin.name());
+                }
+
+                if (origin.nullable()) {
+                    throwUnsupportedDdl("Cannot set NOT NULL for column '{}'.", origin.name());
+                }
+            }
+
+            if (origin.scale() != target.scale()) {
+                throwUnsupportedDdl("Cannot change scale for column '{}'.", origin.name());
+            }
+
+            if (origin.type() != target.type()) {
+                if (isPkColumn) {
+                    throwUnsupportedDdl("Cannot change data type for primary key column '{}'.", origin.name());
+                }
+
+                if (!CatalogUtils.isSupportedColumnTypeChange(origin.type(), target.type())) {
+                    throwUnsupportedDdl("Cannot change data type for column '{}' [from={}, to={}].",
+                            origin.name(), origin.type(), target.type());
+                }
+            }
+
+            if (origin.length() != target.length() && target.type() != ColumnType.STRING && target.type() != ColumnType.BYTE_ARRAY) {
+                throwUnsupportedDdl("Cannot change length for column '{}'.", origin.name());
+            } else if (target.length() < origin.length()) {
+                throwUnsupportedDdl("Cannot decrease length to {} for column '{}'.", target.length(), origin.name());
+            }
+
+            if (origin.precision() != target.precision() && target.type() != ColumnType.DECIMAL) {
+                throwUnsupportedDdl("Cannot change precision for column '{}'.", origin.name());
+            } else if (target.precision() < origin.precision()) {
+                throwUnsupportedDdl("Cannot decrease precision to {} for column '{}'.", params.precision(), origin.name());
+            }
+
+            return List.of(new AlterColumnEntry(table.id(), target));
+        });
+    }
+
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<Void> createIndex(CreateHashIndexParams params) {
@@ -669,6 +755,40 @@ public class CatalogServiceImpl extends Producer<CatalogEvent, CatalogEventParam
                                     schema.indexes()
                             )
                     );
+                } else if (entry instanceof AlterColumnEntry) {
+                    int tableId = ((AlterColumnEntry) entry).tableId();
+                    TableColumnDescriptor target = ((AlterColumnEntry) entry).descriptor();
+
+                    catalog = new Catalog(
+                            version,
+                            System.currentTimeMillis(),
+                            catalog.objectIdGenState(),
+                            new SchemaDescriptor(
+                                    schema.id(),
+                                    schema.name(),
+                                    version,
+                                    Arrays.stream(schema.tables())
+                                            .map(table -> table.id() != tableId
+                                                    ? table
+                                                    : new TableDescriptor(
+                                                            table.id(),
+                                                            table.name(),
+                                                            table.columns().stream()
+                                                                    .map(source -> source.name().equals(target.name())
+                                                                            ? target
+                                                                            : source).collect(Collectors.toList()),
+                                                            table.primaryKeyColumns(),
+                                                            table.colocationColumns())
+                                            )
+                                            .toArray(TableDescriptor[]::new),
+                                    schema.indexes()
+                            )
+                    );
+
+                    eventFutures.add(fireEvent(
+                            CatalogEvent.TABLE_ALTER,
+                            new AlterColumnEventParameters(version, tableId, target)
+                    ));
                 } else {
                     assert false : entry;
                 }
@@ -688,6 +808,10 @@ public class CatalogServiceImpl extends Producer<CatalogEvent, CatalogEventParam
         }
     }
 
+    private static void throwUnsupportedDdl(String msg, Object... params) {
+        throw new SqlException(UNSUPPORTED_DDL_OPERATION_ERR, IgniteStringFormatter.format(msg, params));
+    }
+
     @FunctionalInterface
     interface UpdateProducer {
         List<UpdateEntry> get(Catalog catalog);
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterColumnParams.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterColumnParams.java
new file mode 100644
index 0000000000..b149a3f794
--- /dev/null
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterColumnParams.java
@@ -0,0 +1,139 @@
+/*
+ * 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.ignite.internal.catalog.commands;
+
+import java.util.function.Function;
+import org.apache.ignite.sql.ColumnType;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * ALTER TABLE ... ALTER COLUMN statement.
+ */
+@SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+public class AlterColumnParams extends AbstractTableCommandParams {
+    private String columnName;
+
+    private ColumnType type;
+
+    private Integer precision;
+
+    private Integer length;
+
+    private Integer scale;
+
+    private Boolean notNull;
+
+    private Function<ColumnType, DefaultValue> defaultResolver;
+
+    /** Returns column name. */
+    public String columnName() {
+        return columnName;
+    }
+
+    /** Returns column type. */
+    public @Nullable ColumnType type() {
+        return type;
+    }
+
+    /** Returns column precision. */
+    public @Nullable Integer precision() {
+        return precision;
+    }
+
+    /** Returns column length. */
+    public @Nullable Integer length() {
+        return length;
+    }
+
+    /** Returns column scale. */
+    public @Nullable Integer scale() {
+        return scale;
+    }
+
+    /** Returns the {@code NOT NULL} constraint change flag. */
+    public @Nullable Boolean notNull() {
+        return notNull;
+    }
+
+    /** Returns a default value depending on the column type. */
+    public @Nullable DefaultValue defaultValue(ColumnType type) {
+        return defaultResolver == null ? null : defaultResolver.apply(type);
+    }
+
+    public static AlterColumnParams.Builder builder() {
+        return new AlterColumnParams.Builder();
+    }
+
+    /**
+     * Parameters builder.
+     */
+    public static class Builder extends AbstractBuilder<AlterColumnParams, Builder> {
+        private Builder() {
+            super(new AlterColumnParams());
+        }
+
+        /** Sets column name. */
+        public Builder columnName(String name) {
+            params.columnName = name;
+
+            return this;
+        }
+
+        /** Sets column type. */
+        public Builder type(ColumnType type) {
+            params.type = type;
+
+            return this;
+        }
+
+        /** Sets column precision. */
+        public Builder precision(int precision) {
+            params.precision = precision;
+
+            return this;
+        }
+
+        /** Sets column length. */
+        public Builder length(int length) {
+            params.length = length;
+
+            return this;
+        }
+
+        /** Sets column scale. */
+        public Builder scale(int scale) {
+            params.scale = scale;
+
+            return this;
+        }
+
+        /** Sets the {@code NOT NULL} constraint change flag. */
+        public Builder notNull(@Nullable Boolean notNull) {
+            params.notNull = notNull;
+
+            return this;
+        }
+
+        /** Sets function that resolves a default value depending on the type of the column. */
+        public Builder defaultValueResolver(@Nullable Function<ColumnType, DefaultValue> resolveDfltFunc) {
+            params.defaultResolver = resolveDfltFunc;
+
+            return this;
+        }
+    }
+}
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
index 0b613a4b02..ef8c3fa370 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
@@ -17,7 +17,11 @@
 
 package org.apache.ignite.internal.catalog.commands;
 
+import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.ignite.internal.catalog.descriptors.ColumnCollation;
@@ -27,11 +31,21 @@ import org.apache.ignite.internal.catalog.descriptors.IndexDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.SortedIndexDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.TableDescriptor;
+import org.apache.ignite.sql.ColumnType;
 
 /**
  * Catalog utils.
  */
 public class CatalogUtils {
+    private static final Map<ColumnType, Set<ColumnType>> ALTER_COLUMN_TYPE_TRANSITIONS = new EnumMap<>(ColumnType.class);
+
+    static {
+        ALTER_COLUMN_TYPE_TRANSITIONS.put(ColumnType.INT8, EnumSet.of(ColumnType.INT16, ColumnType.INT32, ColumnType.INT64));
+        ALTER_COLUMN_TYPE_TRANSITIONS.put(ColumnType.INT16, EnumSet.of(ColumnType.INT32, ColumnType.INT64));
+        ALTER_COLUMN_TYPE_TRANSITIONS.put(ColumnType.INT32, EnumSet.of(ColumnType.INT64));
+        ALTER_COLUMN_TYPE_TRANSITIONS.put(ColumnType.FLOAT, EnumSet.of(ColumnType.DOUBLE));
+    }
+
     /**
      * Converts CreateTable command params to descriptor.
      *
@@ -100,4 +114,17 @@ public class CatalogUtils {
         return new TableColumnDescriptor(params.name(), params.type(), params.nullable(),
                 precision, scale, length, defaultValue);
     }
+
+    /**
+     * Checks if the specified column type transition is supported.
+     *
+     * @param source Source column type.
+     * @param target Target column type.
+     * @return {@code True} if the specified type transition is supported, {@code false} otherwise.
+     */
+    public static boolean isSupportedColumnTypeChange(ColumnType source, ColumnType target) {
+        Set<ColumnType> supportedTransitions = ALTER_COLUMN_TYPE_TRANSITIONS.get(source);
+
+        return supportedTransitions != null && supportedTransitions.contains(target);
+    }
 }
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/ColumnParams.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/ColumnParams.java
index 2b5328df16..f95e0544d7 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/ColumnParams.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/ColumnParams.java
@@ -35,6 +35,15 @@ public class ColumnParams {
     /** Nullability flag. */
     private boolean nullable;
 
+    /** Column length. */
+    private Integer length;
+
+    /** Column precision. */
+    private Integer precision;
+
+    /** Column scale. */
+    private Integer scale;
+
     /** Column default value. */
     private DefaultValue defaultValueDefinition = DefaultValue.constant(null);
 
@@ -73,21 +82,21 @@ public class ColumnParams {
      * Get column's precision.
      */
     public Integer precision() {
-        return null;
+        return precision;
     }
 
     /**
      * Get column's scale.
      */
     public Integer scale() {
-        return null;
+        return scale;
     }
 
     /**
      * Get column's length.
      */
     public Integer length() {
-        return null;
+        return length;
     }
 
     /** Parameters builder. */
@@ -144,6 +153,39 @@ public class ColumnParams {
             return this;
         }
 
+        /**
+         * Sets column precision.
+         *
+         * @return {@code this}.
+         */
+        public Builder precision(int precision) {
+            params.precision = precision;
+
+            return this;
+        }
+
+        /**
+         * Sets column scale.
+         *
+         * @return {@code this}.
+         */
+        public Builder scale(int scale) {
+            params.scale = scale;
+
+            return this;
+        }
+
+        /**
+         * Sets column length.
+         *
+         * @return {@code this}.
+         */
+        public Builder length(int length) {
+            params.length = length;
+
+            return this;
+        }
+
         /**
          * Builds parameters.
          *
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DefaultValue.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DefaultValue.java
index c6b2914c38..7fbdfd6967 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DefaultValue.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DefaultValue.java
@@ -82,6 +82,28 @@ public class DefaultValue implements Serializable {
         public String functionName() {
             return functionName;
         }
+
+        /** {@inheritDoc} */
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+
+            FunctionCall that = (FunctionCall) o;
+
+            return Objects.equals(functionName, that.functionName);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public int hashCode() {
+            return Objects.hash(type, functionName);
+        }
     }
 
     /** Defines default value provider as a constant. */
@@ -98,5 +120,27 @@ public class DefaultValue implements Serializable {
         public @Nullable Serializable value() {
             return value;
         }
+
+        /** {@inheritDoc} */
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+
+            ConstantValue that = (ConstantValue) o;
+
+            return Objects.equals(value, that.value);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public int hashCode() {
+            return Objects.hash(type, value);
+        }
     }
 }
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/AlterColumnEventParameters.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/AlterColumnEventParameters.java
new file mode 100644
index 0000000000..deb49aabeb
--- /dev/null
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/AlterColumnEventParameters.java
@@ -0,0 +1,54 @@
+/*
+ * 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.ignite.internal.catalog.events;
+
+import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
+
+/**
+ * Create table event parameters contains a column descriptor for the modified column.
+ */
+public class AlterColumnEventParameters extends CatalogEventParameters {
+
+    private final int tableId;
+
+    private final TableColumnDescriptor columnDescriptor;
+
+    /**
+     * Constructor.
+     *
+     * @param causalityToken Causality token.
+     * @param tableId Returns an id the table to be modified.
+     * @param columnDescriptor Descriptor for the column to be replaced.
+     */
+    public AlterColumnEventParameters(long causalityToken, int tableId, TableColumnDescriptor columnDescriptor) {
+        super(causalityToken);
+
+        this.tableId = tableId;
+        this.columnDescriptor = columnDescriptor;
+    }
+
+    /** Returns an id of a modified table. */
+    public int tableId() {
+        return tableId;
+    }
+
+    /** Returns column descriptor for the column to be replaced. */
+    public TableColumnDescriptor columnDescriptor() {
+        return columnDescriptor;
+    }
+}
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/CatalogEvent.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/CatalogEvent.java
index cfdad9941e..77425e3732 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/CatalogEvent.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/CatalogEvent.java
@@ -29,7 +29,7 @@ public enum CatalogEvent implements Event {
     /** This event is fired, when a table was dropped in Catalog. */
     TABLE_DROP,
 
-    /** This event is fired, when a column was added to or dropped from a table. */
+    /** This event is fired when a column has been modified, added to, or removed from a table. */
     TABLE_ALTER,
 
     /** This event is fired, when an index was created in Catalog. */
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/AlterColumnEntry.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/AlterColumnEntry.java
new file mode 100644
index 0000000000..ce37538c44
--- /dev/null
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/AlterColumnEntry.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.ignite.internal.catalog.storage;
+
+import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes a column replacement.
+ */
+public class AlterColumnEntry implements UpdateEntry {
+    private static final long serialVersionUID = -4552940987881338656L;
+
+    private final int tableId;
+
+    private final TableColumnDescriptor column;
+
+    /**
+     * Constructs the object.
+     *
+     * @param tableId An id the table to be modified.
+     * @param column A modified descriptor of the column to be replaced.
+     */
+    public AlterColumnEntry(int tableId, TableColumnDescriptor column) {
+        this.tableId = tableId;
+        this.column = column;
+    }
+
+    /** Returns an id the table to be modified. */
+    public int tableId() {
+        return tableId;
+    }
+
+    /** Returns a descriptor for the column to be replaced. */
+    public TableColumnDescriptor descriptor() {
+        return column;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public String toString() {
+        return S.toString(this);
+    }
+}
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogServiceSelfTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogServiceSelfTest.java
index 5ff386c26e..ed46081d8c 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogServiceSelfTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogServiceSelfTest.java
@@ -38,11 +38,18 @@ import static org.mockito.Mockito.verifyNoInteractions;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
+import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.catalog.commands.AlterColumnParams;
+import org.apache.ignite.internal.catalog.commands.AlterColumnParams.Builder;
 import org.apache.ignite.internal.catalog.commands.AlterTableAddColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableDropColumnParams;
+import org.apache.ignite.internal.catalog.commands.CatalogUtils;
 import org.apache.ignite.internal.catalog.commands.ColumnParams;
 import org.apache.ignite.internal.catalog.commands.CreateHashIndexParams;
 import org.apache.ignite.internal.catalog.commands.CreateSortedIndexParams;
@@ -85,9 +92,14 @@ import org.apache.ignite.lang.TableAlreadyExistsException;
 import org.apache.ignite.lang.TableNotFoundException;
 import org.apache.ignite.sql.ColumnType;
 import org.apache.ignite.sql.SqlException;
+import org.hamcrest.TypeSafeMatcher;
+import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.junit.jupiter.params.provider.EnumSource.Mode;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatchers;
 import org.mockito.Mockito;
@@ -491,6 +503,390 @@ public class CatalogServiceSelfTest {
         assertNotNull(schema.table(TABLE_NAME).column("VAL"));
     }
 
+    /**
+     * Checks for possible changes to the default value of a column descriptor.
+     *
+     * <p>Set/drop default value allowed for any column.
+     */
+    @Test
+    public void testAlterColumnDefault() {
+        assertThat(service.createTable(simpleTable(TABLE_NAME)), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        // NULL-> NULL : No-op.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, null, () -> DefaultValue.constant(null)),
+                willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // NULL -> 1 : Ok.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, null, () -> DefaultValue.constant(1)),
+                willBe((Object) null));
+        assertNotNull(service.schema(++schemaVer));
+
+        // 1 -> 1 : No-op.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, null, () -> DefaultValue.constant(1)),
+                willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // 1 -> 2 : Ok.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, null, () -> DefaultValue.constant(2)),
+                willBe((Object) null));
+        assertNotNull(service.schema(++schemaVer));
+
+        // 2 -> NULL : Ok.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, null, () -> DefaultValue.constant(null)),
+                willBe((Object) null));
+        assertNotNull(service.schema(++schemaVer));
+    }
+
+    /**
+     * Checks for possible changes of the nullable flag of a column descriptor.
+     *
+     * <ul>
+     *  <li>{@code DROP NOT NULL} is allowed on any non-PK column.
+     *  <li>{@code SET NOT NULL} is forbidden.
+     * </ul>
+     */
+    @Test
+    public void testAlterColumnNotNull() {
+        assertThat(service.createTable(simpleTable(TABLE_NAME)), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        // NULLABLE -> NULLABLE : No-op.
+        // NOT NULL -> NOT NULL : No-op.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, false, null), willBe((Object) null));
+        assertThat(changeColumn(TABLE_NAME, "VAL_NOT_NULL", null, true, null), willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // NOT NULL -> NULlABLE : Ok.
+        assertThat(changeColumn(TABLE_NAME, "VAL_NOT_NULL", null, false, null), willBe((Object) null));
+        assertNotNull(service.schema(++schemaVer));
+
+        // DROP NOT NULL for PK : Error.
+        assertThat(changeColumn(TABLE_NAME, "ID", null, false, null),
+                willThrowFast(SqlException.class, "Cannot change NOT NULL for the primary key column 'ID'."));
+
+        // NULlABLE -> NOT NULL : Error.
+        assertThat(changeColumn(TABLE_NAME, "VAL", null, true, null),
+                willThrowFast(SqlException.class, "Cannot set NOT NULL for column 'VAL'."));
+        assertThat(changeColumn(TABLE_NAME, "VAL_NOT_NULL", null, true, null),
+                willThrowFast(SqlException.class, "Cannot set NOT NULL for column 'VAL_NOT_NULL'."));
+
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    /**
+     * Checks for possible changes of the precision of a column descriptor.
+     *
+     * <ul>
+     *  <li>Increasing precision is allowed for non-PK {@link ColumnType#DECIMAL} column.</li>
+     *  <li>Decreasing precision is forbidden.</li>
+     * </ul>
+     */
+    @ParameterizedTest
+    @EnumSource(value = ColumnType.class, names = {"DECIMAL"}, mode = Mode.INCLUDE)
+    public void testAlterColumnTypePrecision(ColumnType type) {
+        ColumnParams pkCol = ColumnParams.builder().name("ID").type(ColumnType.INT32).build();
+        ColumnParams col = ColumnParams.builder().name("COL_" + type).type(type).build();
+
+        assertThat(service.createTable(simpleTable(TABLE_NAME, List.of(pkCol, col))), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        // ANY-> UNDEFINED PRECISION : No-op.
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type()), null, null),
+                willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // UNDEFINED PRECISION -> 10 : Ok.
+        assertThat(
+                changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), 10, null, null), null, null),
+                willBe((Object) null)
+        );
+        assertNotNull(service.schema(++schemaVer));
+
+        // 10 -> 11 : Ok.
+        assertThat(
+                changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), 11, null, null), null, null),
+                willBe((Object) null)
+        );
+
+        SchemaDescriptor schema = service.schema(++schemaVer);
+        assertNotNull(schema);
+
+        TableColumnDescriptor desc = schema.table(TABLE_NAME).column(col.name());
+
+        assertNotSame(desc.length(), desc.precision());
+        assertEquals(11, col.type() == ColumnType.DECIMAL ? desc.precision() : desc.length());
+
+        // 11 -> 10 : Error.
+        assertThat(
+                changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), 10, null, null), null, null),
+                willThrowFast(SqlException.class, "Cannot decrease precision to 10 for column '" + col.name() + "'.")
+        );
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    @ParameterizedTest
+    @EnumSource(value = ColumnType.class, names = {"NULL", "DECIMAL"}, mode = Mode.EXCLUDE)
+    public void testAlterColumnTypeAnyPrecisionChangeIsRejected(ColumnType type) {
+        ColumnParams pkCol = ColumnParams.builder().name("ID").type(ColumnType.INT32).build();
+        ColumnParams col = ColumnParams.builder().name("COL").type(type).build();
+        ColumnParams colWithPrecision = ColumnParams.builder().name("COL_PRECISION").type(type).precision(10).build();
+
+        assertThat(service.createTable(simpleTable(TABLE_NAME, List.of(pkCol, col, colWithPrecision))), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(type, 10, null, null), null, null),
+                willThrowFast(SqlException.class, "Cannot change precision for column '" + col.name() + "'"));
+
+        assertThat(changeColumn(TABLE_NAME, colWithPrecision.name(), new TestColumnTypeParams(type, 10, null, null), null, null),
+                willBe((Object) null));
+
+        assertThat(changeColumn(TABLE_NAME, colWithPrecision.name(), new TestColumnTypeParams(type, 9, null, null), null, null),
+                willThrowFast(SqlException.class, "Cannot change precision for column '" + colWithPrecision.name() + "'"));
+
+        assertThat(changeColumn(TABLE_NAME, colWithPrecision.name(), new TestColumnTypeParams(type, 11, null, null), null, null),
+                willThrowFast(SqlException.class, "Cannot change precision for column '" + colWithPrecision.name() + "'"));
+
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    /**
+     * Checks for possible changes of the length of a column descriptor.
+     *
+     * <ul>
+     *  <li>Increasing length is allowed for non-PK {@link ColumnType#STRING} and {@link ColumnType#BYTE_ARRAY} column.</li>
+     *  <li>Decreasing length is forbidden.</li>
+     * </ul>
+     */
+    @ParameterizedTest
+    @EnumSource(value = ColumnType.class, names = {"STRING", "BYTE_ARRAY"}, mode = Mode.INCLUDE)
+    public void testAlterColumnTypeLength(ColumnType type) {
+        ColumnParams pkCol = ColumnParams.builder().name("ID").type(ColumnType.INT32).build();
+        ColumnParams col = ColumnParams.builder().name("COL_" + type).type(type).build();
+
+        assertThat(service.createTable(simpleTable(TABLE_NAME, List.of(pkCol, col))), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        // ANY-> UNDEFINED LENGTH : No-op.
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type()), null, null),
+                willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // UNDEFINED LENGTH -> 10 : Ok.
+        assertThat(
+                changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), null, 10, null), null, null),
+                willBe((Object) null)
+        );
+        assertNotNull(service.schema(++schemaVer));
+
+        // 10 -> 11 : Ok.
+        assertThat(
+                changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), null, 11, null), null, null),
+                willBe((Object) null)
+        );
+
+        SchemaDescriptor schema = service.schema(++schemaVer);
+        assertNotNull(schema);
+
+        TableColumnDescriptor desc = schema.table(TABLE_NAME).column(col.name());
+
+        assertNotSame(desc.length(), desc.precision());
+        assertEquals(11, col.type() == ColumnType.DECIMAL ? desc.precision() : desc.length());
+
+        // 11 -> 10 : Error.
+        assertThat(
+                changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), null, 10, null), null, null),
+                willThrowFast(SqlException.class, "Cannot decrease length to 10 for column '" + col.name() + "'.")
+        );
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    @ParameterizedTest
+    @EnumSource(value = ColumnType.class, names = {"NULL", "STRING", "BYTE_ARRAY"}, mode = Mode.EXCLUDE)
+    public void testAlterColumnTypeAnyLengthChangeIsRejected(ColumnType type) {
+        ColumnParams pkCol = ColumnParams.builder().name("ID").type(ColumnType.INT32).build();
+        ColumnParams col = ColumnParams.builder().name("COL").type(type).build();
+        ColumnParams colWithLength = ColumnParams.builder().name("COL_PRECISION").type(type).length(10).build();
+
+        assertThat(service.createTable(simpleTable(TABLE_NAME, List.of(pkCol, col, colWithLength))), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(type, null, 10, null), null, null),
+                willThrowFast(SqlException.class, "Cannot change length for column '" + col.name() + "'"));
+
+        assertThat(changeColumn(TABLE_NAME, colWithLength.name(), new TestColumnTypeParams(type, null, 10, null), null, null),
+                willBe((Object) null));
+
+        assertThat(changeColumn(TABLE_NAME, colWithLength.name(), new TestColumnTypeParams(type, null, 9, null), null, null),
+                willThrowFast(SqlException.class, "Cannot change length for column '" + colWithLength.name() + "'"));
+
+        assertThat(changeColumn(TABLE_NAME, colWithLength.name(), new TestColumnTypeParams(type, null, 11, null), null, null),
+                willThrowFast(SqlException.class, "Cannot change length for column '" + colWithLength.name() + "'"));
+
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    @ParameterizedTest
+    @EnumSource(value = ColumnType.class, names = "NULL", mode = Mode.EXCLUDE)
+    public void testAlterColumnTypeScaleIsRejected(ColumnType type) {
+        ColumnParams pkCol = ColumnParams.builder().name("ID").type(ColumnType.INT32).build();
+        ColumnParams col = ColumnParams.builder().name("COL_" + type).type(type).scale(3).build();
+        assertThat(service.createTable(simpleTable(TABLE_NAME, List.of(pkCol, col))), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        // ANY-> UNDEFINED SCALE : No-op.
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type()), null, null),
+                willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // 3 -> 3 : No-op.
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), null, null, 3), null, null),
+                willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+
+        // 3 -> 4 : Error.
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), null, null, 4), null, null),
+                willThrowFast(SqlException.class, "Cannot change scale for column '" + col.name() + "'."));
+        assertNull(service.schema(schemaVer + 1));
+
+        // 3 -> 2 : Error.
+        assertThat(changeColumn(TABLE_NAME, col.name(), new TestColumnTypeParams(col.type(), null, null, 2), null, null),
+                willThrowFast(SqlException.class, "Cannot change scale for column '" + col.name() + "'."));
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    /**
+     * Checks for possible changes of the type of a column descriptor.
+     *
+     * <p>The following transitions are allowed for non-PK columns:
+     * <ul>
+     *     <li>INT8 -> INT16 -> INT32 -> INT64</li>
+     *     <li>FLOAT -> DOUBLE</li>
+     * </ul>
+     * All other transitions are forbidden.
+     */
+    @ParameterizedTest(name = "set data type {0}")
+    @EnumSource(value = ColumnType.class, names = "NULL", mode = Mode.EXCLUDE)
+    public void testAlterColumnType(ColumnType target) {
+        EnumSet<ColumnType> types = EnumSet.allOf(ColumnType.class);
+        types.remove(ColumnType.NULL);
+
+        List<ColumnParams> testColumns = types.stream()
+                .map(t -> ColumnParams.builder().name("COL_" + t).type(t).build())
+                .collect(Collectors.toList());
+
+        List<ColumnParams> tableColumns = new ArrayList<>(List.of(ColumnParams.builder().name("ID").type(ColumnType.INT32).build()));
+        tableColumns.addAll(testColumns);
+
+        CreateTableParams createTableParams = simpleTable(TABLE_NAME, tableColumns);
+
+        assertThat(service.createTable(createTableParams), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        for (ColumnParams col : testColumns) {
+            TypeSafeMatcher<CompletableFuture<?>> matcher;
+            boolean sameType = col.type() == target;
+
+            if (sameType || CatalogUtils.isSupportedColumnTypeChange(col.type(), target)) {
+                matcher = willBe((Object) null);
+                schemaVer += sameType ? 0 : 1;
+            } else {
+                matcher = willThrowFast(SqlException.class,
+                        "Cannot change data type for column '" + col.name() + "' [from=" + col.type() + ", to=" + target + "].");
+            }
+
+            TestColumnTypeParams tyoeParams = new TestColumnTypeParams(target);
+
+            assertThat(col.type() + " -> " + target, changeColumn(TABLE_NAME, col.name(), tyoeParams, null, null), matcher);
+            assertNotNull(service.schema(schemaVer));
+            assertNull(service.schema(schemaVer + 1));
+        }
+    }
+
+    @Test
+    public void testAlterColumnTypeRejectedForPrimaryKey() {
+        assertThat(service.createTable(simpleTable(TABLE_NAME)), willBe((Object) null));
+
+        assertThat(changeColumn(TABLE_NAME, "ID", new TestColumnTypeParams(ColumnType.INT64), null, null),
+                willThrowFast(SqlException.class, "Cannot change data type for primary key column 'ID'."));
+    }
+
+    /**
+     * Ensures that the compound change command {@code SET DATA TYPE BIGINT NULL DEFAULT NULL}
+     * will change the type, drop NOT NULL and the default value at the same time.
+     */
+    @Test
+    public void testAlterColumnMultipleChanges() {
+        assertThat(service.createTable(simpleTable(TABLE_NAME)), willBe((Object) null));
+
+        int schemaVer = 1;
+        assertNotNull(service.schema(schemaVer));
+        assertNull(service.schema(schemaVer + 1));
+
+        Supplier<DefaultValue> dflt = () -> DefaultValue.constant(null);
+        boolean notNull = false;
+        TestColumnTypeParams typeParams = new TestColumnTypeParams(ColumnType.INT64);
+
+        // Ensures that 3 different actions applied.
+        assertThat(changeColumn(TABLE_NAME, "VAL_NOT_NULL", typeParams, notNull, dflt), willBe((Object) null));
+
+        SchemaDescriptor schema = service.schema(++schemaVer);
+        assertNotNull(schema);
+
+        TableColumnDescriptor desc = schema.table(TABLE_NAME).column("VAL_NOT_NULL");
+        assertEquals(DefaultValue.constant(null), desc.defaultValue());
+        assertTrue(desc.nullable());
+        assertEquals(ColumnType.INT64, desc.type());
+
+        // Ensures that only one of three actions applied.
+        dflt = () -> DefaultValue.constant(2);
+        assertThat(changeColumn(TABLE_NAME, "VAL_NOT_NULL", typeParams, notNull, dflt), willBe((Object) null));
+
+        schema = service.schema(++schemaVer);
+        assertNotNull(schema);
+        assertEquals(DefaultValue.constant(2), schema.table(TABLE_NAME).column("VAL_NOT_NULL").defaultValue());
+
+        // Ensures that no action will be applied.
+        assertThat(changeColumn(TABLE_NAME, "VAL_NOT_NULL", typeParams, notNull, dflt), willBe((Object) null));
+        assertNull(service.schema(schemaVer + 1));
+    }
+
+    @Test
+    public void testAlterColumnForNonExistingTableRejected() {
+        assertNotNull(service.schema(0));
+        assertNull(service.schema(1));
+
+        assertThat(changeColumn(TABLE_NAME, "ID", null, null, null), willThrowFast(TableNotFoundException.class));
+
+        assertNotNull(service.schema(0));
+        assertNull(service.schema(1));
+    }
+
     @Test
     public void testDropTableWithIndex() throws InterruptedException {
         CreateHashIndexParams params = CreateHashIndexParams.builder()
@@ -824,16 +1220,61 @@ public class CatalogServiceSelfTest {
         verifyNoMoreInteractions(eventListener);
     }
 
+    private CompletableFuture<Void> changeColumn(
+            String tab,
+            String col,
+            @Nullable TestColumnTypeParams typeParams,
+            @Nullable Boolean notNull,
+            @Nullable Supplier<DefaultValue> dflt
+    ) {
+        Builder builder = AlterColumnParams.builder()
+                .tableName(tab)
+                .columnName(col)
+                .notNull(notNull);
+
+        if (dflt != null) {
+            builder.defaultValueResolver(ignore -> dflt.get());
+        }
+
+        if (typeParams != null) {
+            builder.type(typeParams.type);
+
+            if (typeParams.precision != null) {
+                builder.precision(typeParams.precision);
+            }
+
+            if (typeParams.length != null) {
+                builder.length(typeParams.length);
+            }
+
+            if (typeParams.scale != null) {
+                builder.scale(typeParams.scale);
+            }
+        }
+
+        return service.alterColumn(builder.build());
+    }
+
     private static CreateTableParams simpleTable(String name) {
+        List<ColumnParams> cols = List.of(
+                ColumnParams.builder().name("ID").type(ColumnType.INT32).build(),
+                ColumnParams.builder().name("VAL").type(ColumnType.INT32).nullable(true).defaultValue(DefaultValue.constant(null)).build(),
+                ColumnParams.builder().name("VAL_NOT_NULL").type(ColumnType.INT32).defaultValue(DefaultValue.constant(1)).build(),
+                ColumnParams.builder().name("DEC").type(ColumnType.DECIMAL).nullable(true).build(),
+                ColumnParams.builder().name("STR").type(ColumnType.STRING).nullable(true).build(),
+                ColumnParams.builder().name("DEC_SCALE").type(ColumnType.DECIMAL).scale(3).build()
+        );
+
+        return simpleTable(name, cols);
+    }
+
+    private static CreateTableParams simpleTable(String name, List<ColumnParams> cols) {
         return CreateTableParams.builder()
                 .schemaName(SCHEMA_NAME)
                 .tableName(name)
                 .zone(ZONE_NAME)
-                .columns(List.of(
-                        ColumnParams.builder().name("ID").type(ColumnType.INT32).build(),
-                        ColumnParams.builder().name("VAL").type(ColumnType.INT32).nullable(true).build()
-                ))
-                .primaryKeyColumns(List.of("ID"))
+                .columns(cols)
+                .primaryKeyColumns(List.of(cols.get(0).name()))
                 .build();
     }
 
@@ -846,4 +1287,22 @@ public class CatalogServiceSelfTest {
                 .collations(List.of(ColumnCollation.ASC_NULLS_LAST))
                 .build();
     }
+
+    private static class TestColumnTypeParams {
+        private final ColumnType type;
+        private final Integer precision;
+        private final Integer length;
+        private final Integer scale;
+
+        private TestColumnTypeParams(ColumnType type) {
+            this(type, null, null, null);
+        }
+
+        private TestColumnTypeParams(ColumnType type, @Nullable Integer precision, @Nullable Integer length, @Nullable Integer scale) {
+            this.type = type;
+            this.precision = precision;
+            this.length = length;
+            this.scale = scale;
+        }
+    }
 }
diff --git a/modules/sql-engine/src/main/codegen/config.fmpp b/modules/sql-engine/src/main/codegen/config.fmpp
index 4f90ad807a..1130c9dd38 100644
--- a/modules/sql-engine/src/main/codegen/config.fmpp
+++ b/modules/sql-engine/src/main/codegen/config.fmpp
@@ -36,9 +36,7 @@ data: {
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlParserUtil",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterTableAddColumn",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterTableDropColumn",
-      "org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterColumnType",
-      "org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterColumnNotNull",
-      "org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterColumnDefault",
+      "org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterColumn",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlCreateTable",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlCreateIndex",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlCreateTableOption",
diff --git a/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl b/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl
index 8f2ff9485e..058e79491d 100644
--- a/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl
+++ b/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl
@@ -91,36 +91,6 @@ SqlTypeNameSpec UuidType(Span s) :
     }
 }
 
-SqlNode ColumnDataType(SqlIdentifier id) :
-{
-    SqlDataTypeSpec type = null;
-    SqlNode dflt = null;
-    boolean nullable = true;
-    ColumnStrategy strategy = null;
-    final Span s = Span.of();
-}
-{
-    type = DataTypeEx()
-    [
-        <NOT> <NULL> {
-            nullable = false;
-        }
-    ]
-    (
-        <DEFAULT_> { s.add(this); } dflt = Literal() {
-            strategy = ColumnStrategy.DEFAULT;
-        }
-    |
-        {
-            dflt = null;
-            strategy = nullable ? ColumnStrategy.NULLABLE
-                : ColumnStrategy.NOT_NULLABLE;
-        }
-    )
-    {
-        return SqlDdlNodes.column(s.add(id).end(this), id, type.withNullable(nullable), dflt, strategy);
-    }
-}
 
 void TableElement(List<SqlNode> list) :
 {
@@ -401,8 +371,25 @@ SqlNode ColumnWithType() :
 }
 {
     id = SimpleIdentifier()
+    type = DataTypeEx()
+    [
+        <NOT> <NULL> {
+            nullable = false;
+        }
+    ]
+    (
+        <DEFAULT_> { s.add(this); } dflt = Literal() {
+            strategy = ColumnStrategy.DEFAULT;
+        }
+    |
     {
-        return ColumnDataType(id);
+            dflt = null;
+            strategy = nullable ? ColumnStrategy.NULLABLE
+                : ColumnStrategy.NOT_NULLABLE;
+        }
+    )
+    {
+        return SqlDdlNodes.column(s.add(id).end(this), id, type.withNullable(nullable), dflt, strategy);
     }
 }
 
@@ -447,34 +434,51 @@ SqlNode SqlAlterTable() :
 SqlNode SqlAlterColumn(Span s, SqlIdentifier tableId, boolean ifExists) :
 {
     SqlIdentifier id;
-    SqlNode col;
+    SqlDataTypeSpec type;
+    Boolean nullable;
     SqlNode dflt;
 }
 {
     id = SimpleIdentifier()
     (
-        <SET> <DATA> <TYPE> { s.add(this); } col = ColumnDataType(id) {
-            return new IgniteSqlAlterColumnType(s.end(this), ifExists, tableId, col);
+        <SET> <DATA> <TYPE> { s.add(this); } type = DataTypeEx() nullable = NullableOptDefaultNull() dflt = DefaultLiteralOrNull() {
+            return new IgniteSqlAlterColumn(s.end(this), ifExists, tableId, id, type, dflt, nullable == null ? null : !nullable);
         }
     |
         <SET> <NOT> <NULL> {
-            return new IgniteSqlAlterColumnNotNull(s.end(this), ifExists, tableId, id, true);
+            return new IgniteSqlAlterColumn(s.end(this), ifExists, tableId, id, null, null, true);
         }
     |
         <DROP> <NOT> <NULL> {
-            return new IgniteSqlAlterColumnNotNull(s.end(this), ifExists, tableId, id, false);
+            return new IgniteSqlAlterColumn(s.end(this), ifExists, tableId, id, null, null, false);
         }
     |
-        <SET> <DEFAULT_> { s.add(this); } dflt = Literal() {
-            return new IgniteSqlAlterColumnDefault(s.end(this), ifExists, tableId, id, dflt);
+        <SET> <DEFAULT_> { s.add(this); } dflt = Literal()
+        {
+            return new IgniteSqlAlterColumn(s.end(this), ifExists, tableId, id, null, dflt, null);
         }
     |
         <DROP> <DEFAULT_> {
-            return new IgniteSqlAlterColumnDefault(s.end(this), ifExists, tableId, id, null);
+            return new IgniteSqlAlterColumn(s.end(this), ifExists, tableId, id, null, SqlLiteral.createNull(s.end(this)), null);
         }
     )
 }
 
+SqlNode DefaultLiteralOrNull() :
+{
+    SqlNode dflt;
+}
+{
+    <DEFAULT_> dflt = Literal()
+    {
+        return dflt;
+    }
+    |
+    {
+        return null;
+    }
+}
+
 <DEFAULT, DQID, BTID> TOKEN :
 {
 < NEGATE: "!" >
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java
index 99d613145f..63e809571b 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java
@@ -69,6 +69,7 @@ import org.apache.ignite.internal.schema.configuration.index.SortedIndexView;
 import org.apache.ignite.internal.schema.configuration.index.TableIndexChange;
 import org.apache.ignite.internal.schema.configuration.index.TableIndexView;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AbstractTableDdlCommand;
+import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterColumnCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterTableAddCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterTableDropCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterZoneRenameCommand;
@@ -135,6 +136,8 @@ public class DdlCommandHandler {
             return handleAlterAddColumn((AlterTableAddCommand) cmd);
         } else if (cmd instanceof AlterTableDropCommand) {
             return handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof AlterColumnCommand) {
+            return completedFuture(true);
         } else if (cmd instanceof CreateIndexCommand) {
             return handleCreateIndex((CreateIndexCommand) cmd);
         } else if (cmd instanceof DropIndexCommand) {
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerWrapper.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerWrapper.java
index 06830dcd70..a7a9b7f78e 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerWrapper.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerWrapper.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.catalog.commands.CreateHashIndexParams;
 import org.apache.ignite.internal.catalog.commands.CreateSortedIndexParams;
 import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
 import org.apache.ignite.internal.index.IndexManager;
+import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterColumnCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterTableAddCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterTableDropCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.CreateIndexCommand;
@@ -93,6 +94,11 @@ public class DdlCommandHandlerWrapper extends DdlCommandHandler {
                     .thenCompose(res -> catalogManager.dropColumn(DdlToCatalogCommandConverter.convert(dropCommand))
                             .handle(handleModificationResult(dropCommand.ifTableExists(), TableNotFoundException.class))
                     );
+        } else if (cmd instanceof AlterColumnCommand) {
+            return ddlCommandFuture
+                    .thenCompose(res -> catalogManager.alterColumn(DdlToCatalogCommandConverter.convert((AlterColumnCommand) cmd))
+                            .handle(handleModificationResult(((AlterColumnCommand) cmd).ifTableExists(), TableNotFoundException.class))
+                    );
         } else if (cmd instanceof CreateIndexCommand) {
             return ddlCommandFuture
                     .thenCompose(res -> {
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlToCatalogCommandConverter.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlToCatalogCommandConverter.java
index f7c4d5f211..04c9d8fe82 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlToCatalogCommandConverter.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlToCatalogCommandConverter.java
@@ -19,7 +19,10 @@ package org.apache.ignite.internal.sql.engine.exec.ddl;
 
 import java.util.List;
 import java.util.stream.Collectors;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.ignite.internal.catalog.commands.AbstractIndexCommandParams;
+import org.apache.ignite.internal.catalog.commands.AlterColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableAddColumnParams;
 import org.apache.ignite.internal.catalog.commands.AlterTableDropColumnParams;
 import org.apache.ignite.internal.catalog.commands.ColumnParams;
@@ -30,6 +33,7 @@ import org.apache.ignite.internal.catalog.commands.DefaultValue;
 import org.apache.ignite.internal.catalog.commands.DropIndexParams;
 import org.apache.ignite.internal.catalog.commands.DropTableParams;
 import org.apache.ignite.internal.catalog.descriptors.ColumnCollation;
+import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterColumnCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterTableAddCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.AlterTableDropCommand;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.ColumnDefinition;
@@ -68,6 +72,35 @@ class DdlToCatalogCommandConverter {
                 .build();
     }
 
+    static AlterColumnParams convert(AlterColumnCommand cmd) {
+        AlterColumnParams.Builder builder = AlterColumnParams.builder()
+                .schemaName(cmd.schemaName())
+                .tableName(cmd.tableName())
+                .columnName(cmd.columnName())
+                .notNull(cmd.notNull())
+                .defaultValueResolver(cmd.defaultValueResolver());
+
+        RelDataType type = cmd.type();
+
+        if (type != null) {
+            builder.type(TypeUtils.columnType(type));
+
+            if (type.getPrecision() != RelDataType.PRECISION_NOT_SPECIFIED) {
+                if (type.getSqlTypeName() == SqlTypeName.VARCHAR || type.getSqlTypeName() == SqlTypeName.VARBINARY) {
+                    builder.length(type.getPrecision());
+                } else {
+                    builder.precision(type.getPrecision());
+                }
+            }
+
+            if (type.getScale() != RelDataType.SCALE_NOT_SPECIFIED) {
+                builder.scale(type.getScale());
+            }
+        }
+
+        return builder.build();
+    }
+
     static AlterTableAddColumnParams convert(AlterTableAddCommand cmd) {
         List<ColumnParams> columns = cmd.columns().stream().map(DdlToCatalogCommandConverter::convert).collect(Collectors.toList());
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/AlterColumnCommand.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/AlterColumnCommand.java
new file mode 100644
index 0000000000..d1fc39f5c0
--- /dev/null
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/AlterColumnCommand.java
@@ -0,0 +1,69 @@
+/*
+ * 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.ignite.internal.sql.engine.prepare.ddl;
+
+import java.util.function.Function;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.catalog.commands.DefaultValue;
+import org.apache.ignite.sql.ColumnType;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * ALTER TABLE ... ALTER COLUMN statement.
+ */
+public class AlterColumnCommand extends AbstractTableDdlCommand {
+    private String columnName;
+
+    private RelDataType type;
+
+    private Boolean notNull;
+
+    private Function<ColumnType, DefaultValue> resolveDfltFunc;
+
+    public String columnName() {
+        return columnName;
+    }
+
+    void columnName(String name) {
+        columnName = name;
+    }
+
+    void type(RelDataType type) {
+        this.type = type;
+    }
+
+    @Nullable public RelDataType type() {
+        return type;
+    }
+
+    void notNull(boolean notNull) {
+        this.notNull = notNull;
+    }
+
+    @Nullable public Boolean notNull() {
+        return notNull;
+    }
+
+    void defaultValueResolver(Function<ColumnType, DefaultValue> resolveDfltFunc) {
+        this.resolveDfltFunc = resolveDfltFunc;
+    }
+
+    @Nullable public Function<ColumnType, DefaultValue> defaultValueResolver() {
+        return resolveDfltFunc;
+    }
+}
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
index 7f3560a22c..2447f950d7 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
@@ -52,6 +52,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import org.apache.calcite.rel.type.RelDataType;
@@ -75,10 +76,12 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.TimeString;
 import org.apache.calcite.util.TimestampString;
+import org.apache.ignite.internal.catalog.commands.DefaultValue;
 import org.apache.ignite.internal.sql.engine.prepare.IgnitePlanner;
 import org.apache.ignite.internal.sql.engine.prepare.PlanningContext;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.CreateIndexCommand.Type;
 import org.apache.ignite.internal.sql.engine.schema.IgniteIndex.Collation;
+import org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterColumn;
 import org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterTableAddColumn;
 import org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterTableDropColumn;
 import org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterZoneRenameTo;
@@ -93,6 +96,7 @@ import org.apache.ignite.internal.sql.engine.sql.IgniteSqlIndexType;
 import org.apache.ignite.internal.sql.engine.sql.IgniteSqlZoneOption;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.sql.ColumnType;
 import org.apache.ignite.sql.SqlException;
 import org.jetbrains.annotations.Nullable;
 
@@ -213,6 +217,10 @@ public class DdlSqlToCommandConverter {
             return convertAlterTableDrop((IgniteSqlAlterTableDropColumn) ddlNode, ctx);
         }
 
+        if (ddlNode instanceof IgniteSqlAlterColumn) {
+            return convertAlterColumn((IgniteSqlAlterColumn) ddlNode, ctx);
+        }
+
         if (ddlNode instanceof IgniteSqlCreateIndex) {
             return convertAddIndex((IgniteSqlCreateIndex) ddlNode, ctx);
         }
@@ -415,6 +423,39 @@ public class DdlSqlToCommandConverter {
         return DefaultValueDefinition.constant(val);
     }
 
+    private AlterColumnCommand convertAlterColumn(IgniteSqlAlterColumn alterColumnNode, PlanningContext ctx) {
+        AlterColumnCommand cmd = new AlterColumnCommand();
+
+        cmd.schemaName(deriveSchemaName(alterColumnNode.name(), ctx));
+        cmd.tableName(deriveObjectName(alterColumnNode.name(), ctx, "table name"));
+        cmd.ifTableExists(alterColumnNode.ifExists());
+        cmd.columnName(alterColumnNode.columnName().getSimple());
+
+        if (alterColumnNode.dataType() != null) {
+            cmd.type(ctx.planner().convert(alterColumnNode.dataType(), true));
+        }
+
+        if (alterColumnNode.notNull() != null) {
+            cmd.notNull(alterColumnNode.notNull());
+        }
+
+        if (alterColumnNode.expression() != null) {
+            SqlNode expr = alterColumnNode.expression();
+
+            Function<ColumnType, DefaultValue> resolveDfltFunc;
+
+            if (expr instanceof SqlLiteral) {
+                resolveDfltFunc = type -> DefaultValue.constant(fromLiteral(type, (SqlLiteral) expr));
+            } else {
+                throw new IllegalStateException("Invalid expression type " + expr.getClass().getName());
+            }
+
+            cmd.defaultValueResolver(resolveDfltFunc);
+        }
+
+        return cmd;
+    }
+
     /**
      * Converts a given IgniteSqlAlterTableDropColumn AST to a AlterTableDropCommand.
      *
@@ -844,6 +885,56 @@ public class DdlSqlToCommandConverter {
         }
     }
 
+    private static @Nullable Object fromLiteral(ColumnType columnType, SqlLiteral literal) {
+        try {
+            switch (columnType) {
+                case NULL:
+                    return null;
+                case STRING:
+                    return literal.getValueAs(String.class);
+                case DATE: {
+                    SqlLiteral literal0 = ((SqlUnknownLiteral) literal).resolve(SqlTypeName.DATE);
+                    return LocalDate.ofEpochDay(literal0.getValueAs(DateString.class).getDaysSinceEpoch());
+                }
+                case TIME: {
+                    SqlLiteral literal0 = ((SqlUnknownLiteral) literal).resolve(SqlTypeName.TIME);
+                    return LocalTime.ofNanoOfDay(TimeUnit.MILLISECONDS.toNanos(literal0.getValueAs(TimeString.class).getMillisOfDay()));
+                }
+                case TIMESTAMP: {
+                    SqlLiteral literal0 = ((SqlUnknownLiteral) literal).resolve(SqlTypeName.TIMESTAMP);
+                    var tsString = literal0.getValueAs(TimestampString.class);
+
+                    return LocalDateTime.ofEpochSecond(
+                            TimeUnit.MILLISECONDS.toSeconds(tsString.getMillisSinceEpoch()),
+                            (int) (TimeUnit.MILLISECONDS.toNanos(tsString.getMillisSinceEpoch() % 1000)),
+                            ZoneOffset.UTC
+                    );
+                }
+                case INT32:
+                    return literal.getValueAs(Integer.class);
+                case INT64:
+                    return literal.getValueAs(Long.class);
+                case INT16:
+                    return literal.getValueAs(Short.class);
+                case INT8:
+                    return literal.getValueAs(Byte.class);
+                case DECIMAL:
+                    return literal.getValueAs(BigDecimal.class);
+                case DOUBLE:
+                    return literal.getValueAs(Double.class);
+                case FLOAT:
+                    return literal.getValueAs(Float.class);
+                case BYTE_ARRAY:
+                    return literal.getValueAs(byte[].class);
+                default:
+                    throw new IllegalStateException("Unknown type [type=" + columnType + ']');
+            }
+        } catch (Throwable th) {
+            // catch throwable here because literal throws an AssertionError when unable to cast value to a given class
+            throw new SqlException(SQL_TO_REL_CONVERSION_ERR, "Unable co convert literal", th);
+        }
+    }
+
     private static IgniteException unexpectedZoneOption(PlanningContext ctx, String optionName) {
         return new IgniteException(QUERY_VALIDATION_ERR,
                 String.format("Unexpected zone option [option=%s, query=%s]", optionName, ctx.query()));
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumn.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumn.java
index 46fea9fa2b..2f32d4b513 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumn.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumn.java
@@ -17,25 +17,81 @@
 
 package org.apache.ignite.internal.sql.engine.sql;
 
+import java.util.List;
+import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Parse tree for {@code ALTER TABLE ... ALTER COLUMN} statement.
  */
-public abstract class IgniteSqlAlterColumn extends IgniteAbstractSqlAlterTable {
+public class IgniteSqlAlterColumn extends IgniteAbstractSqlAlterTable {
+    private final SqlIdentifier columnName;
+    private final SqlDataTypeSpec type;
+    private final SqlNode dflt;
+    private final Boolean notNull;
+
     /** Constructor. */
-    IgniteSqlAlterColumn(SqlParserPos pos, boolean ifExists, SqlIdentifier tblName) {
+    public IgniteSqlAlterColumn(
+            SqlParserPos pos,
+            boolean ifExists,
+            SqlIdentifier tblName,
+            SqlIdentifier columnName,
+            SqlDataTypeSpec type,
+            SqlNode dflt,
+            Boolean notNull
+    ) {
         super(pos, ifExists, tblName);
+
+        this.columnName = columnName;
+        this.type = type;
+        this.dflt = dflt;
+        this.notNull = notNull;
+    }
+
+    /** Gets column name. */
+    public SqlIdentifier columnName() {
+        return columnName;
     }
 
     /**
-     * Gets column name.
+     * Gets column data type specification.
      *
-     * @return Column name.
+     * @return Column data type specification, {@code null} if the type does not need to be changed.
      */
-    public abstract SqlIdentifier columnName();
+    public @Nullable SqlDataTypeSpec dataType() {
+        return type;
+    }
+
+    /**
+     * Gets the new column DEFAULT expression.
+     *
+     * @return DEFAULT expression or {@code null} if the DEFAULT value does not need to be changed.
+     */
+    public @Nullable SqlNode expression() {
+        return dflt;
+    }
+
+    /**
+     * Gets the {@code NOT NULL} constraint change flag.
+     *
+     * @return {@code True} if the constraint should be added, @code false} if the constraint should be removed,{@code null} if this flag
+     *         does not need to be changed.
+     */
+    public @Nullable Boolean notNull() {
+        return notNull;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<SqlNode> getOperandList() {
+        return ImmutableNullableList.of(name, columnName, type, dflt);
+    }
 
     /** {@inheritDoc} */
     @Override protected void unparseAlterTableOperation(SqlWriter writer, int leftPrec, int rightPrec) {
@@ -44,11 +100,41 @@ public abstract class IgniteSqlAlterColumn extends IgniteAbstractSqlAlterTable {
 
         columnName().unparse(writer, leftPrec, rightPrec);
 
-        unparseAlterColumnOperation(writer, leftPrec, rightPrec);
-    }
+        if (type != null) {
+            writer.keyword("SET DATA TYPE");
 
-    /**
-     * Unparse rest of the ALTER TABLE ... ALTER COLUMN command.
-     */
-    protected abstract void unparseAlterColumnOperation(SqlWriter writer, int leftPrec, int rightPrec);
+            type.unparse(writer, 0, 0);
+
+            if (notNull != null) {
+                if (notNull) {
+                    writer.keyword("NOT");
+                }
+
+                writer.keyword("NULL");
+            }
+
+            if (dflt != null) {
+                writer.keyword("DEFAULT");
+
+                dflt.unparse(writer, leftPrec, rightPrec);
+            }
+
+            return;
+        }
+
+        if (notNull != null) {
+            writer.keyword(notNull ? "SET" : "DROP");
+            writer.keyword("NOT NULL");
+        }
+
+        if (dflt != null) {
+            if (dflt instanceof SqlLiteral && ((SqlLiteral) dflt).getTypeName() == SqlTypeName.NULL) {
+                writer.keyword("DROP DEFAULT");
+            } else {
+                writer.keyword("SET DEFAULT");
+
+                dflt.unparse(writer, leftPrec, rightPrec);
+            }
+        }
+    }
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnDefault.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnDefault.java
deleted file mode 100644
index b51d1f9c20..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnDefault.java
+++ /dev/null
@@ -1,75 +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.ignite.internal.sql.engine.sql;
-
-import java.util.List;
-import java.util.Objects;
-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 org.jetbrains.annotations.Nullable;
-
-/**
- * Parse tree for {@code ALTER TABLE ... ALTER COLUMN ... SET/DROP DEFAULT} statement.
- */
-public class IgniteSqlAlterColumnDefault extends IgniteSqlAlterColumn {
-    /** Column name. */
-    private final SqlIdentifier columnName;
-
-    /** DEFAULT expression. */
-    private final SqlNode dflt;
-
-    /** Constructor. */
-    public IgniteSqlAlterColumnDefault(SqlParserPos pos, boolean ifExists, SqlIdentifier tblName, SqlIdentifier colName, SqlNode dflt) {
-        super(pos, ifExists, tblName);
-
-        this.columnName = Objects.requireNonNull(colName, "colName");
-        this.dflt = dflt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<SqlNode> getOperandList() {
-        return ImmutableNullableList.of(name, columnName, dflt);
-    }
-
-    /** {@inheritDoc} */
-    @Override public SqlIdentifier columnName() {
-        return columnName;
-    }
-
-    /**
-     * Gets the new column DEFAULT expression.
-     *
-     * @return DEFAULT expression or {@code null} if the DEFAULT needs to be dropped.
-     */
-    public @Nullable SqlNode expression() {
-        return dflt;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void unparseAlterColumnOperation(SqlWriter writer, int leftPrec, int rightPrec) {
-        writer.keyword(dflt != null ? "SET" : "DROP");
-        writer.keyword("DEFAULT");
-
-        if (dflt != null) {
-            dflt.unparse(writer, leftPrec, rightPrec);
-        }
-    }
-}
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnNotNull.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnNotNull.java
deleted file mode 100644
index 41adf0ed4d..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnNotNull.java
+++ /dev/null
@@ -1,70 +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.ignite.internal.sql.engine.sql;
-
-import java.util.List;
-import java.util.Objects;
-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;
-
-/**
- * Parse tree for {@code ALTER TABLE ... ALTER COLUMN ... SET/DROP NOT NULL} statement.
- */
-public class IgniteSqlAlterColumnNotNull extends IgniteSqlAlterColumn {
-    /** Column name. */
-    private final SqlIdentifier columnName;
-
-    /** NOT NULL constraint change flag. */
-    private final boolean notNull;
-
-    /** Constructor. */
-    public IgniteSqlAlterColumnNotNull(SqlParserPos pos, boolean ifExists, SqlIdentifier tblName, SqlIdentifier colName, boolean notNull) {
-        super(pos, ifExists, tblName);
-
-        this.columnName = Objects.requireNonNull(colName, "colName");
-        this.notNull = notNull;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<SqlNode> getOperandList() {
-        return ImmutableNullableList.of(name, columnName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public SqlIdentifier columnName() {
-        return columnName;
-    }
-
-    /**
-     * Gets the {@code NOT NULL} constraint change flag.
-     *
-     * @return {@code True} if the constraint should be added, @code false} if the constraint should be removed.
-     */
-    public boolean notNull() {
-        return notNull;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void unparseAlterColumnOperation(SqlWriter writer, int leftPrec, int rightPrec) {
-        writer.keyword(notNull ? "SET" : "DROP");
-        writer.keyword("NOT NULL");
-    }
-}
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnType.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnType.java
deleted file mode 100644
index 8d9f3110ff..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlAlterColumnType.java
+++ /dev/null
@@ -1,96 +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.ignite.internal.sql.engine.sql;
-
-import java.util.List;
-import java.util.Objects;
-import org.apache.calcite.schema.ColumnStrategy;
-import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.ImmutableNullableList;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Parse tree for {@code ALTER TABLE ... ALTER COLUMN ... SET DATA TYPE} statement.
- */
-public class IgniteSqlAlterColumnType extends IgniteSqlAlterColumn {
-    /** Column declaration. */
-    private final SqlColumnDeclaration column;
-
-    /** Constructor. */
-    public IgniteSqlAlterColumnType(SqlParserPos pos, boolean ifExists, SqlIdentifier tblName, SqlNode column) {
-        super(pos, ifExists, tblName);
-
-        this.column = (SqlColumnDeclaration) Objects.requireNonNull(column, "column");
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<SqlNode> getOperandList() {
-        return ImmutableNullableList.of(name, column);
-    }
-
-    /** {@inheritDoc} */
-    @Override public SqlIdentifier columnName() {
-        return column.name;
-    }
-
-    /**
-     * Gets column data type specification.
-     *
-     * @return Column data type specification.
-     */
-    public SqlDataTypeSpec dataType() {
-        return column.dataType;
-    }
-
-    /**
-     * Gets column DEFAULT expression.
-     *
-     * @return Column DEFAULT expression.
-     */
-    public @Nullable SqlNode expression() {
-        return column.expression;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void unparseAlterColumnOperation(SqlWriter writer, int leftPrec, int rightPrec) {
-        writer.keyword("SET DATA TYPE");
-
-        column.dataType.unparse(writer, 0, 0);
-
-        if (Boolean.FALSE.equals(column.dataType.getNullable())) {
-            writer.keyword("NOT NULL");
-        }
-
-        SqlNode expression = column.expression;
-
-        if (expression != null) {
-            if (column.strategy == ColumnStrategy.DEFAULT) {
-                writer.keyword("DEFAULT");
-
-                expression.unparse(writer, 0, 0);
-            } else {
-                throw new AssertionError("Unexpected strategy: " + column.strategy);
-            }
-        }
-    }
-}
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlAlterColumnDdlParserTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlAlterColumnDdlParserTest.java
index 966e317138..927741fdb8 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlAlterColumnDdlParserTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlAlterColumnDdlParserTest.java
@@ -22,14 +22,16 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.util.List;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.pretty.SqlFormatOptions;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.sql.SqlException;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.Test;
 
@@ -44,69 +46,92 @@ public class SqlAlterColumnDdlParserTest extends AbstractDdlParserTest {
     /**
      * Verifies parsing of {@code ALTER TABLE ... ALTER COLUMN ... SET/DROP NOT NULL} statement.
      *
-     * <p>The parser is expected to produce a node of {@link IgniteSqlAlterColumnNotNull} class with the specified table name and the
+     * <p>The parser is expected to produce a node of {@link IgniteSqlAlterColumn} class with the specified table name and the
      * column name.
-     * For the {@code SET NOT NULL} statement, {@link IgniteSqlAlterColumnNotNull#notNull()} must return {@code true}.
-     * For the {@code DROP NOT NULL} statement, {@link IgniteSqlAlterColumnNotNull#notNull()} must return {@code false}.
+     * For the {@code SET NOT NULL} statement, {@link IgniteSqlAlterColumn#notNull()} must return {@code true}.
+     * For the {@code DROP NOT NULL} statement, {@link IgniteSqlAlterColumn#notNull()} must return {@code false}.
      */
     @Test
     public void testNotNull() {
-        Class<IgniteSqlAlterColumnNotNull> expCls = IgniteSqlAlterColumnNotNull.class;
-
-        assertThat(parseAlterColumn("SET NOT NULL", expCls).notNull(), is(true));
-        assertThat(parseAlterColumn("DROP NOT NULL", expCls).notNull(), is(false));
+        assertThat(parseAlterColumn("SET NOT NULL").notNull(), is(true));
+        assertThat(parseAlterColumn("DROP NOT NULL").notNull(), is(false));
     }
 
     /**
      * Verifies parsing of {@code ALTER TABLE ... ALTER COLUMN ... SET/DROP DEFAULT} statement.
      *
-     * <p>The parser is expected to produce a node of {@link IgniteSqlAlterColumnDefault} class with the specified table name and the
+     * <p>The parser is expected to produce a node of {@link IgniteSqlAlterColumn} class with the specified table name and the
      * column name.
-     * For {@code SET DEFAULT 'EXPRESSION'}, {@link IgniteSqlAlterColumnDefault#expression()} must return expected default expression.
-     * For {@code DROP DEFAULT}, {@link IgniteSqlAlterColumnDefault#expression()} must return {@code null}.
+     * <ul>
+     *     <li>Command {@code DROP DEFAULT} must be equivalent to {@code SET DEFAULT NULL}, and {@link IgniteSqlAlterColumn#expression()}
+     *         in this case must contain SQL literal with type NULL.</li>
+     *     <li>For {@code SET DEFAULT &lt;LITERAL&gt;} {@link IgniteSqlAlterColumn#expression()} must contain expected SQL literal.</li>
+     *     <li>For {@code SET DEFAULT &lt;ID&gt;} parser should throw an exception.</li>
+     * </ul>
      */
     @Test
     public void testDefault() {
-        Class<IgniteSqlAlterColumnDefault> expCls = IgniteSqlAlterColumnDefault.class;
-
-        assertNull(parseAlterColumn("DROP DEFAULT", expCls).expression());
+        checkDefaultIsNull(parseAlterColumn("DROP DEFAULT").expression());
+        checkDefaultIsNull(parseAlterColumn("SET DEFAULT NULL", "DROP DEFAULT").expression());
 
-        SqlNode dflt = parseAlterColumn("SET DEFAULT 10", expCls).expression();
+        SqlNode dflt = parseAlterColumn("SET DEFAULT 10").expression();
         assertThat(dflt, instanceOf(SqlLiteral.class));
         assertThat(((SqlLiteral) dflt).getValueAs(Integer.class), equalTo(10));
+
+        assertThrows(SqlException.class, () -> parse(QUERY_PREFIX + "SET DEFAULT FUNC"));
     }
 
     /**
      * Verifies parsing of {@code ALTER TABLE ... ALTER COLUMN ... SET DATA TYPE} statement.
      *
-     * <p>The parser is expected to produce a node of {@link IgniteSqlAlterColumnType} class with the specified {@link
-     * IgniteSqlAlterColumnType#name() table name}, {@link IgniteSqlAlterColumnType#columnName() column name}, column {@link
-     * IgniteSqlAlterColumnType#dataType() data type} and an optional {@link IgniteSqlAlterColumnType#expression() default expression}.
+     * <p>Parser must support the following syntax {@code SET DATA TYPE &lt;new_type&gt; [NOT NULL | NULL] [DEFAULT &lt;default value&gt;]}.
+     *
+     * <p>Parser is expected to produce a node of {@link IgniteSqlAlterColumn} class with the specified {@link
+     * IgniteSqlAlterColumn#name() table name}, {@link IgniteSqlAlterColumn#columnName() column name}, column {@link
+     * IgniteSqlAlterColumn#dataType() data type}, an optional {@link IgniteSqlAlterColumn#expression() default expression}, and an optional
+     * {@link IgniteSqlAlterColumn#notNull() notNull flag}.
      */
     @Test
     public void testSetDataType() {
-        validateDataType("SET DATA TYPE LONG", false, null, "LONG");
-        validateDataType("SET DATA TYPE LONG DEFAULT -1", false, -1L, "LONG");
-        validateDataType("SET DATA TYPE INTEGER NOT NULL", true, null, "INTEGER");
-        validateDataType("SET DATA TYPE INTEGER NOT NULL DEFAULT -1", true, -1, "INTEGER");
+        validateDataType("SET DATA TYPE INTEGER", "INTEGER", null, null);
+        validateDataType("SET DATA TYPE INTEGER NOT NULL", "INTEGER", true, null);
+        validateDataType("SET DATA TYPE INTEGER NULL", "INTEGER", false, null);
+        validateDataType("SET DATA TYPE INTEGER DEFAULT -1", "INTEGER", null, -1L);
+        validateDataType("SET DATA TYPE INTEGER DEFAULT NULL", "INTEGER", null, null);
+        validateDataType("SET DATA TYPE INTEGER NOT NULL DEFAULT -1", "INTEGER", true, -1);
+        validateDataType("SET DATA TYPE INTEGER NULL DEFAULT NULL", "INTEGER", false, null);
+
+        assertThrows(SqlException.class, () -> parse(QUERY_PREFIX + "SET DATA TYPE INTEGER DEFAULT FUNC"));
     }
 
-    private void validateDataType(String querySuffix, boolean notNull, @Nullable Object expDefault, @Nullable String typeName) {
-        IgniteSqlAlterColumnType alterColumn = parseAlterColumn(querySuffix, IgniteSqlAlterColumnType.class);
+    private void validateDataType(String querySuffix, @Nullable String typeName, @Nullable Boolean notNull, @Nullable Object expDefault) {
+        IgniteSqlAlterColumn alterColumn = parseAlterColumn(querySuffix);
 
         assertNotNull(alterColumn.dataType());
         assertThat(alterColumn.dataType().getTypeName().getSimple(), equalTo(typeName));
-        assertThat(alterColumn.dataType().getNullable(), is(!notNull));
+        assertThat(alterColumn.notNull(), equalTo(notNull));
 
         if (expDefault == null) {
-            assertNull(alterColumn.expression());
+            if (alterColumn.expression() != null) {
+                checkDefaultIsNull(alterColumn.expression());
+            }
         } else {
             assertThat(alterColumn.expression(), instanceOf(SqlLiteral.class));
             assertThat(((SqlLiteral) alterColumn.expression()).getValueAs(expDefault.getClass()), equalTo(expDefault));
         }
     }
 
-    private <T extends IgniteSqlAlterColumn> T parseAlterColumn(String querySuffix, Class<T> cls) {
+    private void checkDefaultIsNull(@Nullable SqlNode dflt) {
+        assertNotNull(dflt);
+        assertThat(dflt, instanceOf(SqlLiteral.class));
+        assertThat(((SqlLiteral) dflt).getTypeName(), equalTo(SqlTypeName.NULL));
+    }
+
+    private IgniteSqlAlterColumn parseAlterColumn(String querySuffix) {
+        return parseAlterColumn(querySuffix, null);
+    }
+
+    private IgniteSqlAlterColumn parseAlterColumn(String querySuffix, @Nullable String unparseQuerySuffix) {
         String query = QUERY_PREFIX + querySuffix;
 
         SqlNode node = parse(query);
@@ -118,11 +143,9 @@ public class SqlAlterColumnDdlParserTest extends AbstractDdlParserTest {
         assertThat(alterColumn.columnName().getSimple(), equalTo(COLUMN_NAME));
 
         // Validate unparsed expression.
-        assertThat(unparse(alterColumn), equalTo(query));
-
-        assertThat(alterColumn, instanceOf(cls));
+        assertThat(unparse(alterColumn), equalTo(unparseQuerySuffix == null ? query : QUERY_PREFIX + unparseQuerySuffix));
 
-        return (T) alterColumn;
+        return alterColumn;
     }
 
     private String unparse(SqlNode node) {