You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "lowka (via GitHub)" <gi...@apache.org> on 2023/06/19 20:04:13 UTC

[GitHub] [ignite-3] lowka opened a new pull request, #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

lowka opened a new pull request, #2220:
URL: https://github.com/apache/ignite-3/pull/2220

   (no comment)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245255950


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java:
##########
@@ -315,6 +335,252 @@ public void testDecimalLiteral() {
         assertQuery("SELECT id FROM tbl WHERE val = DECIMAL '10.20'").returns(1).check();
     }
 
+
+    /** decimal casts - cast literal to decimal. */
+    @ParameterizedTest(name = "{2}:{1} AS {3} = {4}")
+    @MethodSource("decimalCastFromLiterals")
+    public void testDecimalCastsNumericLiterals(CaseStatus status, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+
+        Assumptions.assumeTrue(status == CaseStatus.RUN);
+
+        String literal = asLiteral(input, inputType);
+        String query = format("SELECT CAST({} AS {})", literal, targetType);
+
+        QueryChecker checker = assertQuery(query);
+        expectResult(checker, result);
+    }
+
+    private static Stream<Arguments> decimalCastFromLiterals() {
+        RelDataType varcharType = varcharType();
+        // ignored
+        RelDataType numeric = decimalType(4);
+
+        return Stream.of(
+                // String
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),

Review Comment:
   @zstan Added these cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245208804


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -196,10 +198,42 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
 
         if (o instanceof Boolean) {
             throw new UnsupportedOperationException();
+        } else if (o instanceof Number) {
+            return toBigDecimal((Number) o, precision, scale);
+        } else {
+            return toBigDecimal(o.toString(), precision, scale);
+        }
+    }
+
+    /**
+     * Converts the given {@code BigDecimal} to a decimal with the given {@code precision} and {@code scale}
+     * according to SQL spec for CAST specification: General Rules, 8.
+     */
+    public static BigDecimal convertDecimal(BigDecimal value, int precision, int scale) {
+        assert precision > 0 : "Invalid precision: " + precision;
+
+        int defaultPrecision = IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL);
+        if (precision == defaultPrecision) {
+            // This branch covers at least one known case: access to dynamic parameter from context.
+            // In this scenario precision = DefaultTypePrecision, because types for dynamic params
+            // are created by toSql(createType(param.class)).
+            return value;
+        }
+
+        boolean nonZero = !value.unscaledValue().equals(BigInteger.ZERO);
+
+        if (nonZero && scale > precision) {
+            throw new SqlException(QUERY_INVALID_ERR, "Numeric overflow");

Review Comment:
   @zstan I looked at PostgeSQL and it returns the same error in both cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245263682


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDecimalLiteralTest.java:
##########
@@ -66,6 +70,32 @@ public void testValueAndType() {
         assertEquals(expectedType, actualType, "type");
     }
 
+    /**
+     * Type of numeric literal and type of decimal literal should match.
+     */
+    @ParameterizedTest
+    @CsvSource({

Review Comment:
   if we call "1000000000000000000000000" here, test will fail, is it ok ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245269593


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDecimalLiteralTest.java:
##########
@@ -66,6 +70,32 @@ public void testValueAndType() {
         assertEquals(expectedType, actualType, "type");
     }
 
+    /**
+     * Type of numeric literal and type of decimal literal should match.
+     */
+    @ParameterizedTest
+    @CsvSource({

Review Comment:
   `1000000000000000000000000` is a `BIGINT`, it is not a `DECIMAL`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245226285


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java:
##########
@@ -315,6 +335,252 @@ public void testDecimalLiteral() {
         assertQuery("SELECT id FROM tbl WHERE val = DECIMAL '10.20'").returns(1).check();
     }
 
+
+    /** decimal casts - cast literal to decimal. */
+    @ParameterizedTest(name = "{2}:{1} AS {3} = {4}")
+    @MethodSource("decimalCastFromLiterals")
+    public void testDecimalCastsNumericLiterals(CaseStatus status, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+
+        Assumptions.assumeTrue(status == CaseStatus.RUN);
+
+        String literal = asLiteral(input, inputType);
+        String query = format("SELECT CAST({} AS {})", literal, targetType);
+
+        QueryChecker checker = assertQuery(query);
+        expectResult(checker, result);
+    }
+
+    private static Stream<Arguments> decimalCastFromLiterals() {
+        RelDataType varcharType = varcharType();
+        // ignored
+        RelDataType numeric = decimalType(4);
+
+        return Stream.of(
+                // String
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),

Review Comment:
   can u add additional tests (or may be already exist and i miss smt?)
   ```
           String query = format("SELECT CAST(? AS DECIMAL(5, 1))");
   
           BigDecimal bd = new BigDecimal("12345");
   
           sql(query).withParams(bd) <- failed
   
           bd = new BigDecimal("1234");
           sql(query).withParams(bd) <- correct
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245147975


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -119,45 +120,38 @@ public static String toString(ByteString b) {
         return b == null ? null : new String(b.getBytes(), Commons.typeFactory().getDefaultCharset());
     }
 
-    private static BigDecimal setScale(int precision, int scale, BigDecimal decimal) {
-        return precision == IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL)
-            ? decimal : decimal.setScale(scale, RoundingMode.HALF_UP);
-    }
-
     /** CAST(DOUBLE AS DECIMAL). */
     public static BigDecimal toBigDecimal(double val, int precision, int scale) {
-        BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Double) val, precision, scale);
     }
 
     /** CAST(FLOAT AS DECIMAL). */
     public static BigDecimal toBigDecimal(float val, int precision, int scale) {
-        BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Float) val, precision, scale);
     }
 
     /** CAST(java long AS DECIMAL). */
     public static BigDecimal toBigDecimal(long val, int precision, int scale) {
         BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(INT AS DECIMAL). */
     public static BigDecimal toBigDecimal(int val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java short AS DECIMAL). */
     public static BigDecimal toBigDecimal(short val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java byte AS DECIMAL). */
     public static BigDecimal toBigDecimal(byte val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));

Review Comment:
   @zstan  I didn't touch that code -  you can ask an author if you want to :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245226285


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java:
##########
@@ -315,6 +335,252 @@ public void testDecimalLiteral() {
         assertQuery("SELECT id FROM tbl WHERE val = DECIMAL '10.20'").returns(1).check();
     }
 
+
+    /** decimal casts - cast literal to decimal. */
+    @ParameterizedTest(name = "{2}:{1} AS {3} = {4}")
+    @MethodSource("decimalCastFromLiterals")
+    public void testDecimalCastsNumericLiterals(CaseStatus status, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+
+        Assumptions.assumeTrue(status == CaseStatus.RUN);
+
+        String literal = asLiteral(input, inputType);
+        String query = format("SELECT CAST({} AS {})", literal, targetType);
+
+        QueryChecker checker = assertQuery(query);
+        expectResult(checker, result);
+    }
+
+    private static Stream<Arguments> decimalCastFromLiterals() {
+        RelDataType varcharType = varcharType();
+        // ignored
+        RelDataType numeric = decimalType(4);
+
+        return Stream.of(
+                // String
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),

Review Comment:
   can u add additional tests (or may be already exist and i miss smt?)
   ```
           String query = format("SELECT CAST(? AS DECIMAL(5, 1))");
   
           BigDecimal bd = new BigDecimal("12345");
   
           sql(query).withParams(bd) <- failed
   
           bd = new BigDecimal("1234");
           sql(query).withParams(bd) <- correct
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245270727


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDecimalLiteralTest.java:
##########
@@ -66,6 +70,32 @@ public void testValueAndType() {
         assertEquals(expectedType, actualType, "type");
     }
 
+    /**
+     * Type of numeric literal and type of decimal literal should match.
+     */
+    @ParameterizedTest
+    @CsvSource({

Review Comment:
   Updated test's javadoc.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245108429


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/CastExpressionTest.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.exec;
+
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.params.provider.Arguments.arguments;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Stream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.sql.engine.AsyncCursor;
+import org.apache.ignite.internal.sql.engine.AsyncCursor.BatchedResult;
+import org.apache.ignite.internal.sql.engine.framework.DataProvider;
+import org.apache.ignite.internal.sql.engine.framework.TestBuilders;
+import org.apache.ignite.internal.sql.engine.framework.TestCluster;
+import org.apache.ignite.internal.sql.engine.framework.TestNode;
+import org.apache.ignite.internal.sql.engine.prepare.QueryPlan;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Cast expression tests.
+ */
+public class CastExpressionTest {
+
+    private static final String NUMERIC_OVERFLOW_ERROR = "Numeric overflow";
+
+    private static final String NUMERIC_FORMAT_ERROR = "neither a decimal digit number";
+
+    private final DataProvider<Object[]> dataProvider = DataProvider.fromRow(
+            new Object[]{42, UUID.randomUUID().toString()}, 3_333
+    );
+
+    // @formatter:off
+    private final TestCluster cluster = TestBuilders.cluster()
+            .nodes("N1")
+            .addTable()
+            .name("T1")
+            .distribution(IgniteDistributions.hash(List.of(0)))
+            .addColumn("ID", NativeTypes.INT32)
+            .addColumn("VAL", NativeTypes.stringOf(64))
+            .defaultDataProvider(dataProvider)
+            .end()
+            .build();
+    // @formatter:on
+
+    private final TestNode gatewayNode = cluster.node("N1");
+
+    /** Starts the cluster and prepares the plan of the query. */
+    @BeforeEach
+    public void setUp() {
+        cluster.start();
+    }
+
+    /** Stops the cluster. */
+    @AfterEach
+    public void tearDown() throws Exception {
+        cluster.stop();
+    }
+
+    /** varchar casts - literals. */
+    @ParameterizedTest
+    @MethodSource("varcharCasts")
+    public void testVarcharCastsLiterals(String value, RelDataType type, String result) {
+        String query = format("SELECT CAST('{}' AS {})", value, type);
+        sql(query).returns(result).ok();
+    }
+
+    /** varchar casts - dynamic params. */
+    @ParameterizedTest
+    @MethodSource("varcharCasts")
+    public void testVarcharCastsDynamicParams(String value, RelDataType type, String result) {
+        String query = format("SELECT CAST(? AS {})", type);
+        sql(query).withParams(value).returns(result).ok();
+    }
+
+    private static Stream<Arguments> varcharCasts() {
+        return Stream.of(
+                // varchar
+                arguments("abcde", varcharType(3), "abc"),
+                arguments("abcde", varcharType(5), "abcde"),
+                arguments("abcde", varcharType(6), "abcde"),
+                arguments("abcde", varcharType(), "abcde"),
+
+                // char
+                arguments("abcde", charType(), "a"),
+                arguments("abcde", charType(3), "abc")
+        );
+    }
+
+    /** decimal casts - cast literal to decimal. */
+    @ParameterizedTest(name = "{2}:{1} AS {3} = {4}")
+    @MethodSource("decimalCastFromLiterals")
+    public void testDecimalCastsNumericLiterals(CaseStatus status, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+
+        Assumptions.assumeTrue(status == CaseStatus.RUN);
+
+        String literal = asLiteral(input, inputType);
+        String query = format("SELECT CAST({} AS {})", literal, targetType);
+
+        sql(query).expect(result);
+    }
+
+    private static Stream<Arguments> decimalCastFromLiterals() {
+        RelDataType varcharType = varcharType();
+        // ignored
+        RelDataType numeric = decimalType(4);
+
+        return Stream.of(
+                // String
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, varcharType, "100.12", decimalType(5, 1), bigDecimalVal("100.1")),
+                arguments(CaseStatus.RUN, varcharType, "lame", decimalType(5, 1), error(NUMERIC_FORMAT_ERROR)),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, varcharType, "100.12", decimalType(1, 0), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Numeric
+                arguments(CaseStatus.RUN, numeric, "100", decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, numeric, "100", decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, numeric, "100.12", decimalType(5, 1), bigDecimalVal("100.1")),
+                arguments(CaseStatus.SKIP, numeric, "100.12", decimalType(5, 0), bigDecimalVal("100")),
+                arguments(CaseStatus.SKIP, numeric, "100", decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, numeric, "100.12", decimalType(5, 2), bigDecimalVal("100.12"))
+        );
+    }
+
+    /** decimal casts - cast dynamic param to decimal. */
+    @ParameterizedTest(name = "{2}:?{1} AS {3} = {4}")
+    @MethodSource("decimalCasts")
+    public void testDecimalCastsDynamicParams(CaseStatus ignore, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+        // We ignore status because every case should work for dynamic parameter.
+
+        String query = format("SELECT CAST(? AS {})", targetType);
+
+        sql(query).withParams(input).expect(result);
+    }
+
+    /** decimals casts - cast numeric literal to specific type then cast the result to decimal. */
+    @ParameterizedTest(name = "{1}: {2}::{1} AS {3} = {4}")
+    @MethodSource("decimalCasts")
+    public void testDecimalCastsFromNumeric(CaseStatus status, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+
+        Assumptions.assumeTrue(status == CaseStatus.RUN);
+
+        String literal = asLiteral(input, inputType);
+        String query = format("SELECT CAST({}::{} AS {})", literal, inputType, targetType);
+
+        sql(query).expect(result);
+    }
+
+    static String asLiteral(Object value, RelDataType type) {
+        if (SqlTypeUtil.isCharacter(type)) {
+            String str = (String) value;
+            return format("'{}'", str);
+        } else {
+            return String.valueOf(value);
+        }
+    }
+
+    /**
+     * Indicates whether a test case should run or should be skipped.
+     * We need this because the set of test cases is the same for both dynamic params
+     * and numeric values.
+     *
+     * <p>TODO Should be removed after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+     */
+    enum CaseStatus {
+        /** Case should run. */
+        RUN,
+        /** Case should be skipped. */
+        SKIP
+    }
+
+    private static Stream<Arguments> decimalCasts() {
+        RelDataType varcharType = varcharType();
+        RelDataType tinyIntType = sqlType(SqlTypeName.TINYINT);
+        RelDataType smallIntType = sqlType(SqlTypeName.SMALLINT);
+        RelDataType integerType = sqlType(SqlTypeName.INTEGER);
+        RelDataType bigintType = sqlType(SqlTypeName.BIGINT);
+        RelDataType realType = sqlType(SqlTypeName.REAL);
+        RelDataType doubleType = sqlType(SqlTypeName.DOUBLE);
+
+        return Stream.of(
+                // String
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, varcharType, "100", decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, varcharType, "100", decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Tinyint
+                arguments(CaseStatus.SKIP, tinyIntType, (byte) 100, decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, tinyIntType, (byte) 100, decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, tinyIntType, (byte) 100, decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, tinyIntType, (byte) 100, decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Smallint
+                arguments(CaseStatus.RUN, smallIntType, (short) 100, decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, smallIntType, (short) 100, decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, smallIntType, (short) 100, decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, smallIntType, (short) 100, decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Integer
+                arguments(CaseStatus.RUN, integerType, 100, decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, integerType, 100, decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, integerType, 100, decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, integerType, 100, decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Bigint
+                arguments(CaseStatus.RUN, bigintType, 100L, decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, bigintType, 100L, decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, bigintType, 100L, decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, bigintType, 100L, decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Real
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, realType, 100.0f, decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.SKIP, realType, 100.0f, decimalType(3, 0), bigDecimalVal("100")),
+                arguments(CaseStatus.SKIP, realType, 100.0f, decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, realType, 100.0f, decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, realType, 0.1f, decimalType(1, 1), bigDecimalVal("0.1")),
+                arguments(CaseStatus.SKIP, realType, 0.1f, decimalType(2, 2), bigDecimalVal("0.10")),
+                arguments(CaseStatus.SKIP, realType, 10.12f, decimalType(2, 1), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, realType, 0.12f, decimalType(1, 2), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Double
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, doubleType, 100.0d, decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.SKIP, doubleType, 100.0d, decimalType(3, 0), bigDecimalVal("100")),
+                arguments(CaseStatus.SKIP, doubleType, 100.0d, decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, doubleType, 100.0d, decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, doubleType, 0.1d, decimalType(1, 1), bigDecimalVal("0.1")),
+                arguments(CaseStatus.SKIP, doubleType, 0.1d, decimalType(2, 2), bigDecimalVal("0.10")),
+                arguments(CaseStatus.SKIP, doubleType, 10.12d, decimalType(2, 1), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, doubleType, 0.12d, decimalType(1, 2), error(NUMERIC_OVERFLOW_ERROR)),
+
+                // Decimal
+                arguments(CaseStatus.RUN, decimalType(1, 1), new BigDecimal("0.1"), decimalType(1, 1), bigDecimalVal("0.1")),
+                arguments(CaseStatus.RUN, decimalType(3), new BigDecimal("100"), decimalType(3), bigDecimalVal("100")),
+                arguments(CaseStatus.RUN, decimalType(3), new BigDecimal("100"), decimalType(3, 0), bigDecimalVal("100")),
+                // TODO Uncomment these test cases after https://issues.apache.org/jira/browse/IGNITE-19822 is fixed.
+                arguments(CaseStatus.SKIP, decimalType(3), new BigDecimal("100"), decimalType(4, 1), bigDecimalVal("100.0")),
+                arguments(CaseStatus.SKIP, decimalType(3), new BigDecimal("100"), decimalType(2, 0), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, decimalType(1, 1), new BigDecimal("0.1"), decimalType(2, 2), bigDecimalVal("0.10")),
+                arguments(CaseStatus.SKIP, decimalType(4, 2), new BigDecimal("10.12"), decimalType(2, 1), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, decimalType(2, 2), new BigDecimal("0.12"), decimalType(1, 2), error(NUMERIC_OVERFLOW_ERROR)),
+                arguments(CaseStatus.SKIP, decimalType(1, 1), new BigDecimal("0.1"), decimalType(1, 1), bigDecimalVal("0.1"))
+        );
+    }
+
+    private static RelDataType sqlType(SqlTypeName typeName) {
+        return Commons.typeFactory().createSqlType(typeName);
+    }
+
+    private static RelDataType decimalType(int precision, int scale) {
+        return Commons.typeFactory().createSqlType(SqlTypeName.DECIMAL, precision, scale);
+    }
+
+    private static RelDataType decimalType(int precision) {
+        return Commons.typeFactory().createSqlType(SqlTypeName.DECIMAL, precision, RelDataType.SCALE_NOT_SPECIFIED);
+    }
+
+    private static RelDataType varcharType(int length) {
+        return Commons.typeFactory().createSqlType(SqlTypeName.VARCHAR, length);
+    }
+
+    private static RelDataType varcharType() {
+        return Commons.typeFactory().createSqlType(SqlTypeName.VARCHAR);
+    }
+
+    private static RelDataType charType(int length) {
+        return Commons.typeFactory().createSqlType(SqlTypeName.CHAR, length);
+    }
+
+    private static RelDataType charType() {
+        return Commons.typeFactory().createSqlType(SqlTypeName.CHAR);
+    }
+
+    private Checker sql(String query) {
+        return new Checker(query, gatewayNode);
+    }
+
+    /**
+     * Result contains a {@code BigDecimal} value represented by the given string.
+     */
+    private static Result<BigDecimal> bigDecimalVal(String value) {
+        return new Result<>(new BigDecimal(value), null);
+    }
+
+    /** Result contains an error which message contains the following substring. */
+    private static <T> Result<T> error(String error) {
+        return new Result<>(null, error);
+    }
+
+    /**
+     * Contains result of a test case. It can either be a value or an error.
+     *
+     * @param <T> Value type.
+     */
+    private static class Result<T> {
+        final T value;
+        final String error;
+
+        Result(T value, String error) {
+            if (error != null && value != null) {
+                throw new IllegalArgumentException("Both error and value have been specified");
+            }
+            if (error == null && value == null) {
+                throw new IllegalArgumentException("Neither error nor value have been specified");
+            }
+            this.value = value;
+            this.error = error;
+        }
+
+        @Override
+        public String toString() {
+            if (value != null) {
+                return "VAL:" + value;
+            } else {
+                return "ERR:" + error;
+            }
+        }
+    }
+
+    private static class Checker {

Review Comment:
   Can we reuse or extend QueryChecker instead of creating a new one?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245210318


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -196,10 +198,42 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
 
         if (o instanceof Boolean) {
             throw new UnsupportedOperationException();
+        } else if (o instanceof Number) {
+            return toBigDecimal((Number) o, precision, scale);
+        } else {
+            return toBigDecimal(o.toString(), precision, scale);

Review Comment:
   Reverted these changes (but it was just formatting) since original code does the same.
   



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -196,10 +198,42 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
 
         if (o instanceof Boolean) {
             throw new UnsupportedOperationException();
+        } else if (o instanceof Number) {
+            return toBigDecimal((Number) o, precision, scale);
+        } else {
+            return toBigDecimal(o.toString(), precision, scale);
+        }
+    }
+
+    /**
+     * Converts the given {@code BigDecimal} to a decimal with the given {@code precision} and {@code scale}
+     * according to SQL spec for CAST specification: General Rules, 8.
+     */
+    public static BigDecimal convertDecimal(BigDecimal value, int precision, int scale) {
+        assert precision > 0 : "Invalid precision: " + precision;
+
+        int defaultPrecision = IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL);
+        if (precision == defaultPrecision) {
+            // This branch covers at least one known case: access to dynamic parameter from context.
+            // In this scenario precision = DefaultTypePrecision, because types for dynamic params
+            // are created by toSql(createType(param.class)).
+            return value;
+        }
+
+        boolean nonZero = !value.unscaledValue().equals(BigInteger.ZERO);
+
+        if (nonZero && scale > precision) {
+            throw new SqlException(QUERY_INVALID_ERR, "Numeric overflow");
+        }
+
+        int currentSignificantDigits = value.precision() - value.scale();

Review Comment:
   Rewrote that part.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245143000


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -119,45 +120,38 @@ public static String toString(ByteString b) {
         return b == null ? null : new String(b.getBytes(), Commons.typeFactory().getDefaultCharset());
     }
 
-    private static BigDecimal setScale(int precision, int scale, BigDecimal decimal) {
-        return precision == IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL)
-            ? decimal : decimal.setScale(scale, RoundingMode.HALF_UP);
-    }
-
     /** CAST(DOUBLE AS DECIMAL). */
     public static BigDecimal toBigDecimal(double val, int precision, int scale) {
-        BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Double) val, precision, scale);
     }
 
     /** CAST(FLOAT AS DECIMAL). */
     public static BigDecimal toBigDecimal(float val, int precision, int scale) {
-        BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Float) val, precision, scale);
     }
 
     /** CAST(java long AS DECIMAL). */
     public static BigDecimal toBigDecimal(long val, int precision, int scale) {
         BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(INT AS DECIMAL). */
     public static BigDecimal toBigDecimal(int val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java short AS DECIMAL). */
     public static BigDecimal toBigDecimal(short val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java byte AS DECIMAL). */
     public static BigDecimal toBigDecimal(byte val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));

Review Comment:
   why do we need to use _String.valueOf_ construction here ? and for _short_ the same ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245189875


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -196,10 +198,42 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
 
         if (o instanceof Boolean) {
             throw new UnsupportedOperationException();
+        } else if (o instanceof Number) {
+            return toBigDecimal((Number) o, precision, scale);
+        } else {
+            return toBigDecimal(o.toString(), precision, scale);
+        }
+    }
+
+    /**
+     * Converts the given {@code BigDecimal} to a decimal with the given {@code precision} and {@code scale}
+     * according to SQL spec for CAST specification: General Rules, 8.
+     */
+    public static BigDecimal convertDecimal(BigDecimal value, int precision, int scale) {
+        assert precision > 0 : "Invalid precision: " + precision;
+
+        int defaultPrecision = IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL);
+        if (precision == defaultPrecision) {
+            // This branch covers at least one known case: access to dynamic parameter from context.
+            // In this scenario precision = DefaultTypePrecision, because types for dynamic params
+            // are created by toSql(createType(param.class)).
+            return value;
+        }
+
+        boolean nonZero = !value.unscaledValue().equals(BigInteger.ZERO);
+
+        if (nonZero && scale > precision) {
+            throw new SqlException(QUERY_INVALID_ERR, "Numeric overflow");

Review Comment:
   why this raises "Numeric overflow" ? i suppose this is something like: erroneous format or smth else ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245195829


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -196,10 +198,42 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
 
         if (o instanceof Boolean) {
             throw new UnsupportedOperationException();
+        } else if (o instanceof Number) {
+            return toBigDecimal((Number) o, precision, scale);
+        } else {
+            return toBigDecimal(o.toString(), precision, scale);

Review Comment:
   do we really want to try to transform any types here ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245254713


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -209,6 +216,39 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
                : toBigDecimal(o.toString(), precision, scale);
     }
 
+    /**
+     * Converts the given {@code BigDecimal} to a decimal with the given {@code precision} and {@code scale}
+     * according to SQL spec for CAST specification: General Rules, 8.
+     */
+    public static BigDecimal convertDecimal(BigDecimal value, int precision, int scale) {
+        assert precision > 0 : "Invalid precision: " + precision;
+
+        int defaultPrecision = IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL);
+        if (precision == defaultPrecision) {
+            // This branch covers at least one known case: access to dynamic parameter from context.
+            // In this scenario precision = DefaultTypePrecision, because types for dynamic params
+            // are created by toSql(createType(param.class)).
+            return value;
+        }
+
+        boolean nonZero = !value.unscaledValue().equals(BigInteger.ZERO);
+
+        if (nonZero) {
+            if (scale > precision) {
+                throw new SqlException(QUERY_INVALID_ERR, "Numeric overflow");

Review Comment:
   Changed a message to `Numeric field overflow`. Moved it to constant but I do not think is a good idea to use the same constant in tests and this method. If we use the same constant, then someone unintentionally changes the constant in `IgniteFunctions` it also changes the tests, thus changing public API. If we use separate constant for tests and system code , then unintentional changes would break tests because error messages won't match.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245235582


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -209,6 +216,39 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
                : toBigDecimal(o.toString(), precision, scale);
     }
 
+    /**
+     * Converts the given {@code BigDecimal} to a decimal with the given {@code precision} and {@code scale}
+     * according to SQL spec for CAST specification: General Rules, 8.
+     */
+    public static BigDecimal convertDecimal(BigDecimal value, int precision, int scale) {
+        assert precision > 0 : "Invalid precision: " + precision;
+
+        int defaultPrecision = IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL);
+        if (precision == defaultPrecision) {
+            // This branch covers at least one known case: access to dynamic parameter from context.
+            // In this scenario precision = DefaultTypePrecision, because types for dynamic params
+            // are created by toSql(createType(param.class)).
+            return value;
+        }
+
+        boolean nonZero = !value.unscaledValue().equals(BigInteger.ZERO);
+
+        if (nonZero) {
+            if (scale > precision) {
+                throw new SqlException(QUERY_INVALID_ERR, "Numeric overflow");

Review Comment:
   probably "Numeric overflow" must be closer to PG ? i.e. "Numeric field overflow" ? 
   additionally why this message is not a public constant for purpose of further test usage ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan merged pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan merged PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245273576


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java:
##########
@@ -315,6 +335,252 @@ public void testDecimalLiteral() {
         assertQuery("SELECT id FROM tbl WHERE val = DECIMAL '10.20'").returns(1).check();
     }
 
+
+    /** decimal casts - cast literal to decimal. */
+    @ParameterizedTest(name = "{2}:{1} AS {3} = {4}")
+    @MethodSource("decimalCastFromLiterals")
+    public void testDecimalCastsNumericLiterals(CaseStatus status, RelDataType inputType, Object input,
+            RelDataType targetType, Result<BigDecimal> result) {
+
+        Assumptions.assumeTrue(status == CaseStatus.RUN);
+
+        String literal = asLiteral(input, inputType);
+        String query = format("SELECT CAST({} AS {})", literal, targetType);
+
+        QueryChecker checker = assertQuery(query);
+        expectResult(checker, result);
+    }
+
+    private static Stream<Arguments> decimalCastFromLiterals() {
+        RelDataType varcharType = varcharType();
+        // ignored
+        RelDataType numeric = decimalType(4);
+
+        return Stream.of(
+                // String
+                arguments(CaseStatus.RUN, varcharType, "100", decimalType(3), bigDecimalVal("100")),

Review Comment:
   oh, i see these tests are present at testConvertDecimal



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245151091


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -119,45 +120,38 @@ public static String toString(ByteString b) {
         return b == null ? null : new String(b.getBytes(), Commons.typeFactory().getDefaultCharset());
     }
 
-    private static BigDecimal setScale(int precision, int scale, BigDecimal decimal) {
-        return precision == IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL)
-            ? decimal : decimal.setScale(scale, RoundingMode.HALF_UP);
-    }
-
     /** CAST(DOUBLE AS DECIMAL). */
     public static BigDecimal toBigDecimal(double val, int precision, int scale) {
-        BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Double) val, precision, scale);
     }
 
     /** CAST(FLOAT AS DECIMAL). */
     public static BigDecimal toBigDecimal(float val, int precision, int scale) {
-        BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Float) val, precision, scale);
     }
 
     /** CAST(java long AS DECIMAL). */
     public static BigDecimal toBigDecimal(long val, int precision, int scale) {
         BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(INT AS DECIMAL). */
     public static BigDecimal toBigDecimal(int val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java short AS DECIMAL). */
     public static BigDecimal toBigDecimal(short val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java byte AS DECIMAL). */
     public static BigDecimal toBigDecimal(byte val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));

Review Comment:
   @zstan removed it. I agree that it is not necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245151072


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -119,45 +120,38 @@ public static String toString(ByteString b) {
         return b == null ? null : new String(b.getBytes(), Commons.typeFactory().getDefaultCharset());
     }
 
-    private static BigDecimal setScale(int precision, int scale, BigDecimal decimal) {
-        return precision == IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL)
-            ? decimal : decimal.setScale(scale, RoundingMode.HALF_UP);
-    }
-
     /** CAST(DOUBLE AS DECIMAL). */
     public static BigDecimal toBigDecimal(double val, int precision, int scale) {
-        BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Double) val, precision, scale);
     }
 
     /** CAST(FLOAT AS DECIMAL). */
     public static BigDecimal toBigDecimal(float val, int precision, int scale) {
-        BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return toBigDecimal((Float) val, precision, scale);
     }
 
     /** CAST(java long AS DECIMAL). */
     public static BigDecimal toBigDecimal(long val, int precision, int scale) {
         BigDecimal decimal = BigDecimal.valueOf(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(INT AS DECIMAL). */
     public static BigDecimal toBigDecimal(int val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(val);
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java short AS DECIMAL). */
     public static BigDecimal toBigDecimal(short val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));
-        return setScale(precision, scale, decimal);
+        return convertDecimal(decimal, precision, scale);
     }
 
     /** CAST(java byte AS DECIMAL). */
     public static BigDecimal toBigDecimal(byte val, int precision, int scale) {
         BigDecimal decimal = new BigDecimal(String.valueOf(val));

Review Comment:
   probably we need to fill simple issue ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] lowka commented on pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#issuecomment-1611337897

   @AMashenkov I have moved tests to ignite-runner.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] zstan commented on a diff in pull request #2220: IGNITE-19353: Sql. Incorrect type conversion for dynamic parameters - CAST operation ignores type precision.

Posted by "zstan (via GitHub)" <gi...@apache.org>.
zstan commented on code in PR #2220:
URL: https://github.com/apache/ignite-3/pull/2220#discussion_r1245197413


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/IgniteSqlFunctions.java:
##########
@@ -196,10 +198,42 @@ public static BigDecimal toBigDecimal(Object o, int precision, int scale) {
 
         if (o instanceof Boolean) {
             throw new UnsupportedOperationException();
+        } else if (o instanceof Number) {
+            return toBigDecimal((Number) o, precision, scale);
+        } else {
+            return toBigDecimal(o.toString(), precision, scale);
+        }
+    }
+
+    /**
+     * Converts the given {@code BigDecimal} to a decimal with the given {@code precision} and {@code scale}
+     * according to SQL spec for CAST specification: General Rules, 8.
+     */
+    public static BigDecimal convertDecimal(BigDecimal value, int precision, int scale) {
+        assert precision > 0 : "Invalid precision: " + precision;
+
+        int defaultPrecision = IgniteTypeSystem.INSTANCE.getDefaultPrecision(SqlTypeName.DECIMAL);
+        if (precision == defaultPrecision) {
+            // This branch covers at least one known case: access to dynamic parameter from context.
+            // In this scenario precision = DefaultTypePrecision, because types for dynamic params
+            // are created by toSql(createType(param.class)).
+            return value;
+        }
+
+        boolean nonZero = !value.unscaledValue().equals(BigInteger.ZERO);
+
+        if (nonZero && scale > precision) {
+            throw new SqlException(QUERY_INVALID_ERR, "Numeric overflow");
+        }
+
+        int currentSignificantDigits = value.precision() - value.scale();

Review Comment:
   no need to calculate it if nonZero == false;



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org