You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2018/07/13 12:55:00 UTC

[GitHub] arina-ielchiieva closed pull request #1380: DRILL-6472: Prevent using zero precision in CAST function

arina-ielchiieva closed pull request #1380: DRILL-6472: Prevent using zero precision in CAST function
URL: https://github.com/apache/drill/pull/1380
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
index aeffe5a8763..68f5475513e 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
@@ -64,17 +64,20 @@ public void setup() {
   public void eval() {
     java.math.BigDecimal bd =
         <#if type.from == "Decimal9" || type.from == "Decimal18">
-        java.math.BigDecimal.valueOf(in.value)
+        java.math.BigDecimal.valueOf(in.value);
         <#else>
         org.apache.drill.exec.util.DecimalUtility
           <#if type.from.contains("Sparse")>
-            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.nDecimalDigits, in.scale, true)
+            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.nDecimalDigits, in.scale, true);
           <#elseif type.from == "VarDecimal">
-            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale)
+            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
           </#if>
         </#if>
-                .setScale(scale.value, java.math.RoundingMode.HALF_UP)
-                .round(new java.math.MathContext(precision.value, java.math.RoundingMode.HALF_UP));
+
+    org.apache.drill.exec.util.DecimalUtility.checkValueOverflow(bd, precision.value, scale.value);
+
+    bd = bd.setScale(scale.value, java.math.RoundingMode.HALF_UP);
+
     out.scale = scale.value;
     out.precision = precision.value;
     out.start = 0;
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
index f051310d2a7..245f6d10b27 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
@@ -71,12 +71,11 @@ public void eval() {
 
     out.start = 0;
     java.math.BigDecimal bd =
-        new java.math.BigDecimal(
-            String.valueOf(in.value),
-            new java.math.MathContext(
-                precision.value,
-                java.math.RoundingMode.HALF_UP))
-        .setScale(scale.value, java.math.RoundingMode.HALF_UP);
+        new java.math.BigDecimal(String.valueOf(in.value));
+
+    org.apache.drill.exec.util.DecimalUtility.checkValueOverflow(bd, precision.value, scale.value);
+
+    bd = bd.setScale(scale.value, java.math.RoundingMode.HALF_UP);
 
     byte[] bytes = bd.unscaledValue().toByteArray();
     int len = bytes.length;
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
index 162c5629e1b..033c92e1c94 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
@@ -67,9 +67,11 @@ public void eval() {
 
     out.start = 0;
     out.buffer = buffer;
-    java.math.BigDecimal bd = new java.math.BigDecimal(in.value,
-        new java.math.MathContext(precision.value, java.math.RoundingMode.HALF_UP))
-        .setScale(out.scale, java.math.BigDecimal.ROUND_DOWN);
+    java.math.BigDecimal bd = new java.math.BigDecimal(in.value);
+
+    org.apache.drill.exec.util.DecimalUtility.checkValueOverflow(bd, precision.value, scale.value);
+
+    bd = bd.setScale(out.scale, java.math.BigDecimal.ROUND_DOWN);
 
     byte[] bytes = bd.unscaledValue().toByteArray();
     int len = bytes.length;
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
index a6e209d1d17..ac1fccba655 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -93,12 +93,12 @@ public void eval() {
     byte[] buf = new byte[in.end - in.start];
     in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
     String s = new String(buf, com.google.common.base.Charsets.UTF_8);
-    java.math.BigDecimal bd =
-        new java.math.BigDecimal(s,
-            new java.math.MathContext(
-                precision.value,
-                java.math.RoundingMode.HALF_UP))
-        .setScale(scale.value, java.math.RoundingMode.HALF_UP);
+    java.math.BigDecimal bd = new java.math.BigDecimal(s);
+
+    org.apache.drill.exec.util.DecimalUtility.checkValueOverflow(bd, precision.value, scale.value);
+
+    bd = bd.setScale(scale.value, java.math.RoundingMode.HALF_UP);
+
     byte[] bytes = bd.unscaledValue().toByteArray();
     int len = bytes.length;
     out.buffer = buffer.reallocIfNeeded(len);
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
index 059080e15fb..da18cb11fde 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
@@ -193,16 +193,18 @@ public void eval() {
 
 </#list>
 
-<#list ["Abs", "Ceil", "Floor", "Trunc", "Round"] as functionName>
+<#list ["Abs", "Ceil", "Floor", "Trunc", "Round", "Negative"] as functionName>
   <#if functionName == "Ceil">
   @FunctionTemplate(names = {"ceil", "ceiling"},
   <#elseif functionName == "Trunc">
   @FunctionTemplate(names = {"trunc", "truncate"},
+  <#elseif functionName == "Negative">
+  @FunctionTemplate(names = {"negative", "u-", "-"},
   <#else>
   @FunctionTemplate(name = "${functionName?lower_case}",
   </#if>
                     scope = FunctionTemplate.FunctionScope.SIMPLE,
-                  <#if functionName == "Abs">
+                  <#if functionName == "Abs" || functionName == "Negative">
                     returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE,
                   <#elseif functionName == "Ceil" || functionName == "Floor"
                       || functionName == "Trunc" || functionName == "Round">
@@ -226,6 +228,9 @@ public void eval() {
               .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale)
           <#if functionName == "Abs">
                   .abs();
+      result.scale = in.scale;
+          <#elseif functionName == "Negative">
+                  .negate();
       result.scale = in.scale;
           <#elseif functionName == "Ceil">
                   .setScale(0, java.math.BigDecimal.ROUND_CEILING);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index d6b0951403c..eee141e77a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -86,6 +86,7 @@
 
 import com.google.common.base.Joiner;
 import org.apache.drill.exec.store.ColumnExplorer;
+import org.apache.drill.exec.util.DecimalUtility;
 
 /**
  * Class responsible for managing parsing, validation and toRel conversion for sql statements.
@@ -562,10 +563,22 @@ public RexNode makeCast(RelDataType type, RexNode exp, boolean matchNullability)
       // that differs from the value from specified RelDataType, cast cannot be removed
       // TODO: remove this code when CALCITE-1468 is fixed
       if (type.getSqlTypeName() == SqlTypeName.DECIMAL && exp instanceof RexLiteral) {
+        if (type.getPrecision() < 1) {
+          throw UserException.validationError()
+              .message("Expected precision greater than 0, but was %s.", type.getPrecision())
+              .build(logger);
+        }
+        if (type.getScale() > type.getPrecision()) {
+          throw UserException.validationError()
+              .message("Expected scale less than or equal to precision, " +
+                  "but was scale %s and precision %s.", type.getScale(), type.getPrecision())
+              .build(logger);
+        }
         RexLiteral literal = (RexLiteral) exp;
         Comparable value = literal.getValueAs(Comparable.class);
         if (value instanceof BigDecimal) {
           BigDecimal bigDecimal = (BigDecimal) value;
+          DecimalUtility.checkValueOverflow(bigDecimal, type.getPrecision(), type.getScale());
           if (bigDecimal.scale() != type.getScale() || bigDecimal.precision() != type.getPrecision()) {
             return makeAbstractCast(type, exp);
           }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
index 80479491aa8..390a6bf8f33 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
@@ -953,7 +953,7 @@ public void testNegate() throws Exception {
             .sqlQuery(query)
             .unOrdered()
             .baselineColumns("col1")
-            .baselineValues(-1.1)
+            .baselineValues(new BigDecimal("-1.1"))
             .go();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
index 01ceb90a0ce..eeded5c81ce 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
@@ -18,18 +18,19 @@
 package org.apache.drill.exec.fn.impl;
 
 import java.math.BigDecimal;
-import java.math.MathContext;
-import java.math.RoundingMode;
 import java.time.LocalDate;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
+import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 
 import com.google.common.collect.Lists;
@@ -37,10 +38,15 @@
 
 import mockit.integration.junit4.JMockit;
 
+import static org.hamcrest.CoreMatchers.containsString;
+
 @RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestCastFunctions extends BaseTestQuery {
 
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
   @Test
   public void testVarbinaryToDate() throws Exception {
     testBuilder()
@@ -380,22 +386,14 @@ public void testCastIntAndBigIntToDecimal() throws Exception {
           .baselineValues(new BigDecimal(1), new BigDecimal(1), new BigDecimal(1), new BigDecimal(1))
           .baselineValues(new BigDecimal(-1), new BigDecimal(-1), new BigDecimal(-1), new BigDecimal(-1))
 
-          .baselineValues(new BigDecimal(Integer.MAX_VALUE)
-                              .round(new MathContext(9, RoundingMode.HALF_UP))
-                              .setScale(0, RoundingMode.HALF_UP),
+          .baselineValues(new BigDecimal(Integer.MAX_VALUE),
                           new BigDecimal(Integer.MAX_VALUE),
-                          new BigDecimal(Long.MAX_VALUE)
-                              .round(new MathContext(9, RoundingMode.HALF_UP))
-                              .setScale(0, RoundingMode.HALF_UP),
+                          new BigDecimal(Long.MAX_VALUE),
                           new BigDecimal(Long.MAX_VALUE))
 
-          .baselineValues(new BigDecimal(Integer.MIN_VALUE)
-                              .round(new MathContext(9, RoundingMode.HALF_UP))
-                              .setScale(0, RoundingMode.HALF_UP),
+          .baselineValues(new BigDecimal(Integer.MIN_VALUE),
                           new BigDecimal(Integer.MIN_VALUE),
-                          new BigDecimal(Long.MIN_VALUE)
-                              .round(new MathContext(9, RoundingMode.HALF_UP))
-                              .setScale(0, RoundingMode.HALF_UP),
+                          new BigDecimal(Long.MIN_VALUE),
                           new BigDecimal(Long.MIN_VALUE))
 
           .baselineValues(new BigDecimal(123456789),
@@ -421,21 +419,13 @@ public void testCastDecimalToIntAndBigInt() throws Exception {
         .baselineValues(0, 0, 0L, 0L)
         .baselineValues(1, 1, 1L, 1L)
         .baselineValues(-1, -1, -1L, -1L)
-        .baselineValues(new BigDecimal(Integer.MAX_VALUE)
-                          .round(new MathContext(9, RoundingMode.HALF_UP))
-                          .setScale(0, RoundingMode.HALF_UP).intValue(),
+        .baselineValues(Integer.MAX_VALUE,
                         (int) Long.MAX_VALUE,
-                        new BigDecimal(Integer.MAX_VALUE)
-                          .round(new MathContext(9, RoundingMode.HALF_UP))
-                          .setScale(0, RoundingMode.HALF_UP).longValue(),
+                        (long) Integer.MAX_VALUE,
                         Long.MAX_VALUE)
-        .baselineValues(new BigDecimal(Integer.MIN_VALUE)
-                          .round(new MathContext(9, RoundingMode.HALF_UP))
-                          .setScale(0, RoundingMode.HALF_UP).intValue(),
+        .baselineValues(Integer.MIN_VALUE,
                         (int) Long.MIN_VALUE,
-                        new BigDecimal(Integer.MIN_VALUE)
-                          .round(new MathContext(9, RoundingMode.HALF_UP))
-                          .setScale(0, RoundingMode.HALF_UP).longValue(),
+                        (long) Integer.MIN_VALUE,
                         Long.MIN_VALUE)
         .baselineValues(123456789, 123456789, 123456789L, 123456789L)
         .go();
@@ -604,4 +594,74 @@ public void testCastDecimalLiteral() throws Exception {
         .baselineValues(new BigDecimal("100.00"))
         .go();
   }
+
+  @Test
+  public void testCastDecimalZeroPrecision() throws Exception {
+    String query = "select cast('123.0' as decimal(0, 5))";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Expected precision greater than 0, but was 0"));
+
+    test(query);
+  }
+
+  @Test
+  public void testCastDecimalGreaterScaleThanPrecision() throws Exception {
+    String query = "select cast('123.0' as decimal(3, 5))";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Expected scale less than or equal to precision, but was scale 5 and precision 3"));
+
+    test(query);
+  }
+
+  @Test
+  public void testCastIntDecimalOverflow() throws Exception {
+    String query = "select cast(i1 as DECIMAL(4, 0)) as s1 from (select cast(123456 as int) as i1)";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Value 123456 overflows specified precision 4 with scale 0"));
+
+    test(query);
+  }
+
+  @Test
+  public void testCastBigIntDecimalOverflow() throws Exception {
+    String query = "select cast(i1 as DECIMAL(4, 0)) as s1 from (select cast(123456 as bigint) as i1)";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Value 123456 overflows specified precision 4 with scale 0"));
+
+    test(query);
+  }
+
+  @Test
+  public void testCastFloatDecimalOverflow() throws Exception {
+    String query = "select cast(i1 as DECIMAL(4, 0)) as s1 from (select cast(123456.123 as float) as i1)";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Value 123456.123 overflows specified precision 4 with scale 0"));
+
+    test(query);
+  }
+
+  @Test
+  public void testCastDoubleDecimalOverflow() throws Exception {
+    String query = "select cast(i1 as DECIMAL(4, 0)) as s1 from (select cast(123456.123 as double) as i1)";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Value 123456.123 overflows specified precision 4 with scale 0"));
+
+    test(query);
+  }
+
+  @Test
+  public void testCastVarCharDecimalOverflow() throws Exception {
+    String query = "select cast(i1 as DECIMAL(4, 0)) as s1 from (select cast(123456.123 as varchar) as i1)";
+
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("VALIDATION ERROR: Value 123456.123 overflows specified precision 4 with scale 0"));
+
+    test(query);
+  }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java
index b74a8607071..4087e675931 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java
@@ -113,7 +113,7 @@ public void testDecimalDivide() throws Exception {
             "/ cast('-1.789' as DECIMAL(4, 3)) as s2,\n" +
             "cast('15.02' as DECIMAL(4, 2)) / cast('15.02' as DECIMAL(4, 2)) as s3,\n" +
             "cast('12.93123456789' as DECIMAL(13, 11)) / cast('1' as DECIMAL(1, 0)) as s4,\n" +
-            "cast('0' as DECIMAL(0, 0)) / cast('15.02' as DECIMAL(4, 2)) as s5";
+            "cast('0' as DECIMAL(1, 0)) / cast('15.02' as DECIMAL(4, 2)) as s5";
     testBuilder()
         .sqlQuery(query)
         .ordered()
@@ -297,8 +297,8 @@ public void testDecimalEquals() throws Exception {
             "<> cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s2,\n" +
             // the same value but different scale and precision
             "cast('1234567.89' as DECIMAL(9, 2)) = cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
-            "cast('0' as DECIMAL(4, 2)) = cast('0' as DECIMAL(0, 0)) as s4,\n" +
-            "cast('0' as DECIMAL(4, 2)) <> cast('0' as DECIMAL(0, 0)) as s5,\n" +
+            "cast('0' as DECIMAL(4, 2)) = cast('0' as DECIMAL(1, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(4, 2)) <> cast('0' as DECIMAL(1, 0)) as s5,\n" +
             "cast('12.93123456789' as DECIMAL(13, 11)) = cast('12.93123456788' as DECIMAL(13, 11)) as s6";
     testBuilder()
         .sqlQuery(query)
@@ -318,7 +318,7 @@ public void testDecimalLessThan() throws Exception {
             "< cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
             // the same value but different scale and precision
             "cast('1234567.89' as DECIMAL(9, 2)) < cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
-            "cast('0' as DECIMAL(4, 2)) < cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(4, 2)) < cast('0' as DECIMAL(1, 0)) as s4,\n" +
             "cast('12.93123456789' as DECIMAL(13, 11)) < cast('12.93123456788' as DECIMAL(13, 11)) as s5";
     testBuilder()
         .sqlQuery(query)
@@ -338,7 +338,7 @@ public void testDecimalLessThanEquals() throws Exception {
             "<= cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
             // the same value but different scale and precision
             "cast('1234567.89' as DECIMAL(9, 2)) <= cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
-            "cast('0' as DECIMAL(4, 2)) <= cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(4, 2)) <= cast('0' as DECIMAL(1, 0)) as s4,\n" +
             "cast('12.93123456789' as DECIMAL(13, 11)) <= cast('12.93123456788' as DECIMAL(13, 11)) as s5";
     testBuilder()
         .sqlQuery(query)
@@ -358,7 +358,7 @@ public void testDecimalGreaterThan() throws Exception {
             "> cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
             // the same value but different scale and precision
             "cast('1234567.89' as DECIMAL(9, 2)) > cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
-            "cast('0' as DECIMAL(4, 2)) > cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(4, 2)) > cast('0' as DECIMAL(1, 0)) as s4,\n" +
             "cast('12.93123456789' as DECIMAL(13, 11)) > cast('12.93123456788' as DECIMAL(13, 11)) as s5";
     testBuilder()
         .sqlQuery(query)
@@ -378,7 +378,7 @@ public void testDecimalGreaterThanEquals() throws Exception {
             ">= cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
             // the same value but different scale and precision
             "cast('1234567.89' as DECIMAL(9, 2)) >= cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
-            "cast('0' as DECIMAL(4, 2)) >= cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(4, 2)) >= cast('0' as DECIMAL(1, 0)) as s4,\n" +
             "cast('12.93123456789' as DECIMAL(13, 11)) >= cast('12.93123456788' as DECIMAL(13, 11)) as s5";
     testBuilder()
         .sqlQuery(query)
@@ -398,8 +398,8 @@ public void testDecimalCompareToNullsHigh() throws Exception {
             "cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37))) as s2,\n" +
             // the same value but different scale and precision
             "compare_to_nulls_high(cast('1234567.89' as DECIMAL(9, 2)), cast('1234567.890' as DECIMAL(10, 3))) as s3,\n" +
-            "compare_to_nulls_high(cast('0' as DECIMAL(4, 2)), cast('0' as DECIMAL(0, 0))) as s4,\n" +
-            "compare_to_nulls_high(cast('0' as DECIMAL(4, 2)), cast(null as DECIMAL(0, 0))) as s5,\n" +
+            "compare_to_nulls_high(cast('0' as DECIMAL(4, 2)), cast('0' as DECIMAL(1, 0))) as s4,\n" +
+            "compare_to_nulls_high(cast('0' as DECIMAL(4, 2)), cast(null as DECIMAL(1, 0))) as s5,\n" +
             "compare_to_nulls_high(cast('12.93123456789' as DECIMAL(13, 11)), " +
             "cast('12.93123456788' as DECIMAL(13, 11))) as s6";
     testBuilder()
@@ -420,8 +420,8 @@ public void testDecimalCompareToNullsLow() throws Exception {
             "cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37))) as s2,\n" +
             // the same value but different scale and precision
             "compare_to_nulls_low(cast('1234567.89' as DECIMAL(9, 2)), cast('1234567.890' as DECIMAL(10, 3))) as s3,\n" +
-            "compare_to_nulls_low(cast('0' as DECIMAL(4, 2)), cast('0' as DECIMAL(0, 0))) as s4,\n" +
-            "compare_to_nulls_low(cast('0' as DECIMAL(4, 2)), cast(null as DECIMAL(0, 0))) as s5,\n" +
+            "compare_to_nulls_low(cast('0' as DECIMAL(4, 2)), cast('0' as DECIMAL(1, 0))) as s4,\n" +
+            "compare_to_nulls_low(cast('0' as DECIMAL(4, 2)), cast(null as DECIMAL(1, 0))) as s5,\n" +
             "compare_to_nulls_low(cast('12.93123456789' as DECIMAL(13, 11)), " +
             "cast('12.93123456788' as DECIMAL(13, 11))) as s6";
     testBuilder()
@@ -571,8 +571,7 @@ public void testCastFloatDecimal() throws Exception {
             "cast(i3 as DECIMAL(8, 7)) as s3,\n" +
             "cast(i4 as DECIMAL(6, 6)) as s4,\n" +
             "cast(i5 as DECIMAL(7, 0)) as s5,\n" +
-            "cast(i6 as DECIMAL(7, 46)) as s6,\n" +
-            "cast(i7 as DECIMAL(17, 0)) as s7\n" +
+            "cast(i6 as DECIMAL(38, 38)) as s6\n" +
         "from (" +
             "select\n" +
                 "cast(0 as float) as i1,\n" +
@@ -580,17 +579,15 @@ public void testCastFloatDecimal() throws Exception {
                 "cast(-1.5022222 as float) as i3,\n" +
                 "cast(-0.987654 as float) as i4,\n" +
                 "cast(9999999 as float) as i5,\n" +
-                "cast('%s' as float) as i6,\n" +
-                "cast('%s' as float) as i7)";
+                "cast('%s' as float) as i6)";
 
     testBuilder()
-        .sqlQuery(query, Float.MIN_VALUE, Float.MAX_VALUE)
-        .ordered()
-        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .sqlQuery(query, Float.MIN_VALUE)
+        .unOrdered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
         .baselineValues(BigDecimal.valueOf(0), new BigDecimal("1.234567"),
             new BigDecimal("-1.5022222"), new BigDecimal("-0.987654"), BigDecimal.valueOf(9999999),
-            new BigDecimal(String.format("%s", Float.MIN_VALUE)),
-            new BigDecimal("340282350000000000000000000000000000000")) // Float.MAX_VALUE in non-scientific format
+            new BigDecimal(Float.MIN_VALUE).setScale(38, RoundingMode.HALF_UP))
         .go();
   }
 
@@ -603,8 +600,7 @@ public void testCastDecimalFloat() throws Exception {
             "cast(i3 as float) as s3,\n" +
             "cast(i4 as float) as s4,\n" +
             "cast(i5 as float) as s5,\n" +
-            "cast(i6 as float) as s6,\n" +
-            "cast(i7 as float) as s7\n" +
+            "cast(i6 as float) as s6\n" +
         "from (" +
             "select\n" +
                 "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
@@ -612,17 +608,16 @@ public void testCastDecimalFloat() throws Exception {
                 "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
                 "cast('0' as DECIMAL(36, 3)) as i4,\n" +
                 "cast('15.02' as DECIMAL(4, 2)) as i5,\n" +
-                "cast('%s' as DECIMAL(2, 46)) as i6,\n" +
-                "cast('%s' as DECIMAL(8, 0)) as i7)";
+                "cast('%s' as DECIMAL(38, 38)) as i6)";
 
     testBuilder()
-        .sqlQuery(query, Float.MIN_VALUE, Float.MAX_VALUE)
-        .ordered()
-        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .sqlQuery(query, Float.MIN_VALUE)
+        .unOrdered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
         .baselineValues(new BigDecimal("999999999999999999999999999.92345678912").floatValue(),
             new BigDecimal("0.32345678912345678912345678912345678912").floatValue(),
             new BigDecimal("-1234567891234567891234567891234567.89").floatValue(),
-            0f, 15.02f, Float.MIN_VALUE, Float.MAX_VALUE)
+            0f, 15.02f, 0.0f)
         .go();
   }
 
@@ -635,8 +630,7 @@ public void testCastDoubleDecimal() throws Exception {
             "cast(i3 as DECIMAL(8, 7)) as s3,\n" +
             "cast(i4 as DECIMAL(6, 6)) as s4,\n" +
             "cast(i5 as DECIMAL(7, 0)) as s5,\n" +
-            "cast(i6 as DECIMAL(17, 325)) as s6,\n" +
-            "cast(i7 as DECIMAL(17, 0)) as s7\n" +
+            "cast(i6 as DECIMAL(38, 38)) as s6\n" +
         "from (" +
             "select\n" +
                 "cast(0 as double) as i1,\n" +
@@ -644,17 +638,15 @@ public void testCastDoubleDecimal() throws Exception {
                 "cast(-1.5022222 as double) as i3,\n" +
                 "cast(-0.987654 as double) as i4,\n" +
                 "cast(9999999 as double) as i5,\n" +
-                "cast('%e' as double) as i6,\n" +
-                "cast('%f' as double) as i7)";
+                "cast('%e' as double) as i6)";
 
     testBuilder()
-        .sqlQuery(query, Double.MIN_VALUE, Double.MAX_VALUE)
-        .ordered()
-        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .sqlQuery(query, Double.MIN_VALUE)
+        .unOrdered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
         .baselineValues(BigDecimal.valueOf(0), new BigDecimal("1.234567"),
             new BigDecimal("-1.5022222"), new BigDecimal("-0.987654"), BigDecimal.valueOf(9999999),
-            new BigDecimal(String.valueOf(Double.MIN_VALUE)),
-            new BigDecimal(String.format("%1.0f", Double.MAX_VALUE))) // non-scientific format
+            new BigDecimal(String.valueOf(Double.MIN_VALUE)).setScale(38, RoundingMode.HALF_UP))
         .go();
   }
 
@@ -667,8 +659,7 @@ public void testCastDecimalDouble() throws Exception {
             "cast(i3 as double) as s3,\n" +
             "cast(i4 as double) as s4,\n" +
             "cast(i5 as double) as s5,\n" +
-            "cast(i6 as double) as s6,\n" +
-            "cast(i7 as double) as s7\n" +
+            "cast(i6 as double) as s6\n" +
         "from (" +
             "select\n" +
                 "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
@@ -676,17 +667,16 @@ public void testCastDecimalDouble() throws Exception {
                 "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
                 "cast('0' as DECIMAL(36, 3)) as i4,\n" +
                 "cast('15.02' as DECIMAL(4, 2)) as i5,\n" +
-                "cast('%e' as DECIMAL(17, 325)) as i6,\n" +
-                "cast('%f' as DECIMAL(17, 0)) as i7)";
+                "cast('%e' as DECIMAL(38, 38)) as i6)";
 
     testBuilder()
-        .sqlQuery(query, Double.MIN_VALUE, Double.MAX_VALUE)
-        .ordered()
-        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .sqlQuery(query, Double.MIN_VALUE)
+        .unOrdered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
         .baselineValues(new BigDecimal("999999999999999999999999999.92345678912").doubleValue(),
             new BigDecimal("0.32345678912345678912345678912345678912").doubleValue(),
             new BigDecimal("-1234567891234567891234567891234567.89").doubleValue(),
-            0d, 15.02, Double.MIN_VALUE, Double.MAX_VALUE)
+            0d, 15.02, 0.)
         .go();
   }
 
@@ -740,4 +730,32 @@ public void testDecimalToChar() throws Exception {
             "-1234567891234567891234567891234567.9", "0", "15")
         .go();
   }
+
+  @Test
+  public void testDecimalNegate() throws Exception {
+    String query =
+        "select\n" +
+            "negative(i1) as s1,\n" +
+            "-i2 as s2,\n" +
+            "negative(i3) as s3,\n" +
+            "-i4 as s4,\n" +
+            "negative(i5) as s5\n" +
+            "from (" +
+            "select\n" +
+            "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+            "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+            "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+            "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+            "cast('15.02' as DECIMAL(4, 2)) as i5)";
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("-999999999999999999999999999.92345678912"),
+            new BigDecimal("-0.32345678912345678912345678912345678912"),
+            new BigDecimal("1234567891234567891234567891234567.89"),
+            new BigDecimal("0.000"),
+            new BigDecimal("-15.02"))
+        .go();
+  }
 }
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
index 27df35aea2f..4e8b7008f07 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
@@ -19,7 +19,7 @@
       "pop" : "project",
       "@id" : 2,
       "exprs" : [
-        { "ref" : "DEC9_COL", "expr": "(cast(cast(INT_COL as int) as vardecimal(9, 0)))" },
+        { "ref" : "DEC9_COL", "expr": "(cast(cast(INT_COL as int) as vardecimal(10, 0)))" },
         { "ref" : "DEC38_COL", "expr": "(cast(BIGINT_COL as vardecimal(38, 0)))" }
       ],
 
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
index 92c8237b426..4f791945fbf 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
@@ -21,8 +21,8 @@
       "exprs" : [
         { "ref" : "DEC28", "expr": "(cast(B as vardecimal(38, 20)))" },
         { "ref" : "DEC38", "expr": "(cast(A as vardecimal(28, 16)))" },
-        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(18, 9)))" },
-        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(9, 0)))" }
+        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(19, 0)))" },
+        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(10, 0)))" }
       ],
 
       "child" : 4
@@ -32,8 +32,8 @@
       "exprs" : [
         {"ref": "DEC28_COL", "expr" : "cast(DEC28 as decimal38sparse(38, 20))"},
         {"ref": "DEC38_COL", "expr" : "cast(DEC38 as decimal28sparse(28, 16))"},
-        {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(18, 0))"},
-        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(9, 0))"}
+        {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(19, 0))"},
+        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(10, 0))"}
       ],
 
       "child" : 3
diff --git a/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
index 2f0999289a5..0ea70764a12 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
@@ -29,9 +29,9 @@
       "pop" : "project",
       "@id" : 4,
       "exprs" : [
-        {"ref": "DEC9_INT", "expr" : "cast(INT_COL as vardecimal(9, 0))"},
+        {"ref": "DEC9_INT", "expr" : "cast(INT_COL as vardecimal(10, 0))"},
         {"ref": "DEC38_INT", "expr" : "cast(INT_COL as vardecimal(38, 0))"},
-        {"ref": "DEC9_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(9, 0))"},
+        {"ref": "DEC9_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(19, 0))"},
         {"ref": "DEC38_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(38, 0))"}
       ],
 
diff --git a/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
index 4a4c5c90f68..1b5252d2a94 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
@@ -21,8 +21,8 @@
       "exprs" : [
         { "ref" : "DEC28", "expr": "(cast(B as vardecimal(38, 16)))" },
         { "ref" : "DEC38", "expr": "(cast(A as vardecimal(28, 16)))" },
-        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(18, 9)))" },
-        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(9, 0)))" }
+        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(18, 0)))" },
+        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(10, 0)))" }
       ],
 
       "child" : 1
@@ -34,7 +34,7 @@
         {"ref": "DEC28_COL", "expr" : "cast(DEC28 as decimal38sparse(38, 16))"},
         {"ref": "DEC38_COL", "expr" : "cast(DEC38 as decimal28sparse(28, 16))"},
         {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(18, 0))"},
-        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(9, 0))"}
+        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(10, 0))"}
       ],
 
       "child" : 2
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
index 16ff0028d4b..5f2dade3691 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
@@ -26,7 +26,10 @@
 import java.math.BigInteger;
 import java.math.RoundingMode;
 
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @SuppressWarnings("WeakerAccess")
 public class DecimalUtility {
@@ -37,6 +40,8 @@
   public final static int DIGITS_BASE = 1000000000;
   public final static int INTEGER_SIZE = Integer.SIZE / 8;
 
+  private static final Logger logger = LoggerFactory.getLogger(DecimalUtility.class);
+
   /**
    * Given the number of actual digits this function returns the
    * number of indexes it will occupy in the array of integers
@@ -417,4 +422,22 @@ public static int getDefaultPrecision(TypeProtos.MinorType minorType, int defaul
         return defaultPrecision;
     }
   }
+
+  /**
+   * Checks that the specified value may be fit into the value with specified
+   * {@code desiredPrecision} precision and {@code desiredScale} scale.
+   * Otherwise, the exception is thrown.
+   *
+   * @param value            BigDecimal value to check
+   * @param desiredPrecision precision for the resulting value
+   * @param desiredScale     scale for the resulting value
+   */
+  public static void checkValueOverflow(BigDecimal value, int desiredPrecision, int desiredScale) {
+    if (value.precision() - value.scale() > desiredPrecision - desiredScale) {
+      throw UserException.validationError()
+          .message("Value %s overflows specified precision %s with scale %s.",
+              value, desiredPrecision, desiredScale)
+          .build(logger);
+    }
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services