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/04/24 09:08:26 UTC

[08/10] git commit: DRILL-452: Conversion functions for external data types

DRILL-452: Conversion functions for external data types


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

Branch: refs/heads/master
Commit: 3a1e6c9fa9df30e14945ff78afd8049e6185bba3
Parents: 166079b
Author: Aditya Kishore <ad...@maprtech.com>
Authored: Tue Mar 25 01:21:50 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Apr 24 00:07:32 2014 -0700

----------------------------------------------------------------------
 .../drill/common/expression/parser/ExprLexer.g  |   1 +
 .../drill/common/expression/parser/ExprParser.g |   8 +-
 .../common/expression/ConvertExpression.java    |  82 +++
 .../expression/ExpressionStringBuilder.java     |  11 +-
 .../common/expression/FunctionCallFactory.java  |   4 +
 .../visitors/AbstractExprVisitor.java           |   6 +
 .../expression/visitors/AggregateChecker.java   |   9 +
 .../expression/visitors/ConstantChecker.java    |   5 +
 .../common/expression/visitors/ExprVisitor.java |   3 +-
 .../visitors/ExpressionValidator.java           |   9 +-
 .../org/apache/drill/common/types/Types.java    |  54 +-
 .../sig/ConstantExpressionIdentifier.java       |   8 +-
 .../drill/exec/expr/EvaluationVisitor.java      |  12 +
 .../exec/expr/ExpressionTreeMaterializer.java   |  12 +
 .../expr/fn/impl/conv/BigIntBEConvertFrom.java  |  46 ++
 .../expr/fn/impl/conv/BigIntBEConvertTo.java    |  53 ++
 .../expr/fn/impl/conv/BigIntConvertFrom.java    |  47 ++
 .../exec/expr/fn/impl/conv/BigIntConvertTo.java |  54 ++
 .../fn/impl/conv/BigIntVLongConvertFrom.java    |  43 ++
 .../expr/fn/impl/conv/BigIntVLongConvertTo.java |  55 ++
 .../fn/impl/conv/BooleanByteConvertFrom.java    |  47 ++
 .../expr/fn/impl/conv/BooleanByteConvertTo.java |  54 ++
 .../fn/impl/conv/DateEpochBEConvertFrom.java    |  48 ++
 .../expr/fn/impl/conv/DateEpochBEConvertTo.java |  53 ++
 .../expr/fn/impl/conv/DateEpochConvertFrom.java |  49 ++
 .../expr/fn/impl/conv/DateEpochConvertTo.java   |  54 ++
 .../expr/fn/impl/conv/DoubleConvertFrom.java    |  47 ++
 .../exec/expr/fn/impl/conv/DoubleConvertTo.java |  54 ++
 .../expr/fn/impl/conv/FloatConvertFrom.java     |  47 ++
 .../exec/expr/fn/impl/conv/FloatConvertTo.java  |  54 ++
 .../expr/fn/impl/conv/IntBEConvertFrom.java     |  46 ++
 .../exec/expr/fn/impl/conv/IntBEConvertTo.java  |  53 ++
 .../exec/expr/fn/impl/conv/IntConvertFrom.java  |  47 ++
 .../exec/expr/fn/impl/conv/IntConvertTo.java    |  54 ++
 .../expr/fn/impl/conv/IntVIntConvertFrom.java   |  43 ++
 .../expr/fn/impl/conv/IntVIntConvertTo.java     |  55 ++
 .../fn/impl/conv/SmallIntBEConvertFrom.java     |  47 ++
 .../expr/fn/impl/conv/SmallIntBEConvertTo.java  |  54 ++
 .../expr/fn/impl/conv/SmallIntConvertFrom.java  |  47 ++
 .../expr/fn/impl/conv/SmallIntConvertTo.java    |  54 ++
 .../fn/impl/conv/TimeEpochBEConvertFrom.java    |  47 ++
 .../expr/fn/impl/conv/TimeEpochBEConvertTo.java |  53 ++
 .../expr/fn/impl/conv/TimeEpochConvertFrom.java |  48 ++
 .../expr/fn/impl/conv/TimeEpochConvertTo.java   |  54 ++
 .../expr/fn/impl/conv/TinyIntConvertFrom.java   |  48 ++
 .../expr/fn/impl/conv/TinyIntConvertTo.java     |  55 ++
 .../expr/fn/impl/conv/UInt8ConvertFrom.java     |  47 ++
 .../exec/expr/fn/impl/conv/UInt8ConvertTo.java  |  54 ++
 .../expr/fn/impl/conv/UTF16ConvertFrom.java     |  52 ++
 .../exec/expr/fn/impl/conv/UTF16ConvertTo.java  |  52 ++
 .../exec/expr/fn/impl/conv/UTF8ConvertFrom.java |  52 ++
 .../exec/expr/fn/impl/conv/UTF8ConvertTo.java   |  52 ++
 .../drill/exec/resolver/TypeCastRules.java      | 112 +++++
 .../drill/exec/store/hive/HiveRecordReader.java |  31 +-
 .../org/apache/drill/exec/util/ConvertUtil.java | 209 ++++++++
 .../physical/impl/TestConvertFunctions.java     | 498 +++++++++++++++++++
 .../conv/conversionTestWithLogicalPlan.json     |  61 +++
 .../conv/conversionTestWithPhysicalPlan.json    |  36 ++
 58 files changed, 3052 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
----------------------------------------------------------------------
diff --git a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
index b0d082d..58ed215 100644
--- a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
@@ -36,6 +36,7 @@ Case     : 'case';
 When     : 'when';
 
 Cast: 'cast';
+Convert  : 'convert_' ('from' | 'to');
 Nullable: 'nullable';
 Repeat: 'repeat';
 As: 'as';

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
----------------------------------------------------------------------
diff --git a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
index ca37e4c..5ad7099 100644
--- a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
@@ -73,7 +73,12 @@ parse returns [LogicalExpression e]
 functionCall returns [LogicalExpression e]
   :  Identifier OParen exprList? CParen {$e = FunctionCallFactory.createExpression($Identifier.text, pos($Identifier), $exprList.listE);  }
   ;
-  
+
+convertCall returns [LogicalExpression e]
+  :  Convert OParen expression Comma String CParen
+      { $e = FunctionCallFactory.createConvert($Convert.text, $String.text, $expression.e, pos($Convert));}
+  ;
+
 castCall returns [LogicalExpression e]
 	@init{
   	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
@@ -299,6 +304,7 @@ arraySegment returns [PathSegment seg]
 
 lookup returns [LogicalExpression e]
   :  functionCall {$e = $functionCall.e ;}
+  | convertCall {$e = $convertCall.e; }
   | castCall {$e = $castCall.e; }
   | pathSegment {$e = new SchemaPath($pathSegment.seg, pos($pathSegment.start) ); }
   | String {$e = new ValueExpressions.QuotedString($String.text, pos($String) ); }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/ConvertExpression.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ConvertExpression.java b/common/src/main/java/org/apache/drill/common/expression/ConvertExpression.java
new file mode 100644
index 0000000..c028083
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/expression/ConvertExpression.java
@@ -0,0 +1,82 @@
+/**
+ * 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.common.expression;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+
+public class ConvertExpression extends LogicalExpressionBase implements Iterable<LogicalExpression>{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConvertExpression.class);
+
+  private final LogicalExpression input;
+  private final MajorType type;
+  private final String convertFunction;
+  private final String conversionType;
+
+  /**
+   * @param conversionType
+   * @param convertFunction
+   * @param input
+   * @param pos
+   */
+  public ConvertExpression(String convertFunction, String conversionType, LogicalExpression input, ExpressionPosition pos) {
+    super(pos);
+    this.input = input;
+    this.convertFunction = convertFunction.toLowerCase();
+    this.conversionType = conversionType.toUpperCase();
+    this.type = Types.getMajorTypeFromName(conversionType.split("_", 2)[0].toLowerCase());
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitConvertExpression(this, value);
+  }
+
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Collections.singleton(input).iterator();
+  }
+
+  public String getConvertFunction() {
+    return convertFunction;
+  }
+
+  public LogicalExpression getInput() {
+    return input;
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return type;
+  }
+
+  public String getConversionType() {
+    return conversionType;
+  }
+
+  @Override
+  public String toString() {
+    return "ConvertExpression [input=" + input + ", type=" + type + ", convertFunction="
+        + convertFunction + ", conversionType=" + conversionType + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 8b43846..2ec4fca 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -238,6 +238,14 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
   }
 
   @Override
+  public Void visitConvertExpression(ConvertExpression e, StringBuilder sb) throws RuntimeException {
+    sb.append(e.getConvertFunction()).append("(");
+    e.getInput().accept(this, sb);
+    sb.append(", \"").append(e.getConversionType()).append("\")");
+    return null;
+  }
+
+  @Override
   public Void visitCastExpression(CastExpression e, StringBuilder sb) throws RuntimeException {
     MajorType mt = e.getMajorType();
 
@@ -310,7 +318,4 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
     return null;
   }
 
-
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
index 8113107..6e8e0f4 100644
--- a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
+++ b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
@@ -79,6 +79,10 @@ public class FunctionCallFactory {
     return new CastExpression(expr, type, ep);
   }
 
+  public static LogicalExpression createConvert(String function, String conversionType, LogicalExpression expr, ExpressionPosition ep) {
+    return new ConvertExpression(function, conversionType, expr, ep);
+  }
+
   public static LogicalExpression createExpression(String functionName, List<LogicalExpression> args){
     return createExpression(functionName, ExpressionPosition.UNKNOWN, args);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java b/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
index 7ed36f6..526275f 100644
--- a/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
+++ b/common/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression.visitors;
 
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.IfExpression;
@@ -144,6 +145,11 @@ public abstract class AbstractExprVisitor<T, VAL, EXCEP extends Exception> imple
   }
 
   @Override
+  public T visitConvertExpression(ConvertExpression e, VAL value) throws EXCEP {
+    return visitUnknown(e, value);
+  }
+
+  @Override
   public T visitUnknown(LogicalExpression e, VAL value) throws EXCEP {
     throw new UnsupportedOperationException(String.format("Expression of type %s not handled by visitor type %s.", e.getClass().getCanonicalName(), this.getClass().getCanonicalName()));
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java b/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
index 4291dee..bf67a6b 100644
--- a/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
+++ b/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression.visitors;
 
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
@@ -147,9 +148,15 @@ public final class AggregateChecker implements ExprVisitor<Boolean, ErrorCollect
   }
 
   @Override
+  public Boolean visitConvertExpression(ConvertExpression e, ErrorCollector errors) throws RuntimeException {
+    return e.getInput().accept(this, errors);
+  }
+
+  @Override
   public Boolean visitDateConstant(DateExpression intExpr, ErrorCollector errors) {
       return false;
   }
+
   @Override
   public Boolean visitTimeConstant(TimeExpression intExpr, ErrorCollector errors) {
       return false;
@@ -164,8 +171,10 @@ public final class AggregateChecker implements ExprVisitor<Boolean, ErrorCollect
   public Boolean visitIntervalYearConstant(IntervalYearExpression intExpr, ErrorCollector errors) {
       return false;
   }
+
   @Override
   public Boolean visitIntervalDayConstant(IntervalDayExpression intExpr, ErrorCollector errors) {
       return false;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java b/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
index 09fbedf..ab94987 100644
--- a/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
+++ b/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression.visitors;
 
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
@@ -175,4 +176,8 @@ final class ConstantChecker implements ExprVisitor<Boolean, ErrorCollector, Runt
     return e.getInput().accept(this, value);
   }
 
+  @Override
+  public Boolean visitConvertExpression(ConvertExpression e, ErrorCollector value) throws RuntimeException {
+    return e.getInput().accept(this, value);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java b/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
index 92fd777..799c9dd 100644
--- a/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
+++ b/common/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression.visitors;
 
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.IfExpression;
@@ -62,5 +63,5 @@ public interface ExprVisitor<T, VAL, EXCEP extends Exception> {
   public T visitQuotedStringConstant(QuotedString e, VAL value) throws EXCEP;
   public T visitUnknown(LogicalExpression e, VAL value) throws EXCEP;
   public T visitCastExpression(CastExpression e, VAL value) throws EXCEP;
-  
+  public T visitConvertExpression(ConvertExpression e, VAL value) throws EXCEP;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java b/common/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
index 1caf145..e9bd03a 100644
--- a/common/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
+++ b/common/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression.visitors;
 
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
@@ -194,7 +195,13 @@ public class ExpressionValidator implements ExprVisitor<Void, ErrorCollector, Ru
 
   @Override
   public Void visitCastExpression(CastExpression e, ErrorCollector value) throws RuntimeException {
-    return e.accept(this, value);
+    return e.getInput().accept(this, value);
+  }
+
+  @Override
+  public Void visitConvertExpression(ConvertExpression e, ErrorCollector value)
+      throws RuntimeException {
+    return e.getInput().accept(this, value);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index 5418e50..c51177d 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -255,6 +255,10 @@ public class Types {
     return type.getMinorType() == MinorType.LATE;
   }
   
+  public static MajorType withMode(MinorType type, DataMode mode){
+    return MajorType.newBuilder().setMode(mode).setMinorType(type).build();
+  }
+  
   public static MajorType required(MinorType type){
     return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
   }
@@ -279,6 +283,54 @@ public class Types {
         throw new UnsupportedOperationException();
     }
   }
+
+  public static MajorType getMajorTypeFromName(String typeName) {
+    return getMajorTypeFromName(typeName, DataMode.REQUIRED);
+  }
   
-  
+  public static MajorType getMajorTypeFromName(String typeName, DataMode mode) {
+    switch (typeName) {
+    case "bool":
+    case "boolean":
+      return withMode(MinorType.BIT, mode);
+    case "tinyint":
+      return withMode(MinorType.TINYINT, mode);
+    case "uint1":
+      return withMode(MinorType.UINT1, mode);
+    case "smallint":
+      return withMode(MinorType.SMALLINT, mode);
+    case "uint2":
+      return withMode(MinorType.UINT2, mode);
+    case "int":
+      return withMode(MinorType.INT, mode);
+    case "uint4":
+      return withMode(MinorType.UINT4, mode);
+    case "bigint":
+      return withMode(MinorType.BIGINT, mode);
+    case "uint8":
+      return withMode(MinorType.UINT8, mode);
+    case "float":
+      return withMode(MinorType.FLOAT4, mode);
+    case "double":
+      return withMode(MinorType.FLOAT8, mode);
+    case "decimal":
+      return withMode(MinorType.DECIMAL38SPARSE, mode);
+    case "utf8":
+    case "varchar":
+      return withMode(MinorType.VARCHAR, mode);
+    case "utf16":
+    case "string":
+    case "var16char":
+      return withMode(MinorType.VAR16CHAR, mode);
+    case "date":
+      return withMode(MinorType.DATE, mode);
+    case "time":
+      return withMode(MinorType.TIME, mode);
+    case "binary":
+      return withMode(MinorType.VARBINARY, mode);
+    default:
+      throw new UnsupportedOperationException("Could not determine type: " + typeName);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
index 2b125cd..5cff001 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.IfExpression;
@@ -202,5 +203,10 @@ public class ConstantExpressionIdentifier implements ExprVisitor<Boolean, Identi
   public Boolean visitUnknown(LogicalExpression e, IdentityHashMap<LogicalExpression, Object> value){
     return checkChildren(e, value, false);
   }
-  
+
+  @Override
+  public Boolean visitConvertExpression(ConvertExpression e,
+      IdentityHashMap<LogicalExpression, Object> value) throws RuntimeException {
+    return e.getInput().accept(this, value);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index fd547e1..dce070f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -23,6 +23,7 @@ import java.util.Set;
 import io.netty.buffer.ByteBuf;
 import com.google.common.collect.Lists;
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.IfExpression;
@@ -440,7 +441,18 @@ public class EvaluationVisitor {
         "It should have been converted to FunctionHolderExpression in materialization");
     }
 
+    @Override
+    public HoldingContainer visitConvertExpression(ConvertExpression e, ClassGenerator<?> value) throws RuntimeException {
+      String convertFunctionName = e.getConvertFunction() + e.getConversionType();
+
+      List<LogicalExpression> newArgs = Lists.newArrayList();
+      newArgs.add(e.getInput());  //input_expr
+
+      FunctionCall fc = new FunctionCall(convertFunctionName, newArgs, e.getPosition());
+      return fc.accept(this, value);
+    }
   }
+
   private class ConstantFilter extends EvalVisitor {
 
     private Set<LogicalExpression> constantBoundaries;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index e3b1002..a602d82 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -21,6 +21,7 @@ import java.util.List;
 
 import com.google.common.base.Joiner;
 import org.apache.drill.common.expression.CastExpression;
+import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FunctionCall;
@@ -340,6 +341,17 @@ public class ExpressionTreeMaterializer {
     }
 
     @Override
+    public LogicalExpression visitConvertExpression(ConvertExpression e, FunctionImplementationRegistry value) {
+      String convertFunctionName = e.getConvertFunction() + e.getConversionType();
+
+      List<LogicalExpression> newArgs = Lists.newArrayList();
+      newArgs.add(e.getInput());  //input_expr
+
+      FunctionCall fc = new FunctionCall(convertFunctionName, newArgs, e.getPosition());
+      return fc.accept(this, value);
+    }
+
+    @Override
     public LogicalExpression visitCastExpression(CastExpression e, FunctionImplementationRegistry value){
       
       // if the cast is pointless, remove it.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertFrom.java
new file mode 100644
index 0000000..01da0b1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertFrom.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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromBIGINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BigIntBEConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output BigIntHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 8);
+
+    in.buffer.readerIndex(in.start);
+    out.value = Long.reverseBytes(in.buffer.readLong());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java
new file mode 100644
index 0000000..7122937
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntBEConvertTo.java
@@ -0,0 +1,53 @@
+/**
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toBIGINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BigIntBEConvertTo implements DrillSimpleFunc {
+
+  @Param BigIntHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(8);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeLong(Long.reverseBytes(in.value));
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertFrom.java
new file mode 100644
index 0000000..093de0f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertFrom.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromBIGINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BigIntConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output BigIntHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 8);
+
+    in.buffer.readerIndex(in.start);
+    out.value = in.buffer.readLong();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java
new file mode 100644
index 0000000..70bb6e7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntConvertTo.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toBIGINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BigIntConvertTo implements DrillSimpleFunc {
+
+  @Param BigIntHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(8);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeLong(in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertFrom.java
new file mode 100644
index 0000000..d7bbfcc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertFrom.java
@@ -0,0 +1,43 @@
+/**
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromBIGINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BigIntVLongConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output BigIntHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    out.value = org.apache.drill.exec.util.ConvertUtil.HadoopWritables.readVLong(in.buffer, in.start, in.end);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java
new file mode 100644
index 0000000..4455781
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BigIntVLongConvertTo.java
@@ -0,0 +1,55 @@
+/**
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toBIGINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BigIntVLongConvertTo implements DrillSimpleFunc {
+
+  @Param BigIntHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    /* Hadoop Variable length integer (represented in the same way as a long)
+     * occupies between 1-9 bytes.
+     */
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(9);
+  }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.HadoopWritables.writeVLong(buffer, 0, 9, in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = buffer.readableBytes();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertFrom.java
new file mode 100644
index 0000000..f72d507
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertFrom.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromBOOLEAN_BYTE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BooleanByteConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output BitHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 1);
+
+    in.buffer.readerIndex(in.start);
+    out.value = in.buffer.readByte()==0 ? 0 : 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java
new file mode 100644
index 0000000..959deb0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/BooleanByteConvertTo.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toBOOLEAN_BYTE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class BooleanByteConvertTo implements DrillSimpleFunc {
+
+  @Param BitHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(1);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeByte(in.value==0 ? 0 : 1);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertFrom.java
new file mode 100644
index 0000000..75bf8d6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertFrom.java
@@ -0,0 +1,48 @@
+/**
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.DateHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromDATE_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DateEpochBEConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output DateHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 8);
+
+    in.buffer.readerIndex(in.start);
+    long epochMillis = Long.reverseBytes(in.buffer.readLong());
+    long millsOfDay = epochMillis % (24*3600*1000);
+    out.value = epochMillis - millsOfDay;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java
new file mode 100644
index 0000000..98b561c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochBEConvertTo.java
@@ -0,0 +1,53 @@
+/**
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.DateHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toDATE_EPOCH_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DateEpochBEConvertTo implements DrillSimpleFunc {
+
+  @Param DateHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(8);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeLong(Long.reverseBytes(in.value));
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertFrom.java
new file mode 100644
index 0000000..713016c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertFrom.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.DateHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromDATE_EPOCH", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DateEpochConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output DateHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 8);
+
+    in.buffer.readerIndex(in.start);
+    long epochMillis = in.buffer.readLong();
+    long millsOfDay = epochMillis % (24*3600*1000);
+    out.value = epochMillis - millsOfDay;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java
new file mode 100644
index 0000000..f0e604e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DateEpochConvertTo.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.DateHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toDATE_EPOCH", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DateEpochConvertTo implements DrillSimpleFunc {
+
+  @Param DateHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(8);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeLong(in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertFrom.java
new file mode 100644
index 0000000..df0bae7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertFrom.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromDOUBLE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DoubleConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output Float8Holder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 8);
+
+    in.buffer.readerIndex(in.start);
+    out.value = in.buffer.readDouble();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java
new file mode 100644
index 0000000..1f5c38f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DoubleConvertTo.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toDOUBLE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class DoubleConvertTo implements DrillSimpleFunc {
+
+  @Param Float8Holder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(8);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeDouble(in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertFrom.java
new file mode 100644
index 0000000..9f42b57
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertFrom.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.Float4Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromFLOAT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class FloatConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output Float4Holder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 4);
+
+    in.buffer.readerIndex(in.start);
+    out.value = in.buffer.readFloat();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java
new file mode 100644
index 0000000..a1d8c7c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/FloatConvertTo.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.Float4Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toFLOAT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class FloatConvertTo implements DrillSimpleFunc {
+
+  @Param Float4Holder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(4);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeFloat(in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 4;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertFrom.java
new file mode 100644
index 0000000..d235c66
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertFrom.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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class IntBEConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output IntHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 4);
+
+    in.buffer.readerIndex(in.start);
+    out.value = Integer.reverseBytes(in.buffer.readInt());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java
new file mode 100644
index 0000000..cd13c31
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntBEConvertTo.java
@@ -0,0 +1,53 @@
+/**
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.IntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toINT_BE", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class IntBEConvertTo implements DrillSimpleFunc {
+
+  @Param IntHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(4);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeInt(Integer.reverseBytes(in.value));
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 4;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertFrom.java
new file mode 100644
index 0000000..4563b8e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertFrom.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class IntConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output IntHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.checkBufferLength(in.buffer, in.start, in.end, 4);
+
+    in.buffer.readerIndex(in.start);
+    out.value = in.buffer.readInt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java
new file mode 100644
index 0000000..8531608
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntConvertTo.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.IntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toINT", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class IntConvertTo implements DrillSimpleFunc {
+
+  @Param IntHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(4);
+  }
+
+  @Override
+  public void eval() {
+    buffer.clear();
+    buffer.writeInt(in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = 4;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertFrom.java
new file mode 100644
index 0000000..acb2666
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertFrom.java
@@ -0,0 +1,43 @@
+/**
+ * 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.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_fromINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class IntVIntConvertFrom implements DrillSimpleFunc {
+
+  @Param VarBinaryHolder in;
+  @Output IntHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) { }
+
+  @Override
+  public void eval() {
+    out.value = org.apache.drill.exec.util.ConvertUtil.HadoopWritables.readVInt(in.buffer, in.start, in.end);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3a1e6c9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java
new file mode 100644
index 0000000..0691838
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/IntVIntConvertTo.java
@@ -0,0 +1,55 @@
+/**
+ * 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.impl.conv;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+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.IntHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@FunctionTemplate(name = "convert_toINT_HADOOPV", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+public class IntVIntConvertTo implements DrillSimpleFunc {
+
+  @Param IntHolder in;
+  @Output VarBinaryHolder out;
+  @Workspace ByteBuf buffer;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+    /* Hadoop Variable length integer (represented in the same way as a long)
+     * occupies between 1-9 bytes.
+     */
+    buffer = org.apache.drill.exec.util.ConvertUtil.createBuffer(9);
+  }
+
+  @Override
+  public void eval() {
+    org.apache.drill.exec.util.ConvertUtil.HadoopWritables.writeVLong(buffer, 0, 9, in.value);
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = buffer.readableBytes();
+  }
+}