You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2014/05/10 02:34:57 UTC

[2/5] DRILL-619: Implement Decimal functions abs ceil floor round truncate divide modulo Fix resolver type precedence Fix bug in multiplication of large decimals Add tests

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java
new file mode 100644
index 0000000..a5de0cd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalAggFuncHolder.java
@@ -0,0 +1,46 @@
+/**
+ * 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.drill.exec.expr.fn;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+
+import java.util.List;
+import java.util.Map;
+
+public class DrillDecimalAggFuncHolder extends DrillAggFuncHolder {
+  public DrillDecimalAggFuncHolder(FunctionTemplate.FunctionScope scope, FunctionTemplate.NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom, String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, Map<String, String> methods, List<String> imports) {
+    super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports);
+  }
+
+  @Override
+  public TypeProtos.MajorType getReturnType(List<LogicalExpression> args) {
+
+    int scale = 0;
+    int precision = 0;
+
+    // Get the max scale and precision from the inputs
+    for (LogicalExpression e : args) {
+      scale = Math.max(scale, e.getMajorType().getScale());
+      precision = Math.max(precision, e.getMajorType().getPrecision());
+    }
+
+    return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(scale).setPrecision(precision).setMode(TypeProtos.DataMode.REQUIRED).build());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java
new file mode 100644
index 0000000..af239e4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalDivScaleFuncHolder.java
@@ -0,0 +1,64 @@
+/**
+ * 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.drill.exec.expr.fn;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+public class DrillDecimalDivScaleFuncHolder extends DrillSimpleFuncHolder{
+
+
+  public DrillDecimalDivScaleFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom,
+                                        String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
+                                        Map<String, String> methods, List<String> imports) {
+    super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports);
+  }
+
+  @Override
+  public MajorType getReturnType(List<LogicalExpression> args) {
+
+    TypeProtos.DataMode mode = returnValue.type.getMode();
+    int scale = 0;
+    int precision = 0;
+
+    if (nullHandling == NullHandling.NULL_IF_NULL) {
+      // if any one of the input types is nullable, then return nullable return type
+      for (LogicalExpression e : args) {
+        if (e.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) {
+          mode = TypeProtos.DataMode.OPTIONAL;
+          break;
+        }
+      }
+    }
+
+    /* Set the scale to be the same as the fist input's scale
+     * Used by divide and modulo functions
+     */
+    scale = args.get(0).getMajorType().getScale();
+    precision = args.get(0).getMajorType().getPrecision();
+
+    return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(scale).setPrecision(precision).setMode(mode).build());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java
new file mode 100644
index 0000000..b7df04f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalSetScaleFuncHolder.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+public class DrillDecimalSetScaleFuncHolder extends DrillSimpleFuncHolder{
+
+
+  public DrillDecimalSetScaleFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom,
+                                        String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
+                                        Map<String, String> methods, List<String> imports) {
+    super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports);
+  }
+
+  @Override
+  public MajorType getReturnType(List<LogicalExpression> args) {
+
+    TypeProtos.DataMode mode = returnValue.type.getMode();
+    int scale = 0;
+    int precision = 0;
+    int i = 0;
+
+    if (nullHandling == NullHandling.NULL_IF_NULL) {
+      // if any one of the input types is nullable, then return nullable return type
+      for (LogicalExpression e : args) {
+
+        precision = Math.max(precision, e.getMajorType().getPrecision());
+        if (e.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) {
+          mode = TypeProtos.DataMode.OPTIONAL;
+        }
+      }
+
+      /* Used by functions like round, truncate which specify the scale for
+       * the output as the second argument
+       */
+      assert (args.size() == 2) && (args.get(1) instanceof ValueExpressions.IntExpression);
+
+      // Get the scale from the second argument which should be a constant
+      scale = ((ValueExpressions.IntExpression) args.get(1)).getInt();
+    }
+
+    return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(scale).setPrecision(precision).setMode(mode).build());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java
new file mode 100644
index 0000000..2ee60bf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillDecimalZeroScaleFuncHolder.java
@@ -0,0 +1,60 @@
+/**
+ * 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.drill.exec.expr.fn;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+public class DrillDecimalZeroScaleFuncHolder extends DrillSimpleFuncHolder{
+
+
+  public DrillDecimalZeroScaleFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, boolean isRandom,
+                                        String[] registeredNames, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
+                                        Map<String, String> methods, List<String> imports) {
+    super(scope, nullHandling, isBinaryCommutative, isRandom, registeredNames, parameters, returnValue, workspaceVars, methods, imports);
+  }
+
+  /* This function scope is used when we need to remove the scale part.
+   * trunc and round functions with single argument use this
+   */
+  @Override
+  public MajorType getReturnType(List<LogicalExpression> args) {
+
+    int precision = 0;
+    TypeProtos.DataMode mode = returnValue.type.getMode();
+
+    if (nullHandling == NullHandling.NULL_IF_NULL) {
+      // if any one of the input types is nullable, then return nullable return type
+      for (LogicalExpression e : args) {
+        if (e.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) {
+          mode = TypeProtos.DataMode.OPTIONAL;
+        }
+        precision = Math.max(precision, e.getMajorType().getPrecision());
+      }
+    }
+
+    return (TypeProtos.MajorType.newBuilder().setMinorType(returnValue.type.getMinorType()).setScale(0).setPrecision(precision).setMode(mode).build());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
index 888829d..4421091 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
@@ -206,6 +206,9 @@ public class FunctionConverter {
       case POINT_AGGREGATE:
         return new DrillAggFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
           template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
+      case DECIMAL_AGGREGATE:
+        return new DrillDecimalAggFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
+          template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
       case SIMPLE:
         return new DrillSimpleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
           template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
@@ -218,6 +221,15 @@ public class FunctionConverter {
       case DECIMAL_CAST:
           return new DrillDecimalCastFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
                   template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
+      case DECIMAL_DIV_SCALE:
+          return new DrillDecimalDivScaleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
+                  template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
+      case DECIMAL_SET_SCALE:
+          return new DrillDecimalSetScaleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
+                  template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
+      case DECIMAL_ZERO_SCALE:
+          return new DrillDecimalZeroScaleFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(),
+                  template.isRandom(), registeredNames, ps, outputField, works, methods, imports);
       case HOLISTIC_AGGREGATE:
       case RANGE_AGGREGATE:
       default:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
index 59be472..7d5a3a6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -23,12 +23,18 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.record.RecordBatch;
 
+import java.math.BigDecimal;
+import java.text.DecimalFormat;
+import java.text.ParseException;
+
 public class MathFunctions{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
   
@@ -74,6 +80,36 @@ public class MathFunctions{
       out.value = java.lang.Math.random();
     }
 
-  }  
-  
+  }
+
+  @FunctionTemplate(name = "to_number", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class ToNumber implements DrillSimpleFunc {
+    @Param  VarCharHolder left;
+    @Param  VarCharHolder right;
+    @Workspace java.text.DecimalFormat inputFormat;
+    @Workspace int decimalDigits;
+    @Output Float8Holder out;
+
+    public void setup(RecordBatch b) {
+      byte[] buf = new byte[right.end - right.start];
+      right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
+      inputFormat = new DecimalFormat(new String(buf));
+      decimalDigits = inputFormat.getMaximumFractionDigits();
+    }
+
+    public void eval() {
+      byte[] buf1 = new byte[left.end - left.start];
+      left.buffer.getBytes(left.start, buf1, 0, left.end - left.start);
+      String input = new String(buf1);
+      try {
+        out.value = inputFormat.parse(input).doubleValue();
+      }  catch(java.text.ParseException e) {
+         throw new UnsupportedOperationException("Cannot parse input: " + input + " with pattern : " + inputFormat.toPattern());
+      }
+
+      // Round the value
+      java.math.BigDecimal roundedValue = new java.math.BigDecimal(out.value);
+      out.value = (roundedValue.setScale(decimalDigits, java.math.BigDecimal.ROUND_HALF_UP)).doubleValue();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
index aca5933..cebe491 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.record.RecordBatch;
 
 public class StringFunctions{
@@ -115,7 +116,19 @@ public class StringFunctions{
     public void eval(){
       out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharLength(input.buffer, input.start, input.end);          
     } 
-    
+  }
+
+  @FunctionTemplate(name = "lengthUtf8", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class ByteLength implements DrillSimpleFunc{
+
+    @Param  VarBinaryHolder input;
+    @Output BigIntHolder out;
+
+    public void setup(RecordBatch incoming){}
+
+    public void eval(){
+      out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharLength(input.buffer, input.start, input.end);
+    }
   }
 
   @FunctionTemplate(name = "octet_length", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
@@ -781,6 +794,92 @@ public class StringFunctions{
     
   }
 
+  @FunctionTemplate(name = "concat", scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL)
+  public static class ConcatRightNullInput implements DrillSimpleFunc{
+
+    @Param  VarCharHolder left;
+    @Param  NullableVarCharHolder right;
+    @Output VarCharHolder out;
+    @Workspace ByteBuf buffer;
+
+
+    public void setup(RecordBatch incoming){
+      buffer = io.netty.buffer.Unpooled.wrappedBuffer(new byte [8000]);
+    }
+
+    public void eval(){
+      out.buffer = buffer;
+      out.start = out.end = 0;
+
+      int id = 0;
+      for (id = left.start; id < left.end; id++)
+        out.buffer.setByte(out.end++, left.buffer.getByte(id));
+
+      if (right.isSet == 1) {
+       for (id = right.start; id < right.end; id++)
+         out.buffer.setByte(out.end++, right.buffer.getByte(id));
+      }
+    }
+  }
+
+  @FunctionTemplate(name = "concat", scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL)
+  public static class ConcatLeftNullInput implements DrillSimpleFunc{
+
+    @Param  NullableVarCharHolder left;
+    @Param  VarCharHolder right;
+    @Output VarCharHolder out;
+    @Workspace ByteBuf buffer;
+
+
+    public void setup(RecordBatch incoming){
+      buffer = io.netty.buffer.Unpooled.wrappedBuffer(new byte [8000]);
+    }
+
+    public void eval(){
+      out.buffer = buffer;
+      out.start = out.end = 0;
+
+      int id = 0;
+      if (left.isSet == 1) {
+        for (id = left.start; id < left.end; id++)
+          out.buffer.setByte(out.end++, left.buffer.getByte(id));
+      }
+
+       for (id = right.start; id < right.end; id++)
+         out.buffer.setByte(out.end++, right.buffer.getByte(id));
+    }
+  }
+
+  @FunctionTemplate(name = "concat", scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL)
+  public static class ConcatBothNullInput implements DrillSimpleFunc{
+
+    @Param  NullableVarCharHolder left;
+    @Param  NullableVarCharHolder right;
+    @Output VarCharHolder out;
+    @Workspace ByteBuf buffer;
+
+
+    public void setup(RecordBatch incoming){
+      buffer = io.netty.buffer.Unpooled.wrappedBuffer(new byte [8000]);
+    }
+
+    public void eval(){
+      out.buffer = buffer;
+      out.start = out.end = 0;
+
+      int id = 0;
+      if (left.isSet == 1) {
+        for (id = left.start; id < left.end; id++)
+          out.buffer.setByte(out.end++, left.buffer.getByte(id));
+      }
+
+      if (right.isSet == 1) {
+       for (id = right.start; id < right.end; id++)
+         out.buffer.setByte(out.end++, right.buffer.getByte(id));
+      }
+    }
+  }
+
   // Converts a hex encoded string into a varbinary type.
   // "\xca\xfe\xba\xbe" => (byte[]) {(byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe}
   @FunctionTemplate(name = "binary_string", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index aee6da4..73e835a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -257,13 +257,14 @@ public class DrillOptiq {
 
     }
 
-    private LogicalExpression getDrillFunctionFromOptiqCall(RexCall call){
+    private LogicalExpression getDrillFunctionFromOptiqCall(RexCall call) {
       List<LogicalExpression> args = Lists.newArrayList();
       for(RexNode n : call.getOperands()){
         args.add(n.accept(this));
       }
       String functionName = call.getOperator().getName().toLowerCase();
 
+      // TODO: once we have more function rewrites and a patter emerges from different rewrites, factor this out in a better fashion
       /* Rewrite extract functions in the following manner
        * extract(year, date '2008-2-23') ---> extractYear(date '2008-2-23')
        */
@@ -288,11 +289,8 @@ public class DrillOptiq {
           default:
             throw new UnsupportedOperationException("extract function supports the following time units: YEAR, MONTH, DAY, HOUR, MINUTE, SECOND");
         }
-      }
-
-      // Rewrite DATE_PART functions as extract functions
-      if (call.getOperator().getName().equalsIgnoreCase("DATE_PART")) {
-
+      } else if (functionName.equals("date_part")) {
+        // Rewrite DATE_PART functions as extract functions
         // assert that the function has exactly two arguments
         assert args.size() == 2;
 
@@ -305,9 +303,39 @@ public class DrillOptiq {
         QuotedString extractString = (QuotedString) args.get(0);
         String functionPostfix = extractString.value.substring(0, 1).toUpperCase() + extractString.value.substring(1).toLowerCase();
         return FunctionCallFactory.createExpression("extract" + functionPostfix, args.subList(1, 2));
+      } else if (functionName.equals("concat")) {
+
+        // Cast arguments to VARCHAR
+        List<LogicalExpression> concatArgs = Lists.newArrayList();
+        MajorType castType = Types.required(MinorType.VARCHAR).toBuilder().setWidth(64000).build();
+        concatArgs.add(FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, args.get(0)));
+        concatArgs.add(FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, args.get(1)));
+
+        LogicalExpression first = FunctionCallFactory.createExpression(functionName, concatArgs);
+
+        for (int i = 2; i < args.size(); i++) {
+          concatArgs = Lists.newArrayList();
+          concatArgs.add(first);
+          concatArgs.add(FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, args.get(i)));
+          first = FunctionCallFactory.createExpression(functionName, concatArgs);
+        }
+
+        return first;
+      } else if (functionName.equals("length")) {
+
+          if (args.size() == 2) {
+
+              // Second argument should always be a literal specifying the encoding format
+              assert args.get(1) instanceof ValueExpressions.QuotedString;
+
+              String encodingType = ((ValueExpressions.QuotedString) args.get(1)).value;
+              functionName += encodingType.substring(0, 1).toUpperCase() + encodingType.substring(1).toLowerCase();
+
+              return FunctionCallFactory.createExpression(functionName, args.subList(0, 1));
+          }
       }
 
-      return FunctionCallFactory.createExpression(call.getOperator().getName().toLowerCase(), args);
+      return FunctionCallFactory.createExpression(functionName, args);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
index 71bf616..9d83941 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
@@ -63,13 +63,13 @@ public class ResolverTypePrecedence {
   	precedenceMap.put(MinorType.UINT8, i += 2);
   	precedenceMap.put(MinorType.MONEY, i += 2);
   	precedenceMap.put(MinorType.FLOAT4, i += 2);
-  	precedenceMap.put(MinorType.FLOAT8, i += 2);
- 	precedenceMap.put(MinorType.DECIMAL9, i += 2);
-  	precedenceMap.put(MinorType.DECIMAL18, i += 2);
-   	precedenceMap.put(MinorType.DECIMAL28DENSE, i += 2);
-  	precedenceMap.put(MinorType.DECIMAL28SPARSE, i += 2);
+    precedenceMap.put(MinorType.DECIMAL9, i += 2);
+    precedenceMap.put(MinorType.DECIMAL18, i += 2);
+    precedenceMap.put(MinorType.DECIMAL28DENSE, i += 2);
+    precedenceMap.put(MinorType.DECIMAL28SPARSE, i += 2);
     precedenceMap.put(MinorType.DECIMAL38DENSE, i += 2);
     precedenceMap.put(MinorType.DECIMAL38SPARSE, i += 2);
+  	precedenceMap.put(MinorType.FLOAT8, i += 2);
   	precedenceMap.put(MinorType.TIME, i += 2);
   	precedenceMap.put(MinorType.DATE, i += 2);
   	precedenceMap.put(MinorType.TIMESTAMP, i += 2);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java
new file mode 100644
index 0000000..10c36cb
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestAggregateFunctionsQuery.java
@@ -0,0 +1,98 @@
+/**
+ * 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.drill.jdbc.test;
+
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
+import org.apache.drill.jdbc.Driver;
+import org.apache.drill.jdbc.JdbcTest;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+import com.google.common.base.Function;
+import com.google.common.base.Stopwatch;
+
+public class TestAggregateFunctionsQuery {
+
+  public static final String WORKING_PATH;
+  static{
+    Driver.load();
+    WORKING_PATH = Paths.get("").toAbsolutePath().toString();
+
+  }
+  @Test
+  public void testDateAggFunction() throws Exception{
+    String query = new String("SELECT max(cast(HIRE_DATE as date)) as MAX_DATE, min(cast(HIRE_DATE as date)) as MIN_DATE" +
+        " FROM `employee.json`");
+
+    JdbcAssert.withFull("cp")
+        .sql(query)
+        .returns(
+            "MAX_DATE=1998-01-01; " +
+                "MIN_DATE=1993-05-01\n"
+        );
+  }
+
+  @Test
+  public void testIntervalAggFunction() throws Exception{
+    String query = new String("select max(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) as MAX_DAYS,  min(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) MIN_DAYS" +
+        " FROM `employee.json`");
+
+    JdbcAssert.withFull("cp")
+        .sql(query)
+        .returns(
+            "MAX_DAYS=7671 days 0:0:0.0; " +
+                "MIN_DAYS=5965 days 0:0:0.0\n"
+        );
+  }
+
+  @Test
+  public void testDecimalAggFunction() throws Exception{
+    String query = new String("SELECT " +
+        "max(cast(EMPLOYEE_ID as decimal(9, 2))) as MAX_DEC9, min(cast(EMPLOYEE_ID as decimal(9, 2))) as MIN_DEC9," +
+        "max(cast(EMPLOYEE_ID as decimal(18, 4))) as MAX_DEC18, min(cast(EMPLOYEE_ID as decimal(18, 4))) as MIN_DEC18," +
+        "max(cast(EMPLOYEE_ID as decimal(28, 9))) as MAX_DEC28, min(cast(EMPLOYEE_ID as decimal(28, 9))) as MIN_DEC28," +
+        "max(cast(EMPLOYEE_ID as decimal(38, 11))) as MAX_DEC38, min(cast(EMPLOYEE_ID as decimal(38, 11))) as MIN_DEC38" +
+        " FROM `employee.json`");
+
+    JdbcAssert.withFull("cp")
+        .sql(query)
+        .returns(
+            "MAX_DEC9=1156.00; " +
+                "MIN_DEC9=1.00; " +
+                "MAX_DEC18=1156.0000; " +
+                "MIN_DEC18=1.0000; " +
+                "MAX_DEC28=1156.000000000; " +
+                "MIN_DEC28=1.000000000; " +
+                "MAX_DEC38=1156.00000000000; " +
+                "MIN_DEC38=1.00000000000\n "
+        );
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java
deleted file mode 100644
index 05c3fae..0000000
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestDateAggregateFunction.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.jdbc.test;
-
-import java.nio.file.Paths;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.Statement;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.drill.common.util.TestTools;
-import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
-import org.apache.drill.jdbc.Driver;
-import org.apache.drill.jdbc.JdbcTest;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-
-import com.google.common.base.Function;
-import com.google.common.base.Stopwatch;
-
-public class TestDateAggregateFunction {
-
-  public static final String WORKING_PATH;
-  static{
-    Driver.load();
-    WORKING_PATH = Paths.get("").toAbsolutePath().toString();
-
-  }
-  @Test
-  public void testDateAggFunction() throws Exception{
-    String query = new String("SELECT max(cast(HIRE_DATE as date)) as MAX_DATE, min(cast(HIRE_DATE as date)) as MIN_DATE" +
-        " FROM `employee.json`");
-
-    JdbcAssert.withFull("cp")
-        .sql(query)
-        .returns(
-                "MAX_DATE=1998-01-01; " +
-                "MIN_DATE=1993-05-01\n"
-        );
-  }
-
-  @Test
-  public void testIntervalAggFunction() throws Exception{
-    String query = new String("select max(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) as MAX_DAYS,  min(date_diff(date'2014-5-2', cast(HIRE_DATE as date))) MIN_DAYS" +
-        " FROM `employee.json`");
-
-    JdbcAssert.withFull("cp")
-        .sql(query)
-        .returns(
-            "MAX_DAYS=7671 days 0:0:0.0; " +
-                "MIN_DAYS=5965 days 0:0:0.0\n"
-        );
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/88e31534/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java
new file mode 100644
index 0000000..5ef25dc
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestFunctionsQuery.java
@@ -0,0 +1,404 @@
+/**
+ * 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.drill.jdbc.test;
+
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
+import org.apache.drill.jdbc.Driver;
+import org.apache.drill.jdbc.JdbcTest;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+import com.google.common.base.Function;
+import com.google.common.base.Stopwatch;
+
+public class TestFunctionsQuery {
+
+  public static final String WORKING_PATH;
+  static{
+    Driver.load();
+    WORKING_PATH = Paths.get("").toAbsolutePath().toString();
+
+  }
+  @Test
+  public void testAbsDecimalFunction() throws Exception{
+    String query = String.format("SELECT abs(cast('1234.4567' as decimal(9, 5))) as DEC9_ABS_1, " +
+        "abs(cast('-1234.4567' as decimal(9, 5))) DEC9_ABS_2, " +
+        "abs(cast('99999912399.4567' as decimal(18, 5))) DEC18_ABS_1, " +
+        "abs(cast('-99999912399.4567' as decimal(18, 5))) DEC18_ABS_2, " +
+        "abs(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_ABS_1, " +
+        "abs(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_ABS_2, " +
+        "abs(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_ABS_1, " +
+        "abs(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_ABS_2 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_ABS_1=1234.45670; " +
+                "DEC9_ABS_2=1234.45670; " +
+                "DEC18_ABS_1=99999912399.45670; " +
+                "DEC18_ABS_2=99999912399.45670; " +
+                "DEC28_ABS_1=12345678912345678912.45670; " +
+                "DEC28_ABS_2=12345678912345678912.45670; " +
+                "DEC38_ABS_1=1234567891234567891234567891234567891.4; " +
+                "DEC38_ABS_2=1234567891234567891234567891234567891.4\n"
+        );
+
+  }
+
+  @Test
+  public void testCeilDecimalFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "ceil(cast('1234.4567' as decimal(9, 5))) as DEC9_1, " +
+        "ceil(cast('1234.0000' as decimal(9, 5))) as DEC9_2, " +
+        "ceil(cast('-1234.4567' as decimal(9, 5))) as DEC9_3, " +
+        "ceil(cast('-1234.000' as decimal(9, 5))) as DEC9_4, " +
+        "ceil(cast('99999912399.4567' as decimal(18, 5))) DEC18_1, " +
+        "ceil(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " +
+        "ceil(cast('-99999912399.4567' as decimal(18, 5))) DEC18_3, " +
+        "ceil(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " +
+        "ceil(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_1, " +
+        "ceil(cast('999999999999999999.4567' as decimal(28, 5))) DEC28_2, " +
+        "ceil(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " +
+        "ceil(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_4, " +
+        "ceil(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " +
+        "ceil(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_1, " +
+        "ceil(cast('999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_2, " +
+        "ceil(cast('1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_3, " +
+        "ceil(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_4, " +
+        "ceil(cast('-1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_5 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_1=1235; " +
+                "DEC9_2=1234; " +
+                "DEC9_3=-1234; " +
+                "DEC9_4=-1234; " +
+                "DEC18_1=99999912400; " +
+                "DEC18_2=99999912399; " +
+                "DEC18_3=-99999912399; " +
+                "DEC18_4=-99999912399; " +
+                "DEC28_1=12345678912345678913; " +
+                "DEC28_2=1000000000000000000; " +
+                "DEC28_3=12345678912345678912; " +
+                "DEC28_4=-12345678912345678912; " +
+                "DEC28_5=-12345678912345678912; " +
+                "DEC38_1=1234567891234567891234567891234567892; " +
+                "DEC38_2=1000000000000000000000000000000000000; " +
+                "DEC38_3=1234567891234567891234567891234567891; " +
+                "DEC38_4=-1234567891234567891234567891234567891; " +
+                "DEC38_5=-1234567891234567891234567891234567891\n"
+        );
+
+  }
+
+  @Test
+  public void testFloorDecimalFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "floor(cast('1234.4567' as decimal(9, 5))) as DEC9_1, " +
+        "floor(cast('1234.0000' as decimal(9, 5))) as DEC9_2, " +
+        "floor(cast('-1234.4567' as decimal(9, 5))) as DEC9_3, " +
+        "floor(cast('-1234.000' as decimal(9, 5))) as DEC9_4, " +
+        "floor(cast('99999912399.4567' as decimal(18, 5))) DEC18_1, " +
+        "floor(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " +
+        "floor(cast('-99999912399.4567' as decimal(18, 5))) DEC18_3, " +
+        "floor(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " +
+        "floor(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_1, " +
+        "floor(cast('999999999999999999.4567' as decimal(28, 5))) DEC28_2, " +
+        "floor(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " +
+        "floor(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_4, " +
+        "floor(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " +
+        "floor(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_1, " +
+        "floor(cast('999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_2, " +
+        "floor(cast('1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_3, " +
+        "floor(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_4, " +
+        "floor(cast('-999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_5 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_1=1234; " +
+                "DEC9_2=1234; " +
+                "DEC9_3=-1235; " +
+                "DEC9_4=-1234; " +
+                "DEC18_1=99999912399; " +
+                "DEC18_2=99999912399; " +
+                "DEC18_3=-99999912400; " +
+                "DEC18_4=-99999912399; " +
+                "DEC28_1=12345678912345678912; " +
+                "DEC28_2=999999999999999999; " +
+                "DEC28_3=12345678912345678912; " +
+                "DEC28_4=-12345678912345678913; " +
+                "DEC28_5=-12345678912345678912; " +
+                "DEC38_1=1234567891234567891234567891234567891; " +
+                "DEC38_2=999999999999999999999999999999999999; " +
+                "DEC38_3=1234567891234567891234567891234567891; " +
+                "DEC38_4=-1234567891234567891234567891234567892; " +
+                "DEC38_5=-1000000000000000000000000000000000000\n"
+        );
+  }
+
+  @Test
+  public void testTruncateDecimalFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "trunc(cast('1234.4567' as decimal(9, 5))) as DEC9_1, " +
+        "trunc(cast('1234.0000' as decimal(9, 5))) as DEC9_2, " +
+        "trunc(cast('-1234.4567' as decimal(9, 5))) as DEC9_3, " +
+        "trunc(cast('0.111' as decimal(9, 5))) as DEC9_4, " +
+        "trunc(cast('99999912399.4567' as decimal(18, 5))) DEC18_1, " +
+        "trunc(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " +
+        "trunc(cast('-99999912399.4567' as decimal(18, 5))) DEC18_3, " +
+        "trunc(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " +
+        "trunc(cast('12345678912345678912.4567' as decimal(28, 5))) DEC28_1, " +
+        "trunc(cast('999999999999999999.4567' as decimal(28, 5))) DEC28_2, " +
+        "trunc(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " +
+        "trunc(cast('-12345678912345678912.4567' as decimal(28, 5))) DEC28_4, " +
+        "trunc(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " +
+        "trunc(cast('1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_1, " +
+        "trunc(cast('999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_2, " +
+        "trunc(cast('1234567891234567891234567891234567891.0' as decimal(38, 1))) DEC38_3, " +
+        "trunc(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_4, " +
+        "trunc(cast('-999999999999999999999999999999999999.4' as decimal(38, 1))) DEC38_5 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_1=1234; " +
+                "DEC9_2=1234; " +
+                "DEC9_3=-1234; " +
+                "DEC9_4=0; " +
+                "DEC18_1=99999912399; " +
+                "DEC18_2=99999912399; " +
+                "DEC18_3=-99999912399; " +
+                "DEC18_4=-99999912399; " +
+                "DEC28_1=12345678912345678912; " +
+                "DEC28_2=999999999999999999; " +
+                "DEC28_3=12345678912345678912; " +
+                "DEC28_4=-12345678912345678912; " +
+                "DEC28_5=-12345678912345678912; " +
+                "DEC38_1=1234567891234567891234567891234567891; " +
+                "DEC38_2=999999999999999999999999999999999999; " +
+                "DEC38_3=1234567891234567891234567891234567891; " +
+                "DEC38_4=-1234567891234567891234567891234567891; " +
+                "DEC38_5=-999999999999999999999999999999999999\n"
+        );
+  }
+
+  @Test
+  public void testTruncateWithParamDecimalFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "trunc(cast('1234.4567' as decimal(9, 5)), 2) as DEC9_1, " +
+        "trunc(cast('1234.45' as decimal(9, 2)), 4) as DEC9_2, " +
+        "trunc(cast('-1234.4567' as decimal(9, 5)), 0) as DEC9_3, " +
+        "trunc(cast('0.111' as decimal(9, 5)), 2) as DEC9_4, " +
+        "trunc(cast('99999912399.4567' as decimal(18, 5)), 2) DEC18_1, " +
+        "trunc(cast('99999912399.0000' as decimal(18, 5)), 2) DEC18_2, " +
+        "trunc(cast('-99999912399.45' as decimal(18, 2)), 6) DEC18_3, " +
+        "trunc(cast('-99999912399.0000' as decimal(18, 5)), 4) DEC18_4, " +
+        "trunc(cast('12345678912345678912.4567' as decimal(28, 5)), 1) DEC28_1, " +
+        "trunc(cast('999999999999999999.456' as decimal(28, 3)), 6) DEC28_2, " +
+        "trunc(cast('12345678912345678912.0000' as decimal(28, 5)), 2) DEC28_3, " +
+        "trunc(cast('-12345678912345678912.45' as decimal(28, 2)), 0) DEC28_4, " +
+        "trunc(cast('-12345678912345678912.0000' as decimal(28, 5)), 1) DEC28_5, " +
+        "trunc(cast('999999999.123456789' as decimal(38, 9)), 7) DEC38_1, " +
+        "trunc(cast('999999999.4' as decimal(38, 1)), 8) DEC38_2, " +
+        "trunc(cast('999999999.1234' as decimal(38, 4)), 12) DEC38_3, " +
+        "trunc(cast('-123456789123456789.4' as decimal(38, 1)), 10) DEC38_4, " +
+        "trunc(cast('-999999999999999999999999999999999999.4' as decimal(38, 1)), 1) DEC38_5 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_1=1234.45; " +
+                "DEC9_2=1234.4500; " +
+                "DEC9_3=-1234; " +
+                "DEC9_4=0.11; " +
+                "DEC18_1=99999912399.45; " +
+                "DEC18_2=99999912399.00; " +
+                "DEC18_3=-99999912399.450000; " +
+                "DEC18_4=-99999912399.0000; " +
+                "DEC28_1=12345678912345678912.4; " +
+                "DEC28_2=999999999999999999.456000; " +
+                "DEC28_3=12345678912345678912.00; " +
+                "DEC28_4=-12345678912345678912; " +
+                "DEC28_5=-12345678912345678912.0; " +
+                "DEC38_1=999999999.1234567; " +
+                "DEC38_2=999999999.40000000; " +
+                "DEC38_3=999999999.123400000000; " +
+                "DEC38_4=-123456789123456789.4000000000; " +
+                "DEC38_5=-999999999999999999999999999999999999.4\n"
+        );
+  }
+
+  @Test
+  public void testRoundDecimalFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "round(cast('1234.5567' as decimal(9, 5))) as DEC9_1, " +
+        "round(cast('1234.1000' as decimal(9, 5))) as DEC9_2, " +
+        "round(cast('-1234.5567' as decimal(9, 5))) as DEC9_3, " +
+        "round(cast('-1234.1234' as decimal(9, 5))) as DEC9_4, " +
+        "round(cast('99999912399.9567' as decimal(18, 5))) DEC18_1, " +
+        "round(cast('99999912399.0000' as decimal(18, 5))) DEC18_2, " +
+        "round(cast('-99999912399.5567' as decimal(18, 5))) DEC18_3, " +
+        "round(cast('-99999912399.0000' as decimal(18, 5))) DEC18_4, " +
+        "round(cast('12345678912345678912.5567' as decimal(28, 5))) DEC28_1, " +
+        "round(cast('999999999999999999.5567' as decimal(28, 5))) DEC28_2, " +
+        "round(cast('12345678912345678912.0000' as decimal(28, 5))) DEC28_3, " +
+        "round(cast('-12345678912345678912.5567' as decimal(28, 5))) DEC28_4, " +
+        "round(cast('-12345678912345678912.0000' as decimal(28, 5))) DEC28_5, " +
+        "round(cast('999999999999999999999999999.5' as decimal(38, 1))) DEC38_1, " +
+        "round(cast('99999999.512345678123456789' as decimal(38, 18))) DEC38_2, " +
+        "round(cast('999999999999999999999999999999999999.5' as decimal(38, 1))) DEC38_3, " +
+        "round(cast('1234567891234567891234567891234567891.2' as decimal(38, 1))) DEC38_4, " +
+        "round(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1))) DEC38_5, " +
+        "round(cast('-999999999999999999999999999999999999.9' as decimal(38, 1))) DEC38_6 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_1=1235; " +
+                "DEC9_2=1234; " +
+                "DEC9_3=-1235; " +
+                "DEC9_4=-1234; " +
+                "DEC18_1=99999912400; " +
+                "DEC18_2=99999912399; " +
+                "DEC18_3=-99999912400; " +
+                "DEC18_4=-99999912399; " +
+                "DEC28_1=12345678912345678913; " +
+                "DEC28_2=1000000000000000000; " +
+                "DEC28_3=12345678912345678912; " +
+                "DEC28_4=-12345678912345678913; " +
+                "DEC28_5=-12345678912345678912; " +
+                "DEC38_1=1000000000000000000000000000; " +
+                "DEC38_2=100000000; " +
+                "DEC38_3=1000000000000000000000000000000000000; " +
+                "DEC38_4=1234567891234567891234567891234567891; " +
+                "DEC38_5=-1234567891234567891234567891234567891; " +
+                "DEC38_6=-1000000000000000000000000000000000000\n"
+        );
+  }
+
+  @Test
+  public void testRoundWithScaleDecimalFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "round(cast('1234.5567' as decimal(9, 5)), 3) as DEC9_1, " +
+        "round(cast('1234.1000' as decimal(9, 5)), 2) as DEC9_2, " +
+        "round(cast('-1234.5567' as decimal(9, 5)), 4) as DEC9_3, " +
+        "round(cast('-1234.1234' as decimal(9, 5)), 3) as DEC9_4, " +
+        "round(cast('-1234.1234' as decimal(9, 2)), 4) as DEC9_5, " +
+        "round(cast('99999912399.9567' as decimal(18, 5)), 3) DEC18_1, " +
+        "round(cast('99999912399.0000' as decimal(18, 5)), 2) DEC18_2, " +
+        "round(cast('-99999912399.5567' as decimal(18, 5)), 2) DEC18_3, " +
+        "round(cast('-99999912399.0000' as decimal(18, 5)), 0) DEC18_4, " +
+        "round(cast('12345678912345678912.5567' as decimal(28, 5)), 2) DEC28_1, " +
+        "round(cast('999999999999999999.5567' as decimal(28, 5)), 1) DEC28_2, " +
+        "round(cast('12345678912345678912.0000' as decimal(28, 5)), 8) DEC28_3, " +
+        "round(cast('-12345678912345678912.5567' as decimal(28, 5)), 3) DEC28_4, " +
+        "round(cast('-12345678912345678912.0000' as decimal(28, 5)), 0) DEC28_5, " +
+        "round(cast('999999999999999999999999999.5' as decimal(38, 1)), 1) DEC38_1, " +
+        "round(cast('99999999.512345678923456789' as decimal(38, 18)), 9) DEC38_2, " +
+        "round(cast('999999999.9999999995678' as decimal(38, 18)), 9) DEC38_3, " +
+        "round(cast('999999999.9999999995678' as decimal(38, 18)), 11) DEC38_4, " +
+        "round(cast('999999999.9999999995678' as decimal(38, 18)), 21) DEC38_5, " +
+        "round(cast('-1234567891234567891234567891234567891.4' as decimal(38, 1)), 1) DEC38_6, " +
+        "round(cast('-999999999999999999999999999999999999.9' as decimal(38, 1)), 0) DEC38_7 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "DEC9_1=1234.557; " +
+                "DEC9_2=1234.10; " +
+                "DEC9_3=-1234.5567; " +
+                "DEC9_4=-1234.123; " +
+                "DEC9_5=-1234.1200; " +
+                "DEC18_1=99999912399.957; " +
+                "DEC18_2=99999912399.00; " +
+                "DEC18_3=-99999912399.56; " +
+                "DEC18_4=-99999912399; " +
+                "DEC28_1=12345678912345678912.56; " +
+                "DEC28_2=999999999999999999.6; " +
+                "DEC28_3=12345678912345678912.00000000; " +
+                "DEC28_4=-12345678912345678912.557; " +
+                "DEC28_5=-12345678912345678912; " +
+                "DEC38_1=999999999999999999999999999.5; " +
+                "DEC38_2=99999999.512345679; " +
+                "DEC38_3=1000000000.000000000; " +
+                "DEC38_4=999999999.99999999957; " +
+                "DEC38_5=999999999.999999999567800000000; " +
+                "DEC38_6=-1234567891234567891234567891234567891.4; " +
+                "DEC38_7=-1000000000000000000000000000000000000\n"
+        );
+  }
+
+  @Test
+  public void testToCharFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "to_char(1234.5567, '#,###.##') as FLOAT8_1, " +
+        "to_char(1234.5, '$#,###.00') as FLOAT8_2, " +
+        "to_char(cast('1234.5567' as decimal(9, 5)), '#,###.##') as DEC9_1, " +
+        "to_char(cast('99999912399.9567' as decimal(18, 5)), '#.#####') DEC18_1, " +
+        "to_char(cast('12345678912345678912.5567' as decimal(28, 5)), '#,###.#####') DEC28_1, " +
+        "to_char(cast('999999999999999999999999999.5' as decimal(38, 1)), '#.#') DEC38_1 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "FLOAT8_1=1,234.56; " +
+                "FLOAT8_2=$1,234.50; " +
+                "DEC9_1=1,234.56; " +
+                "DEC18_1=99999912399.9567; " +
+                "DEC28_1=12,345,678,912,345,678,912.5567; " +
+                "DEC38_1=999999999999999999999999999.5\n"
+        );
+  }
+
+
+  @Test
+  public void testConcatFunction() throws Exception {
+    String query = String.format("SELECT " +
+        "concat('1234', ' COL_VALUE ', R_REGIONKEY, ' - STRING') as STR_1 " +
+        "FROM dfs.`%s/../../sample-data/region.parquet` limit 1", WORKING_PATH);
+
+    JdbcAssert.withNoDefaultSchema()
+        .sql(query)
+        .returns(
+            "STR_1=1234 COL_VALUE 0 - STRING\n"
+        );
+  }
+}