You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2014/06/07 02:16:16 UTC

[1/4] git commit: DRILL-918: During Sort, handle the case where the child SelectionVector produces 0 records.

Repository: incubator-drill
Updated Branches:
  refs/heads/master 3db1d5a32 -> cca3cec18


DRILL-918: During Sort, handle the case where the child SelectionVector produces 0 records.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/5746032b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/5746032b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/5746032b

Branch: refs/heads/master
Commit: 5746032b78438e4c772ed2c6da6c7f7b2e70c868
Parents: 3db1d5a
Author: Aman Sinha <as...@maprtech.com>
Authored: Thu Jun 5 18:58:43 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Jun 5 19:54:42 2014 -0700

----------------------------------------------------------------------
 .../exec/physical/impl/xsort/ExternalSortBatch.java      | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5746032b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index d6cbbc4..7a2b251 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -185,6 +185,8 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       }
     }
 
+    long totalcount = 0;
+    
     try{
       outer: while (true) {
         Stopwatch watch = new Stopwatch();
@@ -218,12 +220,15 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
             }
           }
           int count = sv2.getCount();
-          assert sv2.getCount() > 0;
+          totalcount += count;
+          if (count == 0) {
+            break outer;
+          }
           sorter.setup(context, sv2, incoming);
           Stopwatch w = new Stopwatch();
           w.start();
           sorter.sort(sv2);
-//          logger.debug("Took {} us to sort {} records", w.elapsed(TimeUnit.MICROSECONDS), sv2.getCount());
+//          logger.debug("Took {} us to sort {} records", w.elapsed(TimeUnit.MICROSECONDS), count);
           RecordBatchData rbd = new RecordBatchData(incoming);
           if (incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.NONE) {
             rbd.setSv2(sv2);
@@ -246,7 +251,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         }
       }
 
-      if (schema == null){
+      if (schema == null || totalcount == 0){
         // builder may be null at this point if the first incoming batch is empty
         return IterOutcome.NONE;
       }


[4/4] git commit: DRILL-924: Update trim so that it synchronized within the arena to avoid inconsistent state.

Posted by ja...@apache.org.
DRILL-924: Update trim so that it synchronized within the arena to avoid inconsistent state.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/cca3cec1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/cca3cec1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/cca3cec1

Branch: refs/heads/master
Commit: cca3cec18914f7b25481f57ad2fe422a8ec3d475
Parents: e830f7f
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri Jun 6 10:52:01 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jun 6 17:11:41 2014 -0700

----------------------------------------------------------------------
 .../java/io/netty/buffer/PoolChunkListL.java    | 49 ++++++++++----------
 1 file changed, 25 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cca3cec1/exec/bufferl/src/main/java/io/netty/buffer/PoolChunkListL.java
----------------------------------------------------------------------
diff --git a/exec/bufferl/src/main/java/io/netty/buffer/PoolChunkListL.java b/exec/bufferl/src/main/java/io/netty/buffer/PoolChunkListL.java
index 16f11fa..7e25557 100644
--- a/exec/bufferl/src/main/java/io/netty/buffer/PoolChunkListL.java
+++ b/exec/bufferl/src/main/java/io/netty/buffer/PoolChunkListL.java
@@ -61,7 +61,7 @@ final class PoolChunkListL<T> {
      * @return
      */
     boolean allocate(PooledByteBufL<T> buf, int minRequested, int maxRequested) {
-    	
+
     	// If list is empty, then allocation fails
         if (head == null) {
             return false;
@@ -69,7 +69,7 @@ final class PoolChunkListL<T> {
 
         // Do for each chunk in the list
         for (PoolChunkL<T> cur = head;;) {
-        	
+
         	// If we successfully allocated from the chunk ...
             long handle = cur.allocate(minRequested, maxRequested);
             if (handle < 0) {
@@ -77,11 +77,11 @@ final class PoolChunkListL<T> {
                 if (cur == null) {
                     return false;
                 }
-                
+
             // ... then add the memory to the buffer container
             } else {
                 cur.initBuf(buf, handle, minRequested, maxRequested);
-                
+
                 // If usage changed, then move to next list
                 if (cur.usage() >= maxUsage) {
                     remove(cur);
@@ -92,17 +92,17 @@ final class PoolChunkListL<T> {
         }
     }
 
-    
+
     /**
      * Release a buffer back to the original chunk.
      * @param chunk
      * @param handle
      */
     void free(PoolChunkL<T> chunk, long handle) {
-    	
+
     	// Release buffer back to the original chunk
         chunk.free(handle);
-        
+
         // If usage changed, then move to different list
         if (chunk.usage() < minUsage) {
             remove(chunk);
@@ -114,7 +114,7 @@ final class PoolChunkListL<T> {
             }
         }
     }
-    
+
     /**
      * Shrink the buffer down to the specified size, freeing up unused memory.
      * @param chunk - chunk the buffer resides in
@@ -123,29 +123,30 @@ final class PoolChunkListL<T> {
      * @return a new handle to the smaller buffer
      */
     long trim(PoolChunkL<T> chunk, long handle, int size) {
-    	
-    	// Trim the buffer, possibly getting a new handle.
-    	handle = chunk.trim(handle,  size);
-    	if (handle == -1) return handle;
-    	
-    	// Move the chunk to a different list if usage changed significantly
-    	if (chunk.usage() < minUsage) {
-    		assert chunk.usage() > 0 && prevList != null;
-    		remove(chunk);
-    		prevList.add(chunk);
+    	synchronized(arena){
+        // Trim the buffer, possibly getting a new handle.
+        handle = chunk.trim(handle,  size);
+        if (handle == -1) return handle;
+
+        // Move the chunk to a different list if usage changed significantly
+        if (chunk.usage() < minUsage) {
+          assert chunk.usage() > 0 && prevList != null;
+          remove(chunk);
+          prevList.add(chunk);
+        }
+
+        // return new handle for the smaller buffer
+        return handle;
     	}
-    	
-    	// return new handle for the smaller buffer
-    	return handle;
     }
 
-    
+
     /**
      * Add a chunk to the current chunklist
      * @param chunk
      */
     void add(PoolChunkL<T> chunk) {
-    	
+
     	// If usage has change, then add to the neighboring list instead
     	int usage = chunk.usage();
         if (usage >= maxUsage) {
@@ -170,7 +171,7 @@ final class PoolChunkListL<T> {
         }
     }
 
-    
+
     /**
      * Remove a chunk from the current linked list of chunks
      * @param cur


[3/4] git commit: DRILL-913: Fix IOOB in HashJoinProbe when input is empty batch

Posted by ja...@apache.org.
DRILL-913: Fix IOOB in HashJoinProbe when input is empty batch


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/e830f7fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/e830f7fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/e830f7fa

Branch: refs/heads/master
Commit: e830f7faa1c36fd0915146fea771009c3d4f90b3
Parents: aad2c4f
Author: Mehant Baid <me...@gmail.com>
Authored: Thu Jun 5 12:10:11 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Jun 5 20:08:26 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/physical/impl/join/HashJoinProbeTemplate.java     | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e830f7fa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
index 6e19934..2dec9ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
@@ -126,6 +126,10 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
           case OK:
             recordsToProcess = probeBatch.getRecordCount();
             recordsProcessed = 0;
+            // If we received an empty batch do nothing
+            if (recordsToProcess == 0) {
+              continue;
+            }
         }
       }
       int probeIndex = -1;


[2/4] git commit: DRILL-799: Use static constants instead of Math.pow() to compute powers of 10 while adjusting scale of decimal data type.

Posted by ja...@apache.org.
DRILL-799: Use static constants instead of Math.pow() to compute powers of 10 while adjusting scale of decimal data type.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/aad2c4ff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/aad2c4ff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/aad2c4ff

Branch: refs/heads/master
Commit: aad2c4ff29a5246b55b1640a29e04fc51066a63b
Parents: 5746032
Author: Mehant Baid <me...@gmail.com>
Authored: Thu Jun 5 09:23:28 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Jun 5 20:08:19 2014 -0700

----------------------------------------------------------------------
 .../drill/common/util/DecimalUtility.java       | 54 ++++++++++++++++++++
 exec/java-exec/src/main/codegen/data/Casts.tdd  |  4 +-
 .../templates/Decimal/CastDecimalFloat.java     |  2 +-
 .../templates/Decimal/CastDecimalInt.java       |  2 +-
 .../templates/Decimal/CastDecimalVarchar.java   |  4 +-
 .../templates/Decimal/CastIntDecimal.java       |  2 +-
 .../templates/Decimal/CastSrcDecimalSimple.java |  6 +--
 .../templates/Decimal/CastVarCharDecimal.java   |  6 +--
 .../templates/Decimal/DecimalFunctions.java     | 29 ++++++-----
 9 files changed, 82 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
index ff8725f..ae2af54 100644
--- a/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
+++ b/common/src/main/java/org/apache/drill/common/util/DecimalUtility.java
@@ -45,6 +45,60 @@ public class DecimalUtility {
             "00000000",
             "000000000"};
 
+    public final static long[] scale_long_constants = {
+        1,
+        10,
+        100,
+        1000,
+        10000,
+        100000,
+        1000000,
+        10000000,
+        100000000,
+        1000000000,
+        10000000000l,
+        100000000000l,
+        1000000000000l,
+        10000000000000l,
+        100000000000000l,
+        1000000000000000l,
+        10000000000000000l,
+        100000000000000000l,
+        1000000000000000000l};
+
+    /*
+     * Simple function that returns the static precomputed
+     * power of ten, instead of using Math.pow
+     */
+    public static long getPowerOfTen(int power) {
+      assert power >= 0 && power < scale_long_constants.length;
+      return scale_long_constants[(power)];
+    }
+
+    /*
+     * Math.pow returns a double and while multiplying with large digits
+     * in the decimal data type we encounter noise. So instead of multiplying
+     * with Math.pow we use the static constants to perform the multiplication
+     */
+    public static long adjustScaleMultiply(long input, int factor) {
+      int index = Math.abs(factor);
+      assert index >= 0 && index < scale_long_constants.length;
+      if (factor >= 0) {
+        return input * scale_long_constants[index];
+      } else {
+        return input / scale_long_constants[index];
+      }
+    }
+
+    public static long adjustScaleDivide(long input, int factor) {
+      int index = Math.abs(factor);
+      assert index >= 0 && index < scale_long_constants.length;
+      if (factor >= 0) {
+        return input / scale_long_constants[index];
+      } else {
+        return input * scale_long_constants[index];
+      }
+    }
 
     /* Given the number of actual digits this function returns the
      * number of indexes it will occupy in the array of integers

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/data/Casts.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/Casts.tdd b/exec/java-exec/src/main/codegen/data/Casts.tdd
index 73cdede..e49df93 100644
--- a/exec/java-exec/src/main/codegen/data/Casts.tdd
+++ b/exec/java-exec/src/main/codegen/data/Casts.tdd
@@ -147,8 +147,8 @@
     {from: "Decimal38Sparse", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
     {from: "Decimal38Dense", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
 
-    {from: "VarChar", to: "Decimal9", major: "VarCharDecimalSimple"},
-    {from: "VarChar", to: "Decimal18", major: "VarCharDecimalSimple"},
+    {from: "VarChar", to: "Decimal9", major: "VarCharDecimalSimple", javatype: "int"},
+    {from: "VarChar", to: "Decimal18", major: "VarCharDecimalSimple", javatype: "long"},
     {from: "VarChar", to: "Decimal28Sparse", major: "VarCharDecimalComplex", arraySize: "5"},
     {from: "VarChar", to: "Decimal38Sparse", major: "VarCharDecimalComplex", arraySize: "6"},
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
index 9e2e48d..b259cb3 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
@@ -52,7 +52,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
     public void eval() {
 
         // Divide the decimal with the scale to get the floating point value
-        out.value = (${type.javatype}) (in.value / Math.pow(10, in.scale));
+        out.value = (${type.javatype}) (org.apache.drill.common.util.DecimalUtility.adjustScaleDivide(in.value, (int) in.scale));
     }
 }
 <#elseif type.major == "DecimalComplexFloat" || type.major == "DecimalComplexDouble"> <#-- Cast function template for conversion from Decimal9, Decimal18 to Float4 -->

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
index 646f52c..a89965f 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
@@ -53,7 +53,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
     public void eval() {
 
         // Assign the integer part of the decimal to the output holder
-        out.value = (${type.javatype}) ((in.value / Math.pow(10, in.scale)));
+        out.value = (${type.javatype}) (org.apache.drill.common.util.DecimalUtility.adjustScaleDivide(in.value, (int) in.scale));
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
index 52e5513..783165f 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
@@ -67,7 +67,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
             in.value *= -1;
         }
 
-        ${type.javatype} separator = (${type.javatype}) Math.pow(10, in.scale);
+        ${type.javatype} separator = (${type.javatype}) org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) in.scale);
 
         str.append(in.value / separator);
 
@@ -192,7 +192,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
             if (actualDigits != 0) {
 
                 // Strip padded zeroes at the end that is not part of the scale
-                lastFractionalDigit /= (int) (Math.pow(10, org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - actualDigits));
+                lastFractionalDigit /= (int) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) (org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - actualDigits)));
                 str.append(org.apache.drill.common.util.DecimalUtility.toStringWithZeroes(lastFractionalDigit, actualDigits));
             } else {
                 // Last digit does not have any padding print as is

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
----------------------------------------------------------------------
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 525566a..979e7e2 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
@@ -66,7 +66,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
         out.value = (${type.javatype}) in.value;
 
         // converting from integer to decimal, pad zeroes if scale is non zero
-        out.value = (${type.javatype}) (out.value * Math.pow(10, scale.value));
+        out.value = (${type.javatype}) org.apache.drill.common.util.DecimalUtility.adjustScaleMultiply(out.value, (int) scale.value);
 
         <#else>
         out.start = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
index 36a6ef3..b298c66 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
@@ -189,10 +189,10 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc{
             if (power == 0) {
                 power = 9;
             } else {
-                padding = (int) (Math.pow(10, (org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - power)));
+                padding = (int) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) (org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - power)));
             }
 
-            int mask = (int) Math.pow(10, power);
+            int mask = (int) org.apache.drill.common.util.DecimalUtility.getPowerOfTen(power);
 
             out.setInteger(index, (int) ((value % mask) * padding));
 
@@ -254,7 +254,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
         out.value = in.value;
 
         // Truncate or pad additional zeroes if the output scale is different from input scale
-        out.value = (${type.javatype}) (out.value * ((int) Math.pow(10, (out.scale - in.scale))));
+        out.value = (${type.javatype}) (org.apache.drill.common.util.DecimalUtility.adjustScaleMultiply(out.value, (int) (out.scale - in.scale)));
     }
 }
 </#if>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
----------------------------------------------------------------------
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 8441298..9ca0533 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -134,7 +134,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
 
         // Pad the number with zeroes if number of fractional digits is less than scale
         if (fractionalDigits < scale.value) {
-            out.value *= Math.pow(10, scale.value - fractionalDigits);
+            out.value = (${type.javatype}) (org.apache.drill.common.util.DecimalUtility.adjustScaleMultiply(out.value, (int) (scale.value - fractionalDigits)));
         }
 
         // Negate the number if we saw a -ve sign
@@ -292,7 +292,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
 
             next = (byte) Character.digit(next, radix);
 
-            int value = (((int) Math.pow(10, ndigits)) * next) + (out.getInteger(decimalBufferIndex));
+            int value = (((int) org.apache.drill.common.util.DecimalUtility.getPowerOfTen(ndigits)) * next) + (out.getInteger(decimalBufferIndex));
             out.setInteger(decimalBufferIndex, value);
 
             ndigits++;
@@ -337,7 +337,7 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc {
                 ndigits++;
             }
             // Pad zeroes in the fractional part so that number of digits = MAX_DIGITS
-            int padding = (int) Math.pow(10, org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - ndigits);
+            int padding = (int) org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) (org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - ndigits));
             out.setInteger(decimalBufferIndex, out.getInteger(decimalBufferIndex) * padding);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/aad2c4ff/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
----------------------------------------------------------------------
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 cff122e..b294396 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
@@ -236,10 +236,10 @@ import org.apache.drill.exec.expr.annotations.Workspace;
             int adjustment = 0;
 
             if (left.scale < right.scale) {
-                left.value = (${javaType}) (left.value * Math.pow(10, (right.scale - left.scale)));
+                left.value = (${javaType}) (org.apache.drill.common.util.DecimalUtility.adjustScaleMultiply(left.value, (int) (right.scale - left.scale)));
                 left.scale = right.scale;
             } else if (right.scale < left.scale) {
-                right.value = (${javaType}) (right.value * Math.pow(10, (left.scale - right.scale)));
+                right.value = (${javaType}) (org.apache.drill.common.util.DecimalUtility.adjustScaleMultiply(right.value, (int) (left.scale - right.scale)));
                 right.scale = left.scale;
             }
 </#macro>
@@ -800,7 +800,7 @@ public class ${type.name}Functions {
             // We truncated the decimal digit. Now we need to truncate within the base 1 billion fractional digit
             int truncateFactor = org.apache.drill.common.util.DecimalUtility.MAX_DIGITS - (right.value % org.apache.drill.common.util.DecimalUtility.MAX_DIGITS);
             if (truncateFactor != org.apache.drill.common.util.DecimalUtility.MAX_DIGITS) {
-              truncateFactor = (int) Math.pow(10, truncateFactor);
+              truncateFactor = (int) org.apache.drill.common.util.DecimalUtility.getPowerOfTen(truncateFactor);
               int fractionalDigits = result.getInteger(${type.storage} - 1);
               fractionalDigits /= truncateFactor;
               result.setInteger(${type.storage} - 1, fractionalDigits * truncateFactor);
@@ -1305,7 +1305,7 @@ public class ${type.name}Functions {
 
         public void eval() {
 
-            out.value =(${type.storage}) (in.value / (Math.pow(10, in.scale)));
+            out.value =(${type.storage}) (org.apache.drill.common.util.DecimalUtility.adjustScaleDivide(in.value, (int) in.scale));
             out.precision = out.maxPrecision;
             out.scale = 0;
         }
@@ -1322,7 +1322,7 @@ public class ${type.name}Functions {
 
         public void eval() {
 
-            out.value = (${type.storage}) (left.value / ( Math.pow(10, (left.scale - right.value))));
+            out.value = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.adjustScaleDivide(left.value, (int) (left.scale - right.value)));
             out.precision = out.maxPrecision;
             out.scale = right.value;
         }
@@ -1339,7 +1339,7 @@ public class ${type.name}Functions {
         }
 
         public void eval() {
-          ${type.storage} scaleFactor = (${type.storage}) (Math.pow(10, in.scale));
+          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) in.scale));
 
           // Get the integer part
           ${type.storage} integerPart = in.value / scaleFactor;
@@ -1366,7 +1366,7 @@ public class ${type.name}Functions {
 
         public void eval() {
 
-          ${type.storage} scaleFactor = (${type.storage}) (Math.pow(10, in.scale));
+          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) in.scale));
           out.scale = 0;
           out.value = (in.value / scaleFactor);
 
@@ -1391,7 +1391,7 @@ public class ${type.name}Functions {
 
         public void eval() {
 
-          ${type.storage} scaleFactor = (${type.storage}) (Math.pow(10, in.scale));
+          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) in.scale));
           ${type.storage} extractDigit = scaleFactor / 10;
 
           out.scale = 0;
@@ -1426,13 +1426,14 @@ public class ${type.name}Functions {
 
         public void eval() {
 
-          ${type.storage} scaleFactor = (${type.storage}) (Math.pow(10, left.scale));
-          ${type.storage} newScaleFactor = (${type.storage}) (Math.pow(10, right.value));
-          ${type.storage} truncScaleFactor = (${type.storage}) (Math.pow(10, left.scale - right.value));
+          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) left.scale));
+          ${type.storage} newScaleFactor = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen((int) right.value));
+          ${type.storage} truncScaleFactor = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.getPowerOfTen( Math.abs(left.scale - right.value)));
+          int truncFactor = (int) (left.scale - right.value);
 
           // If rounding scale is >= current scale
           if (right.value >= left.scale) {
-            out.value = (${type.storage}) (left.value * (int) Math.pow(10, (right.value - left.scale)));
+            out.value = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.adjustScaleMultiply(left.value, (int) (right.value - left.scale)));
           }
           else {
             out.scale = right.value;
@@ -1443,12 +1444,12 @@ public class ${type.name}Functions {
             ${type.storage} fractionalPart = left.value % scaleFactor;
 
             // From the entire fractional part extract the digits upto which rounding is needed
-            ${type.storage} newFractionalPart = fractionalPart / truncScaleFactor;
+            ${type.storage} newFractionalPart = (${type.storage}) (org.apache.drill.common.util.DecimalUtility.adjustScaleDivide(fractionalPart, truncFactor));
             ${type.storage} truncatedFraction = fractionalPart % truncScaleFactor;
 
 
             // Get the truncated fractional part and extract the first digit to see if we need to add 1
-            int digit = Math.abs((int) (truncatedFraction / (truncScaleFactor / 10)));
+            int digit = Math.abs((int) org.apache.drill.common.util.DecimalUtility.adjustScaleDivide(truncatedFraction, truncFactor - 1));
 
             if (digit > 4) {
               if (left.value > 0) {