You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by zs...@apache.org on 2024/02/21 13:18:27 UTC

(ignite-3) branch main updated: IGNITE-20593 Sql. Fix all possible default expressions (#3222)

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

zstan 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 93990c5773 IGNITE-20593 Sql. Fix all possible default expressions (#3222)
93990c5773 is described below

commit 93990c5773003c7b788f11d8bd922de8d7ff539f
Author: Evgeniy Stanilovskiy <st...@gmail.com>
AuthorDate: Wed Feb 21 16:18:21 2024 +0300

    IGNITE-20593 Sql. Fix all possible default expressions (#3222)
---
 .../ignite/internal/sql/engine/ItDmlTest.java      |   6 +-
 .../prepare/ddl/DdlSqlToCommandConverter.java      | 178 ++++++++--
 .../ignite/internal/sql/engine/util/Commons.java   |   2 +-
 ...ules.java => IgniteCustomAssignmentsRules.java} |  16 +-
 .../sql/engine/planner/CastResolutionTest.java     |   6 +-
 .../prepare/ddl/DdlSqlToCommandConverterTest.java  | 385 +++++++++++++++++++++
 6 files changed, 544 insertions(+), 49 deletions(-)

diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
index ac541478b7..22e9bf6d7e 100644
--- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
+++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
@@ -500,12 +500,12 @@ public class ItDmlTest extends BaseSqlIntegrationTest {
     public void testInsertDefaultValue() {
         checkDefaultValue(defaultValueArgs().collect(Collectors.toList()));
 
-        checkWrongDefault("VARCHAR", "10");
+        checkWrongDefault("VARCHAR(1)", "10");
         checkWrongDefault("INT", "'10'");
         checkWrongDefault("INT", "TRUE");
         checkWrongDefault("DATE", "10");
         checkWrongDefault("DATE", "TIME '01:01:01'");
-        checkWrongDefault("TIME", "TIMESTAMP '2021-01-01 01:01:01'");
+        checkWrongDefault("TIMESTAMP", "TIME '01:01:01'");
         checkWrongDefault("BOOLEAN", "1");
 
         // TODO: IGNITE-17373
@@ -583,7 +583,7 @@ public class ItDmlTest extends BaseSqlIntegrationTest {
         try {
             assertThrowsSqlException(
                     Sql.STMT_VALIDATION_ERR,
-                    "Unable convert literal",
+                    "Invalid default value for column 'VAL'",
                     () -> sql("CREATE TABLE test (id INT PRIMARY KEY, val " + sqlType + " DEFAULT " + sqlVal + ")")
             );
         } finally {
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 2e226debb0..803afeeee2 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
@@ -19,6 +19,9 @@ package org.apache.ignite.internal.sql.engine.prepare.ddl;
 
 import static java.util.function.Function.identity;
 import static java.util.stream.Collectors.toUnmodifiableMap;
+import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
+import static org.apache.calcite.rel.type.RelDataType.SCALE_NOT_SPECIFIED;
+import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.AFFINITY_FUNCTION;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.DATA_NODES_AUTO_ADJUST;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.DATA_NODES_AUTO_ADJUST_SCALE_DOWN;
@@ -27,14 +30,20 @@ import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.D
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.DATA_STORAGE_ENGINE;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.PARTITIONS;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.ZoneOptionEnum.REPLICAS;
+import static org.apache.ignite.internal.sql.engine.util.IgniteMath.convertToByteExact;
+import static org.apache.ignite.internal.sql.engine.util.IgniteMath.convertToIntExact;
+import static org.apache.ignite.internal.sql.engine.util.IgniteMath.convertToShortExact;
+import static org.apache.ignite.internal.sql.engine.util.TypeUtils.fromInternal;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_VALIDATION_ERR;
 
 import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.time.Duration;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
 import java.time.LocalTime;
-import java.time.ZoneOffset;
+import java.time.Period;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
 import java.util.EnumMap;
@@ -43,27 +52,34 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Set;
+import java.util.UUID;
 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;
+import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.schema.ColumnStrategy;
 import org.apache.calcite.sql.SqlBasicTypeNameSpec;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDdl;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlIntervalLiteral.IntervalValue;
+import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlUnknownLiteral;
+import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
 import org.apache.calcite.sql.ddl.SqlDdlNodes;
 import org.apache.calcite.sql.ddl.SqlKeyConstraint;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.TimeString;
@@ -350,7 +366,7 @@ public class DdlSqlToCommandConverter {
 
             dedupSetPk.remove(name);
 
-            DefaultValueDefinition dflt = convertDefault(col.expression, relType);
+            DefaultValueDefinition dflt = convertDefault(col.expression, relType, name);
             if (dflt.type() == DefaultValueDefinition.Type.FUNCTION_CALL && !pkCols.contains(name)) {
                 throw new SqlException(STMT_VALIDATION_ERR,
                         "Functional defaults are not supported for non-primary key columns [col=" + name + "]");
@@ -392,9 +408,8 @@ public class DdlSqlToCommandConverter {
 
             Boolean nullable = col.dataType.getNullable();
             RelDataType relType = ctx.planner().convert(col.dataType, nullable != null ? nullable : true);
-            DefaultValueDefinition dflt = convertDefault(col.expression, relType);
-
             String name = col.name.getSimple();
+            DefaultValueDefinition dflt = convertDefault(col.expression, relType, name);
 
             cols.add(new ColumnDefinition(name, relType, dflt));
         }
@@ -404,7 +419,7 @@ public class DdlSqlToCommandConverter {
         return alterTblCmd;
     }
 
-    private static DefaultValueDefinition convertDefault(@Nullable SqlNode expression, RelDataType relType) {
+    private static DefaultValueDefinition convertDefault(@Nullable SqlNode expression, RelDataType relType, String name) {
         if (expression == null) {
             return DefaultValueDefinition.constant(null);
         } else if (expression instanceof SqlIdentifier) {
@@ -413,7 +428,7 @@ public class DdlSqlToCommandConverter {
             ColumnType columnType = TypeUtils.columnType(relType);
             assert columnType != null : "RelType to columnType conversion should not return null";
 
-            Object val = fromLiteral(columnType, (SqlLiteral) expression);
+            Object val = fromLiteral(columnType, name, (SqlLiteral) expression, relType.getPrecision(), relType.getScale());
             return DefaultValueDefinition.constant(val);
         } else {
             throw new IllegalArgumentException("Unsupported default expression: " + expression.getKind());
@@ -441,8 +456,14 @@ public class DdlSqlToCommandConverter {
 
             Function<ColumnType, DefaultValue> resolveDfltFunc;
 
+            @Nullable RelDataType relType = cmd.type();
+
+            int precision = relType == null ? PRECISION_NOT_SPECIFIED : relType.getPrecision();
+            int scale = relType == null ? SCALE_NOT_SPECIFIED : relType.getScale();
+            String name = alterColumnNode.columnName().getSimple();
+
             if (expr instanceof SqlLiteral) {
-                resolveDfltFunc = type -> DefaultValue.constant(fromLiteral(type, (SqlLiteral) expr));
+                resolveDfltFunc = type -> DefaultValue.constant(fromLiteral(type, name, (SqlLiteral) expr, precision, scale));
             } else {
                 throw new IllegalStateException("Invalid expression type " + expr.getKind());
             }
@@ -824,52 +845,135 @@ public class DdlSqlToCommandConverter {
     /**
      * Creates a value of required type from the literal.
      */
-    private static @Nullable Object fromLiteral(ColumnType columnType, SqlLiteral literal) {
+    private static @Nullable Object fromLiteral(ColumnType columnType, String name, SqlLiteral literal, int precision, int scale) {
         if (literal.getValue() == null) {
             return null;
         }
 
         try {
             switch (columnType) {
-                case STRING:
-                    return literal.getValueAs(String.class);
+                case PERIOD: {
+                    if (!(literal instanceof SqlIntervalLiteral)) {
+                        throw new SqlException(STMT_VALIDATION_ERR,
+                                "Default expression is not belongs to interval type");
+                    }
+
+                    String strValue = Objects.requireNonNull(literal.toValue());
+                    SqlNumericLiteral numLiteral = SqlLiteral.createExactNumeric(strValue, literal.getParserPosition());
+                    int val = numLiteral.intValue(true);
+                    SqlIntervalLiteral literal0 = (SqlIntervalLiteral) literal;
+                    SqlIntervalQualifier qualifier = ((IntervalValue) literal0.getValue()).getIntervalQualifier();
+                    if (qualifier.typeName() == SqlTypeName.INTERVAL_YEAR) {
+                        val = val * 12;
+                    }
+                    return fromInternal(val, Period.class);
+                }
+                case DURATION: {
+                    if (!(literal instanceof SqlIntervalLiteral)) {
+                        throw new SqlException(STMT_VALIDATION_ERR,
+                                "Default expression is not belongs to interval type");
+                    }
+                    String strValue = Objects.requireNonNull(literal.toValue());
+                    SqlNumericLiteral numLiteral = SqlLiteral.createExactNumeric(strValue, literal.getParserPosition());
+                    long val = numLiteral.longValue(true);
+                    SqlIntervalLiteral literal0 = (SqlIntervalLiteral) literal;
+                    SqlIntervalQualifier qualifier = ((IntervalValue) literal0.getValue()).getIntervalQualifier();
+                    if (qualifier.typeName() == SqlTypeName.INTERVAL_DAY) {
+                        val = Duration.ofDays(val).toMillis();
+                    } else if (qualifier.typeName() == SqlTypeName.INTERVAL_HOUR) {
+                        val = Duration.ofHours(val).toMillis();
+                    } else if (qualifier.typeName() == SqlTypeName.INTERVAL_MINUTE) {
+                        val = Duration.ofMinutes(val).toMillis();
+                    } else if (qualifier.typeName() == SqlTypeName.INTERVAL_SECOND) {
+                        val = Duration.ofSeconds(val).toMillis();
+                    }
+                    return fromInternal(val, Duration.class);
+                }
+                case STRING: {
+                    String val = literal.toValue();
+                    // varchar without limitation
+                    if (precision != PRECISION_NOT_SPECIFIED && Objects.requireNonNull(val).length() > precision) {
+                        throw new SqlException(STMT_VALIDATION_ERR,
+                                format("Value too long for type character({})", precision));
+                    }
+                    return val;
+                }
+                case UUID:
+                    return UUID.fromString(Objects.requireNonNull(literal.toValue()));
                 case DATE: {
-                    SqlLiteral literal0 = ((SqlUnknownLiteral) literal).resolve(SqlTypeName.DATE);
-                    return LocalDate.ofEpochDay(literal0.getValueAs(DateString.class).getDaysSinceEpoch());
+                    try {
+                        literal = SqlParserUtil.parseDateLiteral(literal.getValueAs(String.class), literal.getParserPosition());
+                        int val = literal.getValueAs(DateString.class).getDaysSinceEpoch();
+                        return fromInternal(val, LocalDate.class);
+                    } catch (CalciteContextException e) {
+                        literal = SqlParserUtil.parseTimestampLiteral(literal.getValueAs(String.class), literal.getParserPosition());
+                        TimestampString tsString = literal.getValueAs(TimestampString.class);
+                        int val = convertToIntExact(TimeUnit.MILLISECONDS.toDays(tsString.getMillisSinceEpoch()));
+                        return fromInternal(val, LocalDate.class);
+                    }
                 }
                 case TIME: {
-                    SqlLiteral literal0 = ((SqlUnknownLiteral) literal).resolve(SqlTypeName.TIME);
-                    return LocalTime.ofNanoOfDay(TimeUnit.MILLISECONDS.toNanos(literal0.getValueAs(TimeString.class).getMillisOfDay()));
+                    String strLiteral = literal.getValueAs(String.class).trim();
+                    int pos = strLiteral.indexOf(' ');
+                    if (pos != -1) {
+                        strLiteral = strLiteral.substring(pos);
+                    }
+                    literal = SqlParserUtil.parseTimeLiteral(strLiteral, literal.getParserPosition());
+                    int val = literal.getValueAs(TimeString.class).getMillisOfDay();
+                    return fromInternal(val, LocalTime.class);
                 }
                 case DATETIME: {
-                    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
-                    );
+                    literal = SqlParserUtil.parseTimestampLiteral(literal.getValueAs(String.class), literal.getParserPosition());
+                    var tsString = literal.getValueAs(TimestampString.class);
+
+                    return fromInternal(tsString.getMillisSinceEpoch(), LocalDateTime.class);
                 }
                 case TIMESTAMP:
                     // TODO: IGNITE-17376
                     throw new UnsupportedOperationException("Type is not supported: " + columnType);
-                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 INT32: {
+                    acceptNumericLiteral(literal, columnType);
+                    long val = literal.longValue(true);
+                    return convertToIntExact(val);
+                }
+                case INT64: {
+                    acceptNumericLiteral(literal, columnType);
+                    BigDecimal val = literal.bigDecimalValue();
+                    return Objects.requireNonNull(val).longValueExact();
+                }
+                case INT16: {
+                    acceptNumericLiteral(literal, columnType);
+                    long val = literal.longValue(true);
+                    return convertToShortExact(val);
+                }
+                case INT8: {
+                    acceptNumericLiteral(literal, columnType);
+                    long val = literal.longValue(true);
+                    return convertToByteExact(val);
+                }
                 case DECIMAL:
-                    return literal.getValueAs(BigDecimal.class);
+                    acceptNumericLiteral(literal, columnType);
+                    BigDecimal val = literal.getValueAs(BigDecimal.class);
+                    val = val.setScale(scale, RoundingMode.HALF_UP);
+                    if (val.precision() > precision) {
+                        throw new SqlException(STMT_VALIDATION_ERR, format("Numeric field overflow for type decimal({}, {})",
+                                precision, scale));
+                    }
+                    return val;
                 case DOUBLE:
+                    acceptNumericLiteral(literal, columnType);
                     return literal.getValueAs(Double.class);
                 case FLOAT:
+                    acceptNumericLiteral(literal, columnType);
                     return literal.getValueAs(Float.class);
                 case BYTE_ARRAY:
-                    return literal.getValueAs(byte[].class);
+                    byte[] arr = literal.getValueAs(byte[].class);
+                    // varbinary without limitation
+                    if (precision != PRECISION_NOT_SPECIFIED && Objects.requireNonNull(arr).length > precision) {
+                        throw new SqlException(STMT_VALIDATION_ERR,
+                                format("Value too long for type binary({})", precision));
+                    }
+                    return arr;
                 case BOOLEAN:
                     return literal.getValueAs(Boolean.class);
                 default:
@@ -877,7 +981,13 @@ public class DdlSqlToCommandConverter {
             }
         } catch (Throwable th) {
             // catch throwable here because literal throws an AssertionError when unable to cast value to a given class
-            throw new SqlException(STMT_VALIDATION_ERR, "Unable convert literal", th);
+            throw new SqlException(STMT_VALIDATION_ERR, format("Invalid default value for column '{}'", name), th);
+        }
+    }
+
+    private static void acceptNumericLiteral(SqlLiteral literal, ColumnType columnType) {
+        if (!(literal instanceof SqlNumericLiteral)) {
+            throw new SqlException(STMT_VALIDATION_ERR, "Default expression can`t be applied to type " + columnType);
         }
     }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
index 2af747c5cd..a17481d2ed 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
@@ -182,7 +182,7 @@ public final class Commons {
     }
 
     private static SqlTypeCoercionRule standardCompatibleCoercionRules() {
-        return SqlTypeCoercionRule.instance(IgniteCustomAssigmentsRules.instance().getTypeMapping());
+        return SqlTypeCoercionRule.instance(IgniteCustomAssignmentsRules.instance().getTypeMapping());
     }
 
     /**
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteCustomAssigmentsRules.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteCustomAssignmentsRules.java
similarity index 95%
rename from modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteCustomAssigmentsRules.java
rename to modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteCustomAssignmentsRules.java
index adaced4036..e46cebed09 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteCustomAssigmentsRules.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteCustomAssignmentsRules.java
@@ -64,18 +64,18 @@ import org.apache.calcite.util.Util;
  * Calcite native rules {@link SqlTypeCoercionRule} and {@link SqlTypeAssignmentRule} are not satisfy SQL standard rules,
  * thus custom implementation is implemented.
  */
-public class IgniteCustomAssigmentsRules implements SqlTypeMappingRule {
+public class IgniteCustomAssignmentsRules implements SqlTypeMappingRule {
     private final Map<SqlTypeName, ImmutableSet<SqlTypeName>> map;
 
-    private static final IgniteCustomAssigmentsRules INSTANCE;
+    private static final IgniteCustomAssignmentsRules INSTANCE;
 
-    private IgniteCustomAssigmentsRules(
+    private IgniteCustomAssignmentsRules(
             Map<SqlTypeName, ImmutableSet<SqlTypeName>> map) {
         this.map = ImmutableMap.copyOf(map);
     }
 
     static {
-        IgniteCustomAssigmentsRules.Builder rules = builder();
+        IgniteCustomAssignmentsRules.Builder rules = builder();
 
         Set<SqlTypeName> rule = EnumSet.noneOf(SqlTypeName.class);
 
@@ -235,19 +235,19 @@ public class IgniteCustomAssigmentsRules implements SqlTypeMappingRule {
         rule.add(SqlTypeName.TIMESTAMP);
         rules.add(SqlTypeName.ANY, rule);
 
-        INSTANCE = new IgniteCustomAssigmentsRules(rules.map);
+        INSTANCE = new IgniteCustomAssignmentsRules(rules.map);
     }
 
     @Override public Map<SqlTypeName, ImmutableSet<SqlTypeName>> getTypeMapping() {
         return this.map;
     }
 
-    public static IgniteCustomAssigmentsRules instance() {
+    public static IgniteCustomAssignmentsRules instance() {
         return INSTANCE;
     }
 
-    public static IgniteCustomAssigmentsRules.Builder builder() {
-        return new IgniteCustomAssigmentsRules.Builder();
+    public static IgniteCustomAssignmentsRules.Builder builder() {
+        return new IgniteCustomAssignmentsRules.Builder();
     }
 
     /** Keeps state while building the type mappings. */
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CastResolutionTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CastResolutionTest.java
index 9e16b484d4..0b3aca2358 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CastResolutionTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CastResolutionTest.java
@@ -51,7 +51,7 @@ import org.apache.calcite.sql.type.SqlTypeCoercionRule;
 import org.apache.calcite.sql.type.SqlTypeMappingRule;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.ignite.internal.sql.engine.type.UuidType;
-import org.apache.ignite.internal.sql.engine.util.IgniteCustomAssigmentsRules;
+import org.apache.ignite.internal.sql.engine.util.IgniteCustomAssignmentsRules;
 import org.junit.jupiter.api.DynamicTest;
 import org.junit.jupiter.api.TestFactory;
 
@@ -213,7 +213,7 @@ public class CastResolutionTest extends AbstractPlannerTest {
     public Stream<DynamicTest> allowedCastsFromNull() {
         List<DynamicTest> testItems = new ArrayList<>();
 
-        SqlTypeMappingRule rules = SqlTypeCoercionRule.instance(IgniteCustomAssigmentsRules.instance().getTypeMapping());
+        SqlTypeMappingRule rules = SqlTypeCoercionRule.instance(IgniteCustomAssignmentsRules.instance().getTypeMapping());
 
         for (SqlTypeName type : ALL_TYPES) {
             if (type == SqlTypeName.NULL) {
@@ -242,7 +242,7 @@ public class CastResolutionTest extends AbstractPlannerTest {
         List<SqlTypeName> singleDayIntervals = List.of(INTERVAL_DAY, INTERVAL_HOUR, INTERVAL_MINUTE,
                 INTERVAL_SECOND);
 
-        SqlTypeMappingRule rules = SqlTypeCoercionRule.instance(IgniteCustomAssigmentsRules.instance().getTypeMapping());
+        SqlTypeMappingRule rules = SqlTypeCoercionRule.instance(IgniteCustomAssignmentsRules.instance().getTypeMapping());
 
         for (SqlTypeName toType : singleIntervals) {
             for (SqlTypeName fromType : EXACT_TYPES) {
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverterTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverterTest.java
index cd2d8da7b5..4f37c962b5 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverterTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverterTest.java
@@ -17,8 +17,20 @@
 
 package org.apache.ignite.internal.sql.engine.prepare.ddl;
 
+import static org.apache.calcite.sql.type.SqlTypeName.DECIMAL;
+import static org.apache.calcite.sql.type.SqlTypeName.EXACT_TYPES;
+import static org.apache.calcite.sql.type.SqlTypeName.FLOAT;
+import static org.apache.calcite.sql.type.SqlTypeName.INTERVAL_TYPES;
+import static org.apache.calcite.sql.type.SqlTypeName.NUMERIC_TYPES;
+import static org.apache.calcite.sql.type.SqlTypeName.REAL;
+import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.DdlSqlToCommandConverter.checkDuplicates;
 import static org.apache.ignite.internal.sql.engine.prepare.ddl.DdlSqlToCommandConverter.collectDataStorageNames;
+import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException;
+import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.generateValueByType;
+import static org.apache.ignite.internal.sql.engine.util.TypeUtils.columnType;
+import static org.apache.ignite.internal.sql.engine.util.TypeUtils.fromInternal;
+import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_VALIDATION_ERR;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
@@ -27,24 +39,46 @@ import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.startsWith;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
+import java.math.BigDecimal;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.Period;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
+import java.util.UUID;
 import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.stream.Stream;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlDdl;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.sql.engine.prepare.PlanningContext;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.DefaultValueDefinition.FunctionCall;
 import org.apache.ignite.internal.sql.engine.prepare.ddl.DefaultValueDefinition.Type;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.apache.ignite.internal.testframework.WithSystemProperty;
 import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.sql.ColumnType;
 import org.hamcrest.CustomMatcher;
 import org.hamcrest.Matcher;
 import org.hamcrest.Matchers;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
 
 /**
  * For {@link DdlSqlToCommandConverter} testing.
@@ -158,6 +192,312 @@ public class DdlSqlToCommandConverterTest extends AbstractDdlSqlToCommandConvert
         );
     }
 
+    @SuppressWarnings({"ThrowableNotThrown"})
+    @TestFactory
+    public Stream<DynamicTest> numericDefaultWithIntervalTypes() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        for (SqlTypeName numType : NUMERIC_TYPES) {
+            for (SqlTypeName intervalType : INTERVAL_TYPES) {
+                RelDataType initialNumType = Commons.typeFactory().createSqlType(numType);
+                ColumnType colType = columnType(initialNumType);
+                Object value = generateValueByType(1000, Objects.requireNonNull(colType));
+                String intervalTypeStr = makeUsableIntervalType(intervalType.getName());
+
+                fillTestCase(intervalTypeStr, "" + value, testItems, false, ctx);
+                fillTestCase(intervalTypeStr, "'" + value + "'", testItems, false, ctx);
+            }
+        }
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> intervalDefaultsWithNumericTypes() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        for (SqlTypeName intervalType : INTERVAL_TYPES) {
+            for (SqlTypeName numType : NUMERIC_TYPES) {
+                String value = makeUsableIntervalValue(intervalType.getName());
+
+                fillTestCase(numType.getName(), value, testItems, false, ctx);
+            }
+        }
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> nonIntervalDefaultsWithIntervalTypes() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        String[] values = {"'01:01:02'", "'2020-01-02 01:01:01'", "'2020-01-02'", "true", "'true'", "x'01'"};
+
+        for (String value : values) {
+            for (SqlTypeName intervalType : INTERVAL_TYPES) {
+                fillTestCase(makeUsableIntervalType(intervalType.getName()), value, testItems, false, ctx);
+            }
+        }
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> intervalDefaultsWithIntervalTypes() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        assertEquals(Period.of(1, 1, 0), fromInternal(13, Period.class));
+        assertEquals(Period.of(1, 0, 0), fromInternal(12, Period.class));
+
+        fillTestCase("INTERVAL YEARS", "INTERVAL '1' YEAR", testItems, true, ctx, fromInternal(12, Period.class));
+        fillTestCase("INTERVAL YEARS", "INTERVAL '12' MONTH", testItems, true, ctx, fromInternal(12, Period.class));
+        fillTestCase("INTERVAL YEARS TO MONTHS", "INTERVAL '1' YEAR", testItems, true, ctx, fromInternal(12, Period.class));
+        fillTestCase("INTERVAL YEARS TO MONTHS", "INTERVAL '13' MONTH", testItems, true, ctx, fromInternal(13, Period.class));
+        fillTestCase("INTERVAL MONTHS", "INTERVAL '1' YEAR", testItems, true, ctx, fromInternal(12, Period.class));
+        fillTestCase("INTERVAL MONTHS", "INTERVAL '13' MONTHS", testItems, true, ctx, fromInternal(13, Period.class));
+
+        long oneDayMillis = Duration.ofDays(1).toMillis();
+        long oneHourMillis = Duration.ofHours(1).toMillis();
+        long oneMinuteMillis = Duration.ofMinutes(1).toMillis();
+        long oneSecondMillis = Duration.ofSeconds(1).toMillis();
+
+        fillTestCase("INTERVAL DAYS", "INTERVAL '1' DAY", testItems, true, ctx, fromInternal(oneDayMillis, Duration.class));
+        fillTestCase("INTERVAL DAYS TO HOURS", "INTERVAL '1' HOURS", testItems, true, ctx, fromInternal(oneHourMillis, Duration.class));
+        fillTestCase("INTERVAL HOURS TO SECONDS", "INTERVAL '1' MINUTE", testItems, true, ctx,
+                fromInternal(oneMinuteMillis, Duration.class));
+        fillTestCase("INTERVAL MINUTES TO SECONDS", "INTERVAL '1' MINUTE", testItems, true, ctx,
+                fromInternal(oneMinuteMillis, Duration.class));
+        fillTestCase("INTERVAL MINUTES TO SECONDS", "INTERVAL '1' SECOND", testItems, true, ctx,
+                fromInternal(oneSecondMillis, Duration.class));
+
+        return testItems.stream();
+    }
+
+    @SuppressWarnings({"ThrowableNotThrown"})
+    @Test
+    public void testUuidWithDefaults() throws SqlParseException {
+        PlanningContext ctx = createContext();
+        String template = "CREATE TABLE t (id INTEGER PRIMARY KEY, d UUID DEFAULT {})";
+
+        String sql = format(template, "NULL");
+        CreateTableCommand cmd = (CreateTableCommand) converter.convert((SqlDdl) parse(sql), ctx);
+        ColumnDefinition def = cmd.columns().get(1);
+        DefaultValueDefinition.ConstantValue defVal = def.defaultValueDefinition();
+        assertNull(defVal.value());
+
+        UUID uuid = UUID.randomUUID();
+        sql = format(template, "'" + uuid + "'");
+        cmd = (CreateTableCommand) converter.convert((SqlDdl) parse(sql), ctx);
+        def = cmd.columns().get(1);
+        defVal = def.defaultValueDefinition();
+        assertEquals(uuid, defVal.value());
+
+        String[] values = {"'01:01:02'", "'2020-01-02 01:01:01'", "'2020-01-02'", "true", "'true'", "x'01'", "INTERVAL '1' DAY"};
+        for (String value : values) {
+            String sql0 = format(template, value);
+            assertThrowsSqlException(STMT_VALIDATION_ERR, "Invalid default value for column", () ->
+                    converter.convert((SqlDdl) parse(sql0), ctx));
+        }
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> numericTypesWithNumericDefaults() {
+        Pattern exactNumeric = Pattern.compile("^\\d+$");
+        Pattern numeric = Pattern.compile("^\\d+(\\.{1}\\d*)?$");
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        String[] numbers = {"100.4", "100.6", "100", "'100'", "'100.1'"};
+
+        List<SqlTypeName> typesWithoutDecimal = new ArrayList<>(NUMERIC_TYPES);
+        typesWithoutDecimal.remove(DECIMAL);
+
+        for (String value : numbers) {
+            for (SqlTypeName numericType : typesWithoutDecimal) {
+                Object toCompare = null;
+                boolean acceptable = true;
+
+                if (!numeric.matcher(value).matches()) {
+                    fillTestCase(numericType.getName(), value, testItems, false, ctx);
+                    continue;
+                }
+
+                if (EXACT_TYPES.contains(numericType)) {
+                    if (!exactNumeric.matcher(value).matches()) {
+                        acceptable = false;
+                    }
+                } else if (numericType == FLOAT || numericType == REAL) {
+                    toCompare = Float.parseFloat(value);
+                } else {
+                    toCompare = Double.parseDouble(value);
+                }
+
+                fillTestCase(numericType.getName(), value, testItems, acceptable, ctx, toCompare);
+            }
+        }
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> decimalDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("DECIMAL", "100", testItems, true, ctx, new BigDecimal(100));
+        fillTestCase("DECIMAL", "100.5", testItems, true, ctx, new BigDecimal(101));
+
+        fillTestCase("DECIMAL(4, 1)", "100", testItems, true, ctx, new BigDecimal("100.0"));
+        fillTestCase("DECIMAL(4, 1)", "100.4", testItems, true, ctx, new BigDecimal("100.4"));
+        fillTestCase("DECIMAL(4, 1)", "100.6", testItems, true, ctx, new BigDecimal("100.6"));
+        fillTestCase("DECIMAL(4, 1)", "100.12", testItems, true, ctx, new BigDecimal("100.1"));
+        fillTestCase("DECIMAL(4, 1)", "1000.12", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> numericTypesWithNonNumericDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        String[] values = {"'01:01:02'", "'2020-01-02 01:01:01'", "'2020-01-02'", "true", "'true'", "x'01'", "INTERVAL '1' DAY"};
+
+        for (String value : values) {
+            for (SqlTypeName numericType : NUMERIC_TYPES) {
+                fillTestCase(numericType.getName(), value, testItems, false, ctx);
+            }
+        }
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> testCharTypesWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("CHAR", "1", testItems, true, ctx, "1");
+        fillTestCase("CHAR", "'1'", testItems, true, ctx, "1");
+        fillTestCase("CHAR(2)", "12", testItems, true, ctx, "12");
+        fillTestCase("CHAR", "12", testItems, false, ctx);
+        fillTestCase("VARCHAR", "12", testItems, true, ctx, "12");
+        fillTestCase("VARCHAR", "'12'", testItems, true, ctx, "12");
+        fillTestCase("VARCHAR(2)", "123", testItems, false, ctx);
+        fillTestCase("VARCHAR(2)", "'123'", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> timestampWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("TIMESTAMP", "'2020-01-02 01:01:01.23'", testItems, true, ctx,
+                LocalDateTime.of(2020, 1, 2, 1, 1, 1, 230_000_000));
+        fillTestCase("TIMESTAMP", "'2020-01-02'", testItems, true, ctx,
+                LocalDateTime.of(2020, 1, 2, 0, 0));
+        fillTestCase("TIMESTAMP", "'01:01:02'", testItems, false, ctx);
+        fillTestCase("TIMESTAMP", "'1'", testItems, false, ctx);
+        fillTestCase("TIMESTAMP", "1", testItems, false, ctx);
+        fillTestCase("TIMESTAMP", "'2020-01-02 01:01:01ERR'", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> dateWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("DATE", "'2020-01-02 01:01:01'", testItems, true, ctx,
+                LocalDate.of(2020, 1, 2));
+        fillTestCase("DATE", "'2020-01-02'", testItems, true, ctx,
+                LocalDate.of(2020, 1, 2));
+        fillTestCase("DATE", "'01:01:01'", testItems, false, ctx);
+        fillTestCase("DATE", "'1'", testItems, false, ctx);
+        fillTestCase("DATE", "1", testItems, false, ctx);
+        fillTestCase("DATE", "'2020-01-02ERR'", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> timeWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("TIME", "'2020-01-02 01:01:01'", testItems, true, ctx,
+                LocalTime.of(1, 1, 1));
+        fillTestCase("TIME", "'2020-01-02'", testItems, false, ctx);
+        fillTestCase("TIME", "'01:01:01.2'", testItems, true, ctx,
+                LocalTime.of(1, 1, 1, 200000000));
+        fillTestCase("TIME", "'1'", testItems, false, ctx);
+        fillTestCase("TIME", "1", testItems, false, ctx);
+        fillTestCase("TIME", "'01:01:01ERR'", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> binaryWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("BINARY", "x'01'", testItems, true, ctx, fromInternal(new byte[]{(byte) 1}, byte[].class));
+        fillTestCase("BINARY", "'01'", testItems, false, ctx);
+        fillTestCase("BINARY", "1", testItems, false, ctx);
+        fillTestCase("BINARY", "x'0102'", testItems, false, ctx);
+        fillTestCase("BINARY(2)", "x'0102'", testItems, true, ctx, fromInternal(new byte[]{(byte) 1, (byte) 2}, byte[].class));
+        fillTestCase("VARBINARY", "x'0102'", testItems, true, ctx, fromInternal(new byte[]{(byte) 1, (byte) 2}, byte[].class));
+        fillTestCase("VARBINARY", "'0102'", testItems, false, ctx);
+        fillTestCase("VARBINARY", "1", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @TestFactory
+    public Stream<DynamicTest> booleanWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+
+        fillTestCase("BOOLEAN", "true", testItems, true, ctx, true);
+        fillTestCase("BOOLEAN", "'true'", testItems, false, ctx);
+        fillTestCase("BOOLEAN", "'1'", testItems, false, ctx);
+        fillTestCase("BOOLEAN", "'yes'", testItems, false, ctx);
+
+        fillTestCase("BOOLEAN", "false", testItems, true, ctx);
+        fillTestCase("BOOLEAN", "'false'", testItems, false, ctx);
+        fillTestCase("BOOLEAN", "'0'", testItems, false, ctx);
+        fillTestCase("BOOLEAN", "'no'", testItems, false, ctx);
+
+        fillTestCase("BOOLEAN", "'2'", testItems, false, ctx);
+
+        return testItems.stream();
+    }
+
+    @Disabled("Remove after https://issues.apache.org/jira/browse/IGNITE-19274 is implemented.")
+    @TestFactory
+    public Stream<DynamicTest> timestampWithTzWithDefaults() {
+        List<DynamicTest> testItems = new ArrayList<>();
+        PlanningContext ctx = createContext();
+        String template = "CREATE TABLE t (id INTEGER PRIMARY KEY, d {} DEFAULT {})";
+
+        {
+            String sql = format(template, "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "'2020-01-02 01:01:01'");
+
+            testItems.add(DynamicTest.dynamicTest(String.format("ALLOW: %s", sql), () ->
+                    converter.convert((SqlDdl) parse(sql), ctx)));
+        }
+
+        return testItems.stream();
+    }
+
     @Test
     public void tableWithAutogenPkColumn() throws SqlParseException {
         var node = parse("CREATE TABLE t (id varchar default gen_random_uuid primary key, val int)");
@@ -194,4 +534,49 @@ public class DdlSqlToCommandConverterTest extends AbstractDdlSqlToCommandConvert
             }
         };
     }
+
+    // Transforms INTERVAL_YEAR_MONTH -> INTERVAL YEAR
+    private static String makeUsableIntervalType(String typeName) {
+        if (typeName.lastIndexOf('_') != typeName.indexOf('_')) {
+            typeName = typeName.substring(0, typeName.lastIndexOf('_'));
+        }
+        typeName = typeName.replace("_", " ");
+        return typeName;
+    }
+
+    // Transforms INTERVAL_YEAR_MONTH -> INTERVAL '1' YEAR
+    private static String makeUsableIntervalValue(String typeName) {
+        return makeUsableIntervalType(typeName).replace(" ", " '1' ");
+    }
+
+    private void fillTestCase(String type, String val, List<DynamicTest> testItems, boolean acceptable, PlanningContext ctx) {
+        fillTestCase(type, val, testItems, acceptable, ctx, null);
+    }
+
+    @SuppressWarnings({"ThrowableNotThrown"})
+    private void fillTestCase(String type, String val, List<DynamicTest> testItems, boolean acceptable, PlanningContext ctx,
+            @Nullable Object compare) {
+        String template = "CREATE TABLE t (id INTEGER PRIMARY KEY, d {} DEFAULT {})";
+        String sql = format(template, type, val);
+
+        if (acceptable) {
+            testItems.add(DynamicTest.dynamicTest(String.format("ALLOW: %s", sql), () -> {
+                CreateTableCommand cmd = (CreateTableCommand) converter.convert((SqlDdl) parse(sql), ctx);
+                ColumnDefinition def = cmd.columns().get(1);
+                DefaultValueDefinition.ConstantValue defVal = def.defaultValueDefinition();
+                Object defaultValue = defVal.value();
+                if (compare != null) {
+                    if (compare instanceof byte[]) {
+                        assertArrayEquals((byte[]) compare, (byte[]) defaultValue);
+                    } else {
+                        assertEquals(compare, defaultValue);
+                    }
+                }
+            }));
+        } else {
+            testItems.add(DynamicTest.dynamicTest(String.format("NOT ALLOW: %s", sql), () ->
+                    assertThrowsSqlException(STMT_VALIDATION_ERR, "Invalid default value for column", () ->
+                            converter.convert((SqlDdl) parse(sql), ctx))));
+        }
+    }
 }