You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/09/05 17:59:50 UTC

svn commit: r1622729 [3/8] - in /hive/trunk: ant/src/org/apache/hadoop/hive/ant/ ql/src/gen/vectorization/ExpressionTemplates/ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ ql/src/java...

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringGroupScalarCompareStringGroupColumnBase.txt Fri Sep  5 15:59:47 2014
@@ -0,0 +1,157 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public abstract class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  protected int colNum;
+  protected byte[] value;
+  protected int outputColumn;
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNull = outputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    long[] outVector = outputColVector.vector; 
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+   
+    outputColVector.isRepeating = false;
+    if (inputColVector.noNulls) {
+      outputColVector.noNulls = true;
+      if (inputColVector.isRepeating) {
+        outputColVector.isRepeating = true; 
+        if (StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <OperatorSymbol> 0) {
+          outVector[0] = 1;
+        } else {
+          outVector[0] = 0;
+        }
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      }
+    } else {
+      outputColVector.noNulls = false;
+      if (inputColVector.isRepeating) {
+        outputColVector.isRepeating = true;
+        outNull[0] = nullPos[0];
+        if (!nullPos[0]) {
+          if (StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <OperatorSymbol> 0) {
+            outVector[0] = 1;
+          } else {
+            outVector[0] = 0;
+          }
+        } 
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outNull[i] = nullPos[i];
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outNull[i] = nullPos[i];
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+  
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public byte[] getValue() {
+    return value;
+  }
+
+  public void setValue(byte[] value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}
\ No newline at end of file

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareStringGroupColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareStringGroupColumn.txt?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareStringGroupColumn.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareStringGroupColumn.txt Fri Sep  5 15:59:47 2014
@@ -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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.<BaseClassName>;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  private static final long serialVersionUID = 1L;
+
+  public <ClassName>(byte[] value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.STRING,
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/TruncStringScalarCompareStringGroupColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/TruncStringScalarCompareStringGroupColumn.txt?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/TruncStringScalarCompareStringGroupColumn.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/TruncStringScalarCompareStringGroupColumn.txt Fri Sep  5 15:59:47 2014
@@ -0,0 +1,62 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.<BaseClassName>;
+
+import org.apache.hadoop.hive.common.type.<TruncStringHiveType>;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  private static final long serialVersionUID = 1L;
+
+  public <ClassName>(<TruncStringHiveType> value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value.<TruncStringHiveGetBytes>;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<TruncStringTypeName>"),
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/AggregateDefinition.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/AggregateDefinition.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/AggregateDefinition.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/AggregateDefinition.java Fri Sep  5 15:59:47 2014
@@ -0,0 +1,44 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
+
+class AggregateDefinition {
+  private String name;
+  private VectorExpressionDescriptor.ArgumentType type;
+  private Class<? extends VectorAggregateExpression> aggClass;
+
+  AggregateDefinition(String name, VectorExpressionDescriptor.ArgumentType type, 
+            Class<? extends VectorAggregateExpression> aggClass) {
+    this.name = name;
+    this.type = type;
+    this.aggClass = aggClass;
+  }
+
+  String getName() {
+    return name;
+  }
+  VectorExpressionDescriptor.ArgumentType getType() {
+    return type;
+  }
+  Class<? extends VectorAggregateExpression> getAggClass() {
+    return aggClass;
+  }
+}
\ No newline at end of file

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java Fri Sep  5 15:59:47 2014
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hive.common.util.AnnotationUtils;
@@ -28,15 +30,46 @@ import org.apache.hive.common.util.Annot
  */
 public class VectorExpressionDescriptor {
 
+  private static final Log LOG = LogFactory.getLog(
+            VectorExpressionDescriptor.class.getName());
+
   final static int MAX_NUM_ARGUMENTS = 3;
 
+  //
+  // Special handling is needed at times for DATE, TIMESTAMP, (STRING), CHAR, and VARCHAR so they can
+  // be named specifically as argument types.
+  //
+  // LongColumnVector -->
+  //    INT_FAMILY
+  //    DATE
+  //    TIMESTAMP
+  //
+  // DoubleColumnVector -->
+  //    FLOAT_FAMILY
+  //
+  // DecimalColumnVector -->
+  //    DECIMAL
+  //
+  // BytesColumnVector -->
+  //    STRING
+  //    CHAR
+  //    VARCHAR
+  //
   public enum ArgumentType {
-    NONE(0),
-    LONG(1),
-    DOUBLE(2),
-    STRING(3),
-    DECIMAL(4),
-    ANY(7);
+    NONE                    (0x000),
+    INT_FAMILY              (0x001),
+    FLOAT_FAMILY            (0x002),
+    DECIMAL                 (0x004),
+    STRING                  (0x008),
+    CHAR                    (0x010),
+    VARCHAR                 (0x020),
+    STRING_FAMILY           (STRING.value | CHAR.value | VARCHAR.value),
+    DATE                    (0x040),
+    TIMESTAMP               (0x080),
+    DATETIME_FAMILY         (DATE.value | TIMESTAMP.value),
+    INT_DATETIME_FAMILY     (INT_FAMILY.value | DATETIME_FAMILY.value),
+    STRING_DATETIME_FAMILY  (STRING_FAMILY.value | DATETIME_FAMILY.value),
+    ALL_FAMILY              (0xFFF);
 
     private final int value;
 
@@ -48,12 +81,79 @@ public class VectorExpressionDescriptor 
       return value;
     }
 
+    public static ArgumentType fromHiveTypeName(String hiveTypeName) {
+      String lower = hiveTypeName.toLowerCase();
+      if (lower.equals("tinyint") ||
+          lower.equals("smallint") ||
+          lower.equals("int") ||
+          lower.equals("bigint") ||
+          lower.equals("boolean") ||
+          lower.equals("long")) {
+        return INT_FAMILY;
+      } else if (lower.equals("double") || lower.equals("float")) {
+        return FLOAT_FAMILY;
+      } else if (lower.equals("string")) {
+        return STRING;
+      } else if (VectorizationContext.charTypePattern.matcher(lower).matches()) {
+        return CHAR;
+      } else if (VectorizationContext.varcharTypePattern.matcher(lower).matches()) {
+        return VARCHAR;
+      } else if (VectorizationContext.decimalTypePattern.matcher(lower).matches()) {
+        return DECIMAL;
+      } else if (lower.equals("timestamp")) {
+        return TIMESTAMP;
+      } else if (lower.equals("date")) {
+        return DATE;
+      } else if (lower.equals("void")) {
+        // The old code let void through...
+        return INT_FAMILY;
+      } else {
+        return NONE;
+      }
+    }
+
     public static ArgumentType getType(String inType) {
-      String type = VectorizationContext.getNormalizedTypeName(inType);
-      if (VectorizationContext.decimalTypePattern.matcher(type).matches()) {
-        type = "decimal";
+      if (inType.equalsIgnoreCase("long")) {
+        // A synonym in some places in the code...
+        return INT_FAMILY;
+      } else if (inType.equalsIgnoreCase("double")) {
+        // A synonym in some places in the code...
+        return FLOAT_FAMILY;
+      } else if (VectorizationContext.decimalTypePattern.matcher(inType).matches()) {
+        return DECIMAL;
+      } else if (VectorizationContext.charTypePattern.matcher(inType).matches()) {
+        return CHAR;
+      } else if (VectorizationContext.varcharTypePattern.matcher(inType).matches()) {
+        return VARCHAR;
+      }
+      return valueOf(inType.toUpperCase());
+    }
+
+    public boolean isSameTypeOrFamily(ArgumentType other) {
+      return ((value & other.value) != 0);
+    }
+
+    public static String getVectorColumnSimpleName(ArgumentType argType) {
+      if (argType == INT_FAMILY ||
+          argType == DATE ||
+          argType == TIMESTAMP) {
+        return "Long";
+      } else if (argType == FLOAT_FAMILY) {
+        return "Double";
+      } else if (argType == DECIMAL) {
+        return "Decimal";
+      } else if (argType == STRING ||
+                 argType == CHAR ||
+                 argType == VARCHAR) {
+        return "String";
+      } else {
+        return "None";
       }
-      return valueOf(type.toUpperCase());
+    }
+
+    public static String getVectorColumnSimpleName(String hiveTypeName) {
+      ArgumentType argType = fromHiveTypeName(hiveTypeName);
+      return getVectorColumnSimpleName(argType);
     }
   }
 
@@ -162,15 +262,12 @@ public class VectorExpressionDescriptor 
    */
   public static final class Descriptor {
 
-    @Override
-    public boolean equals(Object o) {
-      Descriptor other = (Descriptor) o;
+    public boolean matches(Descriptor other) {
       if (!mode.equals(other.mode) || (argCount != other.argCount) ) {
         return false;
       }
       for (int i = 0; i < argCount; i++) {
-        if (!argTypes[i].equals(other.argTypes[i]) && (!argTypes[i].equals(ArgumentType.ANY) &&
-            !other.argTypes[i].equals(ArgumentType.ANY))) {
+        if (!argTypes[i].isSameTypeOrFamily(other.argTypes[i])) {
           return false;
         }
         if (!exprTypes[i].equals(other.exprTypes[i])) {
@@ -228,13 +325,23 @@ public class VectorExpressionDescriptor 
     Class<? extends VectorExpression>[] list = annotation.value();
     for (Class<? extends VectorExpression> ve : list) {
       try {
-        if (ve.newInstance().getDescriptor().equals(descriptor)) {
+        if (ve.newInstance().getDescriptor().matches(descriptor)) {
           return ve;
         }
       } catch (Exception ex) {
         throw new HiveException(ex);
       }
     }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("getVectorExpressionClass udf " + udf.getSimpleName() + " descriptor: " + descriptor.toString());
+      for (Class<? extends VectorExpression> ve : list) {
+        try {
+          LOG.debug("getVectorExpressionClass doesn't match " + ve.getSimpleName() + " " + ve.newInstance().getDescriptor().toString());
+        } catch (Exception ex) {
+          throw new HiveException(ex);
+        }
+      }
+    }
     return null;
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java Fri Sep  5 15:59:47 2014
@@ -33,7 +33,9 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
@@ -97,11 +99,13 @@ import org.apache.hadoop.hive.ql.udf.gen
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Context class for vectorization execution.
@@ -123,6 +127,15 @@ public class VectorizationContext {
   public static final Pattern decimalTypePattern = Pattern.compile("decimal.*",
       Pattern.CASE_INSENSITIVE);
 
+  public static final Pattern charTypePattern = Pattern.compile("char.*",
+	      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern varcharTypePattern = Pattern.compile("varchar.*",
+	      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern charVarcharTypePattern = Pattern.compile("char.*|varchar.*",
+      Pattern.CASE_INSENSITIVE);
+
   //Map column number to type
   private final OutputColumnManager ocm;
 
@@ -210,14 +223,17 @@ public class VectorizationContext {
 
     private final Set<Integer> usedOutputColumns = new HashSet<Integer>();
 
-    int allocateOutputColumn(String columnType) {
-      if (initialOutputCol < 0) {
-        // This is a test
-        return 0;
+    int allocateOutputColumn(String hiveTypeName) {
+        if (initialOutputCol < 0) {
+          // This is a test
+          return 0;
+        }
+
+        // We need to differentiate DECIMAL columns by their precision and scale...
+        String normalizedTypeName = getNormalizedName(hiveTypeName);
+        int relativeCol = allocateOutputColumnInternal(normalizedTypeName);
+        return initialOutputCol + relativeCol;
       }
-      int relativeCol = allocateOutputColumnInternal(columnType);
-      return initialOutputCol + relativeCol;
-    }
 
     private int allocateOutputColumnInternal(String columnType) {
       for (int i = 0; i < outputColCount; i++) {
@@ -548,6 +564,12 @@ public class VectorizationContext {
       case STRING:
         udfClass = new UDFToString();
         break;
+      case CHAR:
+        genericUdf = new GenericUDFToChar();
+        break;
+      case VARCHAR:
+        genericUdf = new GenericUDFToVarchar();
+        break;
       case BOOLEAN:
         udfClass = new UDFToBoolean();
         break;
@@ -592,15 +614,15 @@ public class VectorizationContext {
       Class<? extends UDF> udfClass = bridge.getUdfClass();
       if (udfClass.equals(UDFHex.class)
           || udfClass.equals(UDFConv.class)
-          || isCastToIntFamily(udfClass) && arg0Type(expr).equals("string")
-          || isCastToFloatFamily(udfClass) && arg0Type(expr).equals("string")
+          || isCastToIntFamily(udfClass) && isStringFamily(arg0Type(expr))
+          || isCastToFloatFamily(udfClass) && isStringFamily(arg0Type(expr))
           || udfClass.equals(UDFToString.class) &&
                (arg0Type(expr).equals("timestamp")
                    || arg0Type(expr).equals("double")
                    || arg0Type(expr).equals("float"))) {
         return true;
       }
-    } else if ((gudf instanceof GenericUDFTimestamp && arg0Type(expr).equals("string"))
+    } else if ((gudf instanceof GenericUDFTimestamp && isStringFamily(arg0Type(expr)))
 
             /* GenericUDFCase and GenericUDFWhen are implemented with the UDF Adaptor because
              * of their complexity and generality. In the future, variations of these
@@ -615,6 +637,16 @@ public class VectorizationContext {
             || gudf instanceof GenericUDFCase
             || gudf instanceof GenericUDFWhen) {
       return true;
+    } else if (gudf instanceof GenericUDFToChar &&
+               (arg0Type(expr).equals("timestamp")
+                   || arg0Type(expr).equals("double")
+                   || arg0Type(expr).equals("float"))) {
+      return true;
+    } else if (gudf instanceof GenericUDFToVarchar &&
+            (arg0Type(expr).equals("timestamp")
+                || arg0Type(expr).equals("double")
+                || arg0Type(expr).equals("float"))) {
+      return true;
     }
     return false;
   }
@@ -721,27 +753,21 @@ public class VectorizationContext {
 
   private VectorExpression getConstantVectorExpression(Object constantValue, TypeInfo typeInfo,
       Mode mode) throws HiveException {
-    String type =  typeInfo.getTypeName();
-    String colVectorType = getNormalizedTypeName(type);
+    String typeName =  typeInfo.getTypeName();
+    VectorExpressionDescriptor.ArgumentType vectorArgType = VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(typeName);
+    if (vectorArgType == VectorExpressionDescriptor.ArgumentType.NONE) {
+      throw new HiveException("No vector argument type for type name " + typeName);
+    }
     int outCol = -1;
     if (mode == Mode.PROJECTION) {
-      outCol = ocm.allocateOutputColumn(colVectorType);
+      outCol = ocm.allocateOutputColumn(typeName);
     }
     if (constantValue == null) {
-      return new ConstantVectorExpression(outCol, type, true);
-    } else if (decimalTypePattern.matcher(type).matches()) {
-      VectorExpression ve = new ConstantVectorExpression(outCol, (Decimal128) constantValue);
-      ve.setOutputType(typeInfo.getTypeName());
-      return ve;
-    } else if (type.equalsIgnoreCase("long") || type.equalsIgnoreCase("int") ||
-        type.equalsIgnoreCase("short") || type.equalsIgnoreCase("byte")) {
-      return new ConstantVectorExpression(outCol,
-          ((Number) constantValue).longValue());
-    } else if (type.equalsIgnoreCase("double") || type.equalsIgnoreCase("float")) {
-      return new ConstantVectorExpression(outCol, ((Number) constantValue).doubleValue());
-    } else if (type.equalsIgnoreCase("string")) {
-      return new ConstantVectorExpression(outCol, ((String) constantValue).getBytes());
-    } else if (type.equalsIgnoreCase("boolean")) {
+      return new ConstantVectorExpression(outCol, typeName, true);
+    }
+
+    // Boolean is special case.
+    if (typeName.equalsIgnoreCase("boolean")) {
       if (mode == Mode.FILTER) {
         if (((Boolean) constantValue).booleanValue()) {
           return new FilterConstantBooleanVectorExpression(1);
@@ -756,7 +782,26 @@ public class VectorizationContext {
         }
       }
     }
-    throw new HiveException("Unsupported constant type: "+type.toString());
+
+    switch (vectorArgType) {
+    case INT_FAMILY:
+      return new ConstantVectorExpression(outCol, ((Number) constantValue).longValue());
+    case FLOAT_FAMILY:
+      return new ConstantVectorExpression(outCol, ((Number) constantValue).doubleValue());
+    case DECIMAL:
+      VectorExpression ve = new ConstantVectorExpression(outCol, (Decimal128) constantValue);
+      // Set type name with decimal precision, scale, etc.
+      ve.setOutputType(typeName);
+      return ve;
+    case STRING:
+      return new ConstantVectorExpression(outCol, ((String) constantValue).getBytes());
+    case CHAR:
+      return new ConstantVectorExpression(outCol, ((HiveChar) constantValue));
+    case VARCHAR:
+      return new ConstantVectorExpression(outCol, ((HiveVarchar) constantValue));
+    default:
+      throw new HiveException("Unsupported constant type: " + typeName);
+    }
   }
 
   /**
@@ -799,7 +844,15 @@ public class VectorizationContext {
     builder.setMode(mode);
     for (int i = 0; i < numChildren; i++) {
       ExprNodeDesc child = childExpr.get(i);
-      builder.setArgumentType(i, child.getTypeString());
+      String childTypeString = child.getTypeString();
+      if (childTypeString == null) {
+        throw new HiveException("Null child type name string");
+      }
+      String undecoratedTypeName = getUndecoratedName(childTypeString);
+      if (undecoratedTypeName == null) {
+        throw new HiveException("No match for type string " + childTypeString + " from undecorated type name method");
+      }
+      builder.setArgumentType(i, undecoratedTypeName);
       if ((child instanceof ExprNodeGenericFuncDesc) || (child instanceof ExprNodeColumnDesc)) {
         builder.setInputExpressionType(i, InputExpressionType.COLUMN);
       } else if (child instanceof ExprNodeConstantDesc) {
@@ -829,7 +882,11 @@ public class VectorizationContext {
     try {
       for (int i = 0; i < numChildren; i++) {
         ExprNodeDesc child = childExpr.get(i);
-        inputTypes[i] = VectorExpression.Type.getValue(child.getTypeInfo().getTypeName());
+        String undecoratedName = getUndecoratedName(child.getTypeInfo().getTypeName());
+        inputTypes[i] = VectorExpression.Type.getValue(undecoratedName);
+        if (inputTypes[i] == VectorExpression.Type.OTHER){
+          throw new HiveException("No vector type for " + vectorClass.getSimpleName() + " argument #" + i + " type name " + undecoratedName);
+        }
         if (child instanceof ExprNodeGenericFuncDesc) {
           VectorExpression vChild = getVectorExpression(child, childrenMode);
             children.add(vChild);
@@ -870,36 +927,71 @@ public class VectorizationContext {
     return Mode.PROJECTION;
   }
 
+  private String getNewInstanceArgumentString(Object [] args) {
+    if (args == null) {
+      return "arguments: NULL";
+    }
+    ArrayList<String> argClasses = new ArrayList<String>();
+    for (Object obj : args) {
+      argClasses.add(obj.getClass().getSimpleName());
+    }
+    return "arguments: " + Arrays.toString(args) + ", argument classes: " + argClasses.toString();
+  }
+
   private VectorExpression instantiateExpression(Class<?> vclass, TypeInfo returnType, Object...args)
       throws HiveException {
     VectorExpression ve = null;
     Constructor<?> ctor = getConstructor(vclass);
     int numParams = ctor.getParameterTypes().length;
     int argsLength = (args == null) ? 0 : args.length;
-    try {
-      if (numParams == 0) {
+    if (numParams == 0) {
+      try {
         ve = (VectorExpression) ctor.newInstance();
-      } else if (numParams == argsLength) {
+      } catch (Exception ex) {
+        throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with 0 arguments, exception: " +
+                    StringUtils.stringifyException(ex));
+      }
+    } else if (numParams == argsLength) {
+      try {
         ve = (VectorExpression) ctor.newInstance(args);
-      } else if (numParams == argsLength + 1) {
-        // Additional argument is needed, which is the outputcolumn.
+      } catch (Exception ex) {
+          throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with " + getNewInstanceArgumentString(args) + ", exception: " +
+                      StringUtils.stringifyException(ex));
+      }
+    } else if (numParams == argsLength + 1) {
+      // Additional argument is needed, which is the outputcolumn.
+      Object [] newArgs = null;
+      try {
         String outType;
 
         // Special handling for decimal because decimal types need scale and precision parameter.
         // This special handling should be avoided by using returnType uniformly for all cases.
         if (returnType != null) {
-          outType = getNormalizedTypeName(returnType.getTypeName()).toLowerCase();
+          outType = getNormalizedName(returnType.getTypeName()).toLowerCase();
+          if (outType == null) {
+           throw new HiveException("No vector type for type name " + returnType);
+          }
         } else {
           outType = ((VectorExpression) vclass.newInstance()).getOutputType();
         }
         int outputCol = ocm.allocateOutputColumn(outType);
-        Object [] newArgs = Arrays.copyOf(args, numParams);
+        newArgs = Arrays.copyOf(args, numParams);
         newArgs[numParams-1] = outputCol;
+
         ve = (VectorExpression) ctor.newInstance(newArgs);
         ve.setOutputType(outType);
+      } catch (Exception ex) {
+          throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with arguments " + getNewInstanceArgumentString(newArgs) + ", exception: " +
+                      StringUtils.stringifyException(ex));
+      }
+    }
+    // Add maxLength parameter to UDFs that have CHAR or VARCHAR output.
+    if (ve instanceof TruncStringOutput) {
+      TruncStringOutput truncStringOutput = (TruncStringOutput) ve;
+      if (returnType instanceof BaseCharTypeInfo) {
+        BaseCharTypeInfo baseCharTypeInfo = (BaseCharTypeInfo) returnType;
+        truncStringOutput.setMaxLength(baseCharTypeInfo.getLength());
       }
-    } catch (Exception ex) {
-      throw new HiveException("Could not instantiate " + vclass.getSimpleName(), ex);
     }
     return ve;
   }
@@ -933,6 +1025,10 @@ public class VectorizationContext {
       }
     } else if (udf instanceof GenericUDFToDecimal) {
       return getCastToDecimal(childExpr, returnType);
+    } else if (udf instanceof GenericUDFToChar) {
+      return getCastToChar(childExpr, returnType);
+    } else if (udf instanceof GenericUDFToVarchar) {
+      return getCastToVarChar(childExpr, returnType);
     }
 
     // Now do a general lookup
@@ -962,7 +1058,7 @@ public class VectorizationContext {
         inputColumns[i++] = ve.getOutputColumn();
       }
 
-      int outColumn = ocm.allocateOutputColumn(getNormalizedTypeName(returnType.getTypeName()));
+      int outColumn = ocm.allocateOutputColumn(returnType.getTypeName());
       VectorCoalesce vectorCoalesce = new VectorCoalesce(inputColumns, outColumn);
       vectorCoalesce.setOutputType(returnType.getTypeName());
       vectorCoalesce.setChildExpressions(vectorChildren);
@@ -989,7 +1085,7 @@ public class VectorizationContext {
         inputColumns[i++] = ve.getOutputColumn();
       }
 
-      int outColumn = ocm.allocateOutputColumn(getNormalizedTypeName(returnType.getTypeName()));
+      int outColumn = ocm.allocateOutputColumn(returnType.getTypeName());
       VectorElt vectorElt = new VectorElt(inputColumns, outColumn);
       vectorElt.setOutputType(returnType.getTypeName());
       vectorElt.setChildExpressions(vectorChildren);
@@ -1265,6 +1361,64 @@ public class VectorizationContext {
     throw new HiveException("Unhandled cast input type: " + inputType);
   }
 
+  private VectorExpression getCastToChar(List<ExprNodeDesc> childExpr, TypeInfo returnType)
+      throws HiveException {
+    ExprNodeDesc child = childExpr.get(0);
+    String inputType = childExpr.get(0).getTypeString();
+    if (child instanceof ExprNodeConstantDesc) {
+      // Don't do constant folding here.  Wait until the optimizer is changed to do it.
+      // Family of related JIRAs: HIVE-7421, HIVE-7422, and HIVE-7424.
+      return null;
+    }
+    if (inputType.equals("boolean")) {
+      // Boolean must come before the integer family. It's a special case.
+      return createVectorExpression(CastBooleanToCharViaLongToChar.class, childExpr, Mode.PROJECTION, null);
+    } else if (isIntFamily(inputType)) {
+      return createVectorExpression(CastLongToChar.class, childExpr, Mode.PROJECTION, null);
+    } else if (isDecimalFamily(inputType)) {
+      return createVectorExpression(CastDecimalToChar.class, childExpr, Mode.PROJECTION, returnType);
+    } else if (isDateFamily(inputType)) {
+      return createVectorExpression(CastDateToChar.class, childExpr, Mode.PROJECTION, returnType);
+    } else if (isStringFamily(inputType)) {
+      return createVectorExpression(CastStringGroupToChar.class, childExpr, Mode.PROJECTION, returnType);
+    }
+
+    /* 
+     * Timestamp, float, and double types are handled by the legacy code path. See isLegacyPathUDF.
+     */
+
+    throw new HiveException("Unhandled cast input type: " + inputType);
+  }
+
+  private VectorExpression getCastToVarChar(List<ExprNodeDesc> childExpr, TypeInfo returnType)
+      throws HiveException {
+    ExprNodeDesc child = childExpr.get(0);
+    String inputType = childExpr.get(0).getTypeString();
+    if (child instanceof ExprNodeConstantDesc) {
+      // Don't do constant folding here.  Wait until the optimizer is changed to do it.
+      // Family of related JIRAs: HIVE-7421, HIVE-7422, and HIVE-7424.
+      return null;
+    }
+    if (inputType.equals("boolean")) {
+      // Boolean must come before the integer family. It's a special case.
+      return createVectorExpression(CastBooleanToVarCharViaLongToVarChar.class, childExpr, Mode.PROJECTION, null);
+    } else if (isIntFamily(inputType)) {
+      return createVectorExpression(CastLongToVarChar.class, childExpr, Mode.PROJECTION, null);
+    } else if (isDecimalFamily(inputType)) {
+      return createVectorExpression(CastDecimalToVarChar.class, childExpr, Mode.PROJECTION, returnType);
+    } else if (isDateFamily(inputType)) {
+      return createVectorExpression(CastDateToVarChar.class, childExpr, Mode.PROJECTION, returnType);
+    } else if (isStringFamily(inputType)) {
+      return createVectorExpression(CastStringGroupToVarChar.class, childExpr, Mode.PROJECTION, returnType);
+    }
+
+    /*
+     * Timestamp, float, and double types are handled by the legacy code path. See isLegacyPathUDF.
+     */
+
+    throw new HiveException("Unhandled cast input type: " + inputType);
+  }
+
   private VectorExpression getCastToDoubleExpression(Class<?> udf, List<ExprNodeDesc> childExpr,
       TypeInfo returnType) throws HiveException {
     ExprNodeDesc child = childExpr.get(0);
@@ -1304,12 +1458,12 @@ public class VectorizationContext {
       return getConstantVectorExpression(null, TypeInfoFactory.booleanTypeInfo, Mode.PROJECTION);
     }
     // Long and double are handled using descriptors, string needs to be specially handled.
-    if (inputType.equals("string")) {
+    if (isStringFamily(inputType)) {
       // string casts to false if it is 0 characters long, otherwise true
       VectorExpression lenExpr = createVectorExpression(StringLength.class, childExpr,
           Mode.PROJECTION, null);
 
-      int outputCol = ocm.allocateOutputColumn("integer");
+      int outputCol = ocm.allocateOutputColumn("Long");
       VectorExpression lenToBoolExpr =
           new CastLongToBooleanViaLongToLong(lenExpr.getOutputColumn(), outputCol);
       lenToBoolExpr.setChildExpressions(new VectorExpression[] {lenExpr});
@@ -1411,6 +1565,14 @@ public class VectorizationContext {
       cl = FilterStringColumnBetween.class;
     } else if (colType.equals("string") && notKeywordPresent) {
       cl = FilterStringColumnNotBetween.class;
+    } else if (varcharTypePattern.matcher(colType).matches() && !notKeywordPresent) {
+      cl = FilterVarCharColumnBetween.class;
+    } else if (varcharTypePattern.matcher(colType).matches() && notKeywordPresent) {
+      cl = FilterVarCharColumnNotBetween.class;
+    } else if (charTypePattern.matcher(colType).matches() && !notKeywordPresent) {
+      cl = FilterCharColumnBetween.class;
+    } else if (charTypePattern.matcher(colType).matches() && notKeywordPresent) {
+      cl = FilterCharColumnNotBetween.class;
     } else if (colType.equals("timestamp")) {
 
       // Get timestamp boundary values as longs instead of the expected strings
@@ -1483,13 +1645,13 @@ public class VectorizationContext {
 
     // Allocate output column and get column number;
     int outputCol = -1;
-    String resultType = expr.getTypeInfo().getTypeName();
-    String resultColVectorType = getNormalizedTypeName(resultType);
+    String resultTypeName = expr.getTypeInfo().getTypeName();
 
-    outputCol = ocm.allocateOutputColumn(resultColVectorType);
+    outputCol = ocm.allocateOutputColumn(resultTypeName);
 
     // Make vectorized operator
-    VectorExpression ve = new VectorUDFAdaptor(expr, outputCol, resultColVectorType, argDescs);
+    String normalizedName = getNormalizedName(resultTypeName);
+    VectorExpression ve = new VectorUDFAdaptor(expr, outputCol, normalizedName, argDescs);
 
     // Set child expressions
     VectorExpression[] childVEs = null;
@@ -1509,7 +1671,7 @@ public class VectorizationContext {
   }
 
   public static boolean isStringFamily(String resultType) {
-    return resultType.equalsIgnoreCase("string");
+    return resultType.equalsIgnoreCase("string") || charVarcharTypePattern.matcher(resultType).matches();
   }
 
   public static boolean isDatetimeFamily(String resultType) {
@@ -1617,7 +1779,7 @@ public class VectorizationContext {
           "Non-constant argument not supported for vectorization.");
     }
     ExprNodeConstantDesc constExpr = (ExprNodeConstantDesc) expr;
-    if (constExpr.getTypeString().equals("string")) {
+    if (isStringFamily(constExpr.getTypeString())) {
 
       // create expression tree with type cast from string to timestamp
       ExprNodeGenericFuncDesc expr2 = new ExprNodeGenericFuncDesc();
@@ -1667,63 +1829,99 @@ public class VectorizationContext {
     }
   }
 
-  static String getNormalizedTypeName(String colType){
-    String normalizedType = null;
-    if (colType.equalsIgnoreCase("Double") || colType.equalsIgnoreCase("Float")) {
-      normalizedType = "Double";
-    } else if (colType.equalsIgnoreCase("String")) {
-      normalizedType = "String";
-   } else if (decimalTypePattern.matcher(colType).matches()) {
+  static String getNormalizedName(String hiveTypeName) {
+    VectorExpressionDescriptor.ArgumentType argType = VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(hiveTypeName);
+    switch (argType) {
+    case INT_FAMILY:
+      return "Long";
+    case FLOAT_FAMILY:
+      return "Double";
+    case DECIMAL:
       //Return the decimal type as is, it includes scale and precision.
-      normalizedType = colType;
-    } else {
-      normalizedType = "Long";
+      return hiveTypeName;
+    case STRING:
+      return "String";
+    case CHAR:
+      //Return the CHAR type as is, it includes maximum length
+      return hiveTypeName;
+    case VARCHAR:
+      //Return the VARCHAR type as is, it includes maximum length.
+      return hiveTypeName;
+    case DATE:
+      return "Date";
+    case TIMESTAMP:
+      return "Timestamp";
+    default:
+      return "None";
+    }
+  }
+  
+  static String getUndecoratedName(String hiveTypeName) {
+    VectorExpressionDescriptor.ArgumentType argType = VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(hiveTypeName);
+    switch (argType) {
+    case INT_FAMILY:
+      return "Long";
+    case FLOAT_FAMILY:
+      return "Double";
+    case DECIMAL:
+      return "Decimal";
+    case STRING:
+      return "String";
+    case CHAR:
+      return "Char";
+    case VARCHAR:
+      return "VarChar";
+    case DATE:
+      return "Date";
+    case TIMESTAMP:
+      return "Timestamp";
+    default:
+      return "None";
     }
-    return normalizedType;
   }
 
-  static Object[][] aggregatesDefinition = {
-    {"min",       "Long",   VectorUDAFMinLong.class},
-    {"min",       "Double", VectorUDAFMinDouble.class},
-    {"min",       "String", VectorUDAFMinString.class},
-    {"min",       "Decimal",VectorUDAFMinDecimal.class},
-    {"max",       "Long",   VectorUDAFMaxLong.class},
-    {"max",       "Double", VectorUDAFMaxDouble.class},
-    {"max",       "String", VectorUDAFMaxString.class},
-    {"max",       "Decimal",VectorUDAFMaxDecimal.class},
-    {"count",     null,     VectorUDAFCountStar.class},
-    {"count",     "Long",   VectorUDAFCount.class},
-    {"count",     "Double", VectorUDAFCount.class},
-    {"count",     "String", VectorUDAFCount.class},
-    {"count",     "Decimal",VectorUDAFCount.class},
-    {"sum",       "Long",   VectorUDAFSumLong.class},
-    {"sum",       "Double", VectorUDAFSumDouble.class},
-    {"sum",       "Decimal",VectorUDAFSumDecimal.class},
-    {"avg",       "Long",   VectorUDAFAvgLong.class},
-    {"avg",       "Double", VectorUDAFAvgDouble.class},
-    {"avg",       "Decimal",VectorUDAFAvgDecimal.class},
-    {"variance",  "Long",   VectorUDAFVarPopLong.class},
-    {"var_pop",   "Long",   VectorUDAFVarPopLong.class},
-    {"variance",  "Double", VectorUDAFVarPopDouble.class},
-    {"var_pop",   "Double", VectorUDAFVarPopDouble.class},
-    {"variance",  "Decimal",VectorUDAFVarPopDecimal.class},
-    {"var_pop",   "Decimal",VectorUDAFVarPopDecimal.class},
-    {"var_samp",  "Long",   VectorUDAFVarSampLong.class},
-    {"var_samp" , "Double", VectorUDAFVarSampDouble.class},
-    {"var_samp" , "Decimal",VectorUDAFVarSampDecimal.class},
-    {"std",       "Long",   VectorUDAFStdPopLong.class},
-    {"stddev",    "Long",   VectorUDAFStdPopLong.class},
-    {"stddev_pop","Long",   VectorUDAFStdPopLong.class},
-    {"std",       "Double", VectorUDAFStdPopDouble.class},
-    {"stddev",    "Double", VectorUDAFStdPopDouble.class},
-    {"stddev_pop","Double", VectorUDAFStdPopDouble.class},
-    {"std",       "Decimal",VectorUDAFStdPopDecimal.class},
-    {"stddev",    "Decimal",VectorUDAFStdPopDecimal.class},
-    {"stddev_pop","Decimal",VectorUDAFStdPopDecimal.class},
-    {"stddev_samp","Long",  VectorUDAFStdSampLong.class},
-    {"stddev_samp","Double",VectorUDAFStdSampDouble.class},
-    {"stddev_samp","Decimal",VectorUDAFStdSampDecimal.class},
-  };
+  static ArrayList<AggregateDefinition> aggregatesDefinition = new ArrayList<AggregateDefinition>() {{
+    add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFMinLong.class));
+    add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFMinDouble.class));
+    add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.STRING_FAMILY, VectorUDAFMinString.class));
+    add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFMinDecimal.class));
+    add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFMaxLong.class));
+    add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFMaxDouble.class));
+    add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.STRING_FAMILY, VectorUDAFMaxString.class));
+    add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFMaxDecimal.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.NONE,          VectorUDAFCountStar.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFCount.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFCount.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.STRING_FAMILY, VectorUDAFCount.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFCount.class));
+    add(new AggregateDefinition("sum",         VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFSumLong.class));
+    add(new AggregateDefinition("sum",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFSumDouble.class));
+    add(new AggregateDefinition("sum",         VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFSumDecimal.class));
+    add(new AggregateDefinition("avg",         VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFAvgLong.class));
+    add(new AggregateDefinition("avg",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFAvgDouble.class));
+    add(new AggregateDefinition("avg",         VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFAvgDecimal.class));
+    add(new AggregateDefinition("variance",    VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFVarPopLong.class));
+    add(new AggregateDefinition("var_pop",     VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFVarPopLong.class));
+    add(new AggregateDefinition("variance",    VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFVarPopDouble.class));
+    add(new AggregateDefinition("var_pop",     VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFVarPopDouble.class));
+    add(new AggregateDefinition("variance",    VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFVarPopDecimal.class));
+    add(new AggregateDefinition("var_pop",     VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFVarPopDecimal.class));
+    add(new AggregateDefinition("var_samp",    VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFVarSampLong.class));
+    add(new AggregateDefinition("var_samp" ,   VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFVarSampDouble.class));
+    add(new AggregateDefinition("var_samp" ,   VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFVarSampDecimal.class));
+    add(new AggregateDefinition("std",         VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFStdPopLong.class));
+    add(new AggregateDefinition("stddev",      VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFStdPopLong.class));
+    add(new AggregateDefinition("stddev_pop",  VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFStdPopLong.class));
+    add(new AggregateDefinition("std",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFStdPopDouble.class));
+    add(new AggregateDefinition("stddev",      VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFStdPopDouble.class));
+    add(new AggregateDefinition("stddev_pop",  VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFStdPopDouble.class));
+    add(new AggregateDefinition("std",         VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFStdPopDecimal.class));
+    add(new AggregateDefinition("stddev",      VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFStdPopDecimal.class));
+    add(new AggregateDefinition("stddev_pop",  VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFStdPopDecimal.class));
+    add(new AggregateDefinition("stddev_samp", VectorExpressionDescriptor.ArgumentType.INT_FAMILY,    VectorUDAFStdSampLong.class));
+    add(new AggregateDefinition("stddev_samp", VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,  VectorUDAFStdSampDouble.class));
+    add(new AggregateDefinition("stddev_samp", VectorExpressionDescriptor.ArgumentType.DECIMAL,       VectorUDAFStdSampDecimal.class));
+  }};
 
   public VectorAggregateExpression getAggregatorExpression(AggregationDesc desc)
       throws HiveException {
@@ -1737,22 +1935,22 @@ public class VectorizationContext {
     }
 
     String aggregateName = desc.getGenericUDAFName();
-    String inputType = null;
+    VectorExpressionDescriptor.ArgumentType inputType = VectorExpressionDescriptor.ArgumentType.NONE;
 
     if (paramDescList.size() > 0) {
       ExprNodeDesc inputExpr = paramDescList.get(0);
-      inputType = getNormalizedTypeName(inputExpr.getTypeString());
-      if (decimalTypePattern.matcher(inputType).matches()) {
-        inputType = "Decimal";
+      inputType = VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(inputExpr.getTypeString());
+      if (inputType == VectorExpressionDescriptor.ArgumentType.NONE) {
+        throw new HiveException("No vector argument type for Hive type name " + inputExpr.getTypeString());
       }
     }
 
-    for (Object[] aggDef : aggregatesDefinition) {
-      if (aggregateName.equalsIgnoreCase((String) aggDef[0]) &&
-          ((aggDef[1] == null && inputType == null) ||
-          (aggDef[1] != null && aggDef[1].equals(inputType)))) {
-        Class<? extends VectorAggregateExpression> aggClass =
-            (Class<? extends VectorAggregateExpression>) (aggDef[2]);
+    for (AggregateDefinition aggDef : aggregatesDefinition) {
+      if (aggregateName.equalsIgnoreCase(aggDef.getName()) &&
+          ((aggDef.getType() == VectorExpressionDescriptor.ArgumentType.NONE &&
+            inputType == VectorExpressionDescriptor.ArgumentType.NONE) ||
+          (aggDef.getType().isSameTypeOrFamily(inputType)))) {
+        Class<? extends VectorAggregateExpression> aggClass = aggDef.getAggClass();
         try
         {
           Constructor<? extends VectorAggregateExpression> ctor =
@@ -1769,7 +1967,7 @@ public class VectorizationContext {
     }
 
     throw new HiveException("Vector aggregate not implemented: \"" + aggregateName +
-        "\" for type: \"" + inputType + "");
+        "\" for type: \"" + inputType.name() + "");
   }
 
   public Map<Integer, String> getOutputColumnTypeMap() {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java Fri Sep  5 15:59:47 2014
@@ -23,11 +23,16 @@ import java.sql.Timestamp;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -126,6 +131,8 @@ public class VectorizedBatchUtil {
           break;
         case BINARY:
         case STRING:
+        case CHAR:
+        case VARCHAR:
           cvList.add(new BytesColumnVector(VectorizedRowBatch.DEFAULT_SIZE));
           break;
         case DECIMAL:
@@ -375,6 +382,51 @@ public class VectorizedBatchUtil {
         }
       }
         break;
+      case CHAR: {
+        BytesColumnVector bcv = (BytesColumnVector) batch.cols[off + i];
+        if (writableCol != null) {
+          bcv.isNull[rowIndex] = false;
+          HiveChar colHiveChar = ((HiveCharWritable) writableCol).getHiveChar();
+          byte[] bytes = colHiveChar.getStrippedValue().getBytes();
+          
+          // We assume the CHAR maximum length was enforced when the object was created.
+          int length = bytes.length;
+
+          int start = buffer.getLength();
+          try {
+            // In vector mode, we store CHAR as unpadded.
+            buffer.write(bytes, 0, length);
+          } catch (IOException ioe) {
+            throw new IllegalStateException("bad write", ioe);
+          }
+          bcv.setRef(rowIndex, buffer.getData(), start, length);
+        } else {
+          setNullColIsNullValue(bcv, rowIndex);
+        }
+      }
+        break;
+      case VARCHAR: {
+          BytesColumnVector bcv = (BytesColumnVector) batch.cols[off + i];
+          if (writableCol != null) {
+            bcv.isNull[rowIndex] = false;
+            HiveVarchar colHiveVarchar = ((HiveVarcharWritable) writableCol).getHiveVarchar();
+            byte[] bytes = colHiveVarchar.getValue().getBytes();
+
+            // We assume the VARCHAR maximum length was enforced when the object was created.
+            int length = bytes.length;
+
+            int start = buffer.getLength();
+            try {
+              buffer.write(bytes, 0, length);
+            } catch (IOException ioe) {
+              throw new IllegalStateException("bad write", ioe);
+            }
+            bcv.setRef(rowIndex, buffer.getData(), start, length);
+          } else {
+            setNullColIsNullValue(bcv, rowIndex);
+          }
+        }
+          break;
       case DECIMAL:
         DecimalColumnVector dcv = (DecimalColumnVector) batch.cols[off + i];
         if (writableCol != null) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedColumnarSerDe.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedColumnarSerDe.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedColumnarSerDe.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedColumnarSerDe.java Fri Sep  5 15:59:47 2014
@@ -158,7 +158,10 @@ public class VectorizedColumnarSerDe ext
                 serializeVectorStream.write(bytes, 0, bytes.length);
               }
                 break;
-              case STRING: {
+              case STRING:
+              case CHAR:
+              case VARCHAR: {
+                // Is it correct to escape CHAR and VARCHAR?
                 BytesColumnVector bcv = (BytesColumnVector) batch.cols[k];
                 LazyUtils.writeEscaped(serializeVectorStream, bcv.vector[rowIndex],
                     bcv.start[rowIndex],

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java Fri Sep  5 15:59:47 2014
@@ -278,7 +278,7 @@ public class VectorizedRowBatchCtx {
         case PRIMITIVE: {
           PrimitiveObjectInspector poi = (PrimitiveObjectInspector) foi;
           // Vectorization currently only supports the following data types:
-          // BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, BINARY, STRING, TIMESTAMP,
+          // BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, BINARY, STRING, CHAR, VARCHAR, TIMESTAMP,
           // DATE and DECIMAL
           switch (poi.getPrimitiveCategory()) {
           case BOOLEAN:
@@ -296,6 +296,8 @@ public class VectorizedRowBatchCtx {
             break;
           case BINARY:
           case STRING:
+          case CHAR:
+          case VARCHAR:
             result.cols[j] = new BytesColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
             break;
           case DECIMAL:
@@ -544,7 +546,9 @@ public class VectorizedRowBatchCtx {
           }
           break;
 
-        case STRING: {
+        case STRING:
+        case CHAR:
+        case VARCHAR: {
           BytesColumnVector bcv = (BytesColumnVector) batch.cols[colIndex];
           String sVal = (String) value;
           if (sVal == null) {
@@ -566,13 +570,17 @@ public class VectorizedRowBatchCtx {
     }
   }
 
-  private void addScratchColumnsToBatch(VectorizedRowBatch vrb) {
+  private void addScratchColumnsToBatch(VectorizedRowBatch vrb) throws HiveException {
     if (columnTypeMap != null && !columnTypeMap.isEmpty()) {
       int origNumCols = vrb.numCols;
       int newNumCols = vrb.cols.length+columnTypeMap.keySet().size();
       vrb.cols = Arrays.copyOf(vrb.cols, newNumCols);
       for (int i = origNumCols; i < newNumCols; i++) {
-        vrb.cols[i] = allocateColumnVector(columnTypeMap.get(i),
+       String typeName = columnTypeMap.get(i);
+       if (typeName == null) {
+         throw new HiveException("No type found for column type entry " + i);
+       }
+        vrb.cols[i] = allocateColumnVector(typeName,
             VectorizedRowBatch.DEFAULT_SIZE);
       }
       vrb.numCols = vrb.cols.length;
@@ -599,13 +607,17 @@ public class VectorizedRowBatchCtx {
   private ColumnVector allocateColumnVector(String type, int defaultSize) {
     if (type.equalsIgnoreCase("double")) {
       return new DoubleColumnVector(defaultSize);
-    } else if (type.equalsIgnoreCase("string")) {
+    } else if (VectorizationContext.isStringFamily(type)) {
       return new BytesColumnVector(defaultSize);
     } else if (VectorizationContext.decimalTypePattern.matcher(type).matches()){
       int [] precisionScale = getScalePrecisionFromDecimalType(type);
       return new DecimalColumnVector(defaultSize, precisionScale[0], precisionScale[1]);
-    } else {
+    } else if (type.equalsIgnoreCase("long") ||
+               type.equalsIgnoreCase("date") ||
+               type.equalsIgnoreCase("timestamp")) {
       return new LongColumnVector(defaultSize);
+    } else {
+      throw new Error("Cannot allocate vector column for " + type);
     }
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java Fri Sep  5 15:59:47 2014
@@ -420,8 +420,8 @@ public abstract class AbstractFilterStri
             VectorExpressionDescriptor.Mode.FILTER)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("string"),
-            VectorExpressionDescriptor.ArgumentType.getType("string"))
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY,
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR).build();

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToCharViaLongToChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToCharViaLongToChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToCharViaLongToChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToCharViaLongToChar.java Fri Sep  5 15:59:47 2014
@@ -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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+public class CastBooleanToCharViaLongToChar extends CastBooleanToStringViaLongToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastBooleanToCharViaLongToChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  public CastBooleanToCharViaLongToChar() {
+    super();
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "Char";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToStringViaLongToString.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToStringViaLongToString.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToStringViaLongToString.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToStringViaLongToString.java Fri Sep  5 15:59:47 2014
@@ -22,17 +22,18 @@ import org.apache.hadoop.hive.ql.exec.ve
 
 public class CastBooleanToStringViaLongToString extends LongToStringUnaryUDF {
   private static final long serialVersionUID = 1L;
-  private transient byte[] temp; // space to put date string
   private static final byte[][] dictionary = { {'F', 'A', 'L', 'S', 'E'}, {'T', 'R', 'U', 'E'} };
 
+  public CastBooleanToStringViaLongToString(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
   public CastBooleanToStringViaLongToString() {
     super();
-    temp = new byte[8];
   }
 
-  public CastBooleanToStringViaLongToString(int inputColumn, int outputColumn) {
-    super(inputColumn, outputColumn);
-    temp = new byte[8];
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    outV.setVal(i, bytes, 0, length);
   }
 
   @Override
@@ -41,6 +42,6 @@ public class CastBooleanToStringViaLongT
     /* 0 is false and 1 is true in the input vector, so a simple dictionary is used
      * with two entries. 0 references FALSE and 1 references TRUE in the dictionary.
      */
-    outV.setVal(i, dictionary[(int) vector[i]], 0, dictionary[(int) vector[i]].length);
+    assign(outV, i, dictionary[(int) vector[i]], dictionary[(int) vector[i]].length);
   }
-}
+}
\ No newline at end of file

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToVarCharViaLongToVarChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToVarCharViaLongToVarChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToVarCharViaLongToVarChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastBooleanToVarCharViaLongToVarChar.java Fri Sep  5 15:59:47 2014
@@ -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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+public class CastBooleanToVarCharViaLongToVarChar extends CastBooleanToStringViaLongToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastBooleanToVarCharViaLongToVarChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  public CastBooleanToVarCharViaLongToVarChar() {
+    super();
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.truncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "Char";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToChar.java Fri Sep  5 15:59:47 2014
@@ -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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+public class CastDateToChar extends CastDateToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastDateToChar() {
+    super();
+  }
+
+  public CastDateToChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "Char";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java Fri Sep  5 15:59:47 2014
@@ -35,10 +35,15 @@ public class CastDateToString extends Lo
     super(inputColumn, outputColumn);
   }
 
+  // The assign method will be overridden for CHAR and VARCHAR.
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    outV.setVal(i, bytes, 0, length);
+  }
+
   @Override
   protected void func(BytesColumnVector outV, long[] vector, int i) {
     dt.setTime(DateWritable.daysToMillis((int) vector[i]));
     byte[] temp = dt.toString().getBytes();
-    outV.setVal(i, temp, 0, temp.length);
+    assign(outV, i, temp, temp.length);
   }
 }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarChar.java Fri Sep  5 15:59:47 2014
@@ -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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+public class CastDateToVarChar extends CastDateToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastDateToVarChar() {
+    super();
+  }
+
+  public CastDateToVarChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.truncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "VarChar";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java Fri Sep  5 15:59:47 2014
@@ -0,0 +1,58 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+/**
+ * To support vectorized cast of decimal to string.
+ */
+public class CastDecimalToChar extends CastDecimalToString implements TruncStringOutput {
+
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastDecimalToChar() {
+    super();
+  }
+
+  public CastDecimalToChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "Char";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java Fri Sep  5 15:59:47 2014
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java Fri Sep  5 15:59:47 2014
@@ -36,6 +36,11 @@ public class CastDecimalToString extends
     super(inputColumn, outputColumn);
   }
 
+  // The assign method will be overridden for CHAR and VARCHAR.
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    outV.setVal(i, bytes, 0, length);
+  }
+
   @Override
   protected void func(BytesColumnVector outV, DecimalColumnVector inV, int i) {
     String s = inV.vector[i].getHiveDecimalString();
@@ -47,6 +52,6 @@ public class CastDecimalToString extends
       // This should never happen. If it does, there is a bug.
       throw new RuntimeException("Internal error:  unable to convert decimal to string");
     }
-    outV.setVal(i, b, 0, b.length);
+    assign(outV, i, b, b.length);
   }
 }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java Fri Sep  5 15:59:47 2014
@@ -0,0 +1,58 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+/**
+ * To support vectorized cast of decimal to string.
+ */
+public class CastDecimalToVarChar extends CastDecimalToString implements TruncStringOutput {
+
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastDecimalToVarChar() {
+    super();
+  }
+
+  public CastDecimalToVarChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.truncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "VarChar";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToChar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToChar.java?rev=1622729&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToChar.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToChar.java Fri Sep  5 15:59:47 2014
@@ -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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+public class CastLongToChar extends CastLongToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastLongToChar() {
+    super();
+  }
+
+  public CastLongToChar(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public String getOutputType() {
+    return "Char";
+  }
+  
+    @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDate.java Fri Sep  5 15:59:47 2014
@@ -122,6 +122,8 @@ public class CastLongToDate extends Vect
       case DATE:
         inV.copySelected(batch.selectedInUse, batch.selected, batch.size, outV);
         break;
+      default:
+        throw new Error("Unsupported input type " + inputTypes[0].name());
     }
   }
 
@@ -153,7 +155,7 @@ public class CastLongToDate extends Vect
     b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(1)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.LONG)
+            VectorExpressionDescriptor.ArgumentType.DATETIME_FAMILY)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN);
     return b.build();

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToString.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToString.java?rev=1622729&r1=1622728&r2=1622729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToString.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToString.java Fri Sep  5 15:59:47 2014
@@ -34,9 +34,14 @@ public class CastLongToString extends Lo
     temp = new byte[20];
   }
 
+  // The assign method will be overridden for CHAR and VARCHAR.
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    outV.setVal(i, bytes, 0, length);
+  }
+
   @Override
   protected void func(BytesColumnVector outV, long[] vector, int i) {
     int len = MathExpr.writeLongToUTF8(temp, vector[i]);
-    outV.setVal(i, temp, 0, len);
+    assign(outV, i, temp, len);
   }
 }