You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2013/10/29 18:25:59 UTC

svn commit: r1536823 [3/8] - in /hive/branches/tez: ./ cli/src/test/org/apache/hadoop/hive/cli/ common/src/java/conf/ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/common/type/ common/src/java/org/apache/hadoop/h...

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java Tue Oct 29 17:25:55 2013
@@ -22,9 +22,11 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -119,13 +121,86 @@ public final class ParseUtils {
         tableFieldTypeInfo, column);
   }
 
-  public static VarcharTypeInfo getVarcharTypeInfo(String typeName, ASTNode node)
+  public static VarcharTypeInfo getVarcharTypeInfo(ASTNode node)
       throws SemanticException {
     if (node.getChildCount() != 1) {
-      throw new SemanticException("Bad params for type " + typeName);
+      throw new SemanticException("Bad params for type varchar");
     }
 
     String lengthStr = node.getChild(0).getText();
     return TypeInfoFactory.getVarcharTypeInfo(Integer.valueOf(lengthStr));
   }
+
+  static int getIndex(String[] list, String elem) {
+    for(int i=0; i < list.length; i++) {
+      if (list[i].toLowerCase().equals(elem)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /*
+   * if the given filterCondn refers to only 1 table alias in the QBJoinTree,
+   * we return that alias's position. Otherwise we return -1
+   */
+  static int checkJoinFilterRefersOneAlias(String[] tabAliases, ASTNode filterCondn) {
+
+    switch(filterCondn.getType()) {
+    case HiveParser.TOK_TABLE_OR_COL:
+      String tableOrCol = SemanticAnalyzer.unescapeIdentifier(filterCondn.getChild(0).getText()
+          .toLowerCase());
+      return getIndex(tabAliases, tableOrCol);
+    case HiveParser.Identifier:
+    case HiveParser.Number:
+    case HiveParser.StringLiteral:
+    case HiveParser.BigintLiteral:
+    case HiveParser.SmallintLiteral:
+    case HiveParser.TinyintLiteral:
+    case HiveParser.DecimalLiteral:
+    case HiveParser.TOK_STRINGLITERALSEQUENCE:
+    case HiveParser.TOK_CHARSETLITERAL:
+    case HiveParser.TOK_DATELITERAL:
+    case HiveParser.KW_TRUE:
+    case HiveParser.KW_FALSE:
+    case HiveParser.TOK_NULL:
+      return -1;
+    default:
+      int idx = -1;
+      int i = filterCondn.getType() == HiveParser.TOK_FUNCTION ? 1 : 0;
+      for (; i < filterCondn.getChildCount(); i++) {
+        int cIdx = checkJoinFilterRefersOneAlias(tabAliases, (ASTNode) filterCondn.getChild(i));
+        if ( cIdx != idx ) {
+          if ( idx != -1 && cIdx != -1 ) {
+            return -1;
+          }
+          idx = idx == -1 ? cIdx : idx;
+        }
+      }
+      return idx;
+    }
+  }
+
+  public static DecimalTypeInfo getDecimalTypeTypeInfo(ASTNode node)
+      throws SemanticException {
+    if (node.getChildCount() > 2) {
+        throw new SemanticException("Bad params for type decimal");
+      }
+
+      int precision = HiveDecimal.DEFAULT_PRECISION;
+      int scale = HiveDecimal.DEFAULT_SCALE;
+
+      if (node.getChildCount() >= 1) {
+        String precStr = node.getChild(0).getText();
+        precision = Integer.valueOf(precStr);
+      }
+
+      if (node.getChildCount() == 2) {
+        String scaleStr = node.getChild(1).getText();
+        scale = Integer.valueOf(scaleStr);
+      }
+
+      return TypeInfoFactory.getDecimalTypeInfo(precision, scale);
+  }
+
 }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java Tue Oct 29 17:25:55 2013
@@ -75,6 +75,14 @@ public class QBJoinTree implements Seria
   // big tables that should be streamed
   private List<String> streamAliases;
 
+  /*
+   * when a QBJoinTree is merged into this one, its left(pos =0) filters can
+   * refer to any of the srces in this QBJoinTree. If a particular filterForPushing refers
+   * to multiple srces in this QBJoinTree, we collect them into 'postJoinFilters'
+   * We then add a Filter Operator after the Join Operator for this QBJoinTree.
+   */
+  private final List<ASTNode> postJoinFilters;
+
   /**
    * constructor.
    */
@@ -84,6 +92,7 @@ public class QBJoinTree implements Seria
     noSemiJoin = true;
     rhsSemijoin = new HashMap<String, ArrayList<ASTNode>>();
     aliasToOpInfo = new HashMap<String, Operator<? extends OperatorDesc>>();
+    postJoinFilters = new ArrayList<ASTNode>();
   }
 
   /**
@@ -346,4 +355,12 @@ public class QBJoinTree implements Seria
   public void setId(String id) {
     this.id = id;
   }
+
+  public void addPostJoinFilter(ASTNode filter) {
+    postJoinFilters.add(filter);
+  }
+
+  public List<ASTNode> getPostJoinFilters() {
+    return postJoinFilters;
+  }
 }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Tue Oct 29 17:25:55 2013
@@ -6192,7 +6192,12 @@ public class SemanticAnalyzer extends Ba
     JoinOperator joinOp = (JoinOperator) genJoinOperatorChildren(joinTree,
       joinSrcOp, srcOps, omitOpts);
     joinContext.put(joinOp, joinTree);
-    return joinOp;
+
+    Operator op = joinOp;
+    for(ASTNode condn : joinTree.getPostJoinFilters() ) {
+      op = genFilterPlan(qb, condn, op);
+    }
+    return op;
   }
 
   /**
@@ -6915,9 +6920,23 @@ public class SemanticAnalyzer extends Ba
     }
 
     if (node.getFiltersForPushing().get(0).size() != 0) {
-      ArrayList<ASTNode> filterPos = filter.get(pos);
-      filterPos.addAll(node.getFiltersForPushing().get(0));
-    }
+      /*
+       * for each predicate:
+       * - does it refer to one or many aliases
+       * - if one: add it to the filterForPushing list of that alias
+       * - if many: add as a filter from merging trees.
+       */
+
+      for(ASTNode nodeFilter : node.getFiltersForPushing().get(0) ) {
+        int fPos = ParseUtils.checkJoinFilterRefersOneAlias(target.getBaseSrc(), nodeFilter);
+
+        if ( fPos != - 1 ) {
+          filter.get(fPos).add(nodeFilter);
+        } else {
+          target.addPostJoinFilter(nodeFilter);
+        }
+      }
+     }
 
     if (node.getNoOuterJoin() && target.getNoOuterJoin()) {
       target.setNoOuterJoin(true);

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java Tue Oct 29 17:25:55 2013
@@ -62,6 +62,7 @@ import org.apache.hadoop.hive.ql.udf.gen
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
@@ -796,13 +797,17 @@ public final class TypeCheckProcFactory 
           ASTNode funcNameNode = (ASTNode)expr.getChild(0);
           switch (funcNameNode.getType()) {
             case HiveParser.TOK_VARCHAR:
-              // Add type params
-              VarcharTypeInfo varcharTypeInfo = TypeInfoFactory.getVarcharTypeInfo(
-                  Integer.valueOf((funcNameNode.getChild(0).getText())));
+              VarcharTypeInfo varcharTypeInfo = ParseUtils.getVarcharTypeInfo(funcNameNode);
               if (genericUDF != null) {
                 ((SettableUDF)genericUDF).setTypeInfo(varcharTypeInfo);
               }
               break;
+            case HiveParser.TOK_DECIMAL:
+              DecimalTypeInfo decTypeInfo = ParseUtils.getDecimalTypeTypeInfo(funcNameNode);
+              if (genericUDF != null) {
+                ((SettableUDF)genericUDF).setTypeInfo(decTypeInfo);
+              }
+              break;
             default:
               // Do nothing
               break;

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java Tue Oct 29 17:25:55 2013
@@ -59,11 +59,11 @@ public class ExprNodeConstantDesc extend
 
   @Override
   public ConstantObjectInspector getWritableObjectInspector() {
-    PrimitiveCategory pc = ((PrimitiveTypeInfo)getTypeInfo())
-        .getPrimitiveCategory();
+    PrimitiveTypeInfo pti = (PrimitiveTypeInfo) getTypeInfo();
+    PrimitiveCategory pc = pti.getPrimitiveCategory();
     // Convert from Java to Writable
     Object writableValue = PrimitiveObjectInspectorFactory
-        .getPrimitiveJavaObjectInspector(pc).getPrimitiveWritableObject(
+        .getPrimitiveJavaObjectInspector(pti).getPrimitiveWritableObject(
           getValue());
     return PrimitiveObjectInspectorFactory
         .getPrimitiveWritableConstantObjectInspector((PrimitiveTypeInfo) getTypeInfo(), writableValue);

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPDivide.java Tue Oct 29 17:25:55 2013
@@ -74,13 +74,17 @@ public class UDFOPDivide extends UDF {
     if ((a == null) || (b == null)) {
       return null;
     }
+
     if (b.getHiveDecimal().compareTo(HiveDecimal.ZERO) == 0) {
       return null;
-    } else {
-        decimalWritable.set(a.getHiveDecimal().divide(
-          b.getHiveDecimal()));
     }
 
+    HiveDecimal dec = a.getHiveDecimal().divide(b.getHiveDecimal());
+    if (dec == null) {
+      return null;
+    }
+
+    decimalWritable.set(dec);
     return decimalWritable;
   }
 }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPMod.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPMod.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPMod.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPMod.java Tue Oct 29 17:25:55 2013
@@ -61,7 +61,7 @@ public class UDFOPMod extends UDFBaseNum
   public ByteWritable evaluate(ByteWritable a, ByteWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0) {
       return null;
     }
 
@@ -73,7 +73,7 @@ public class UDFOPMod extends UDFBaseNum
   public ShortWritable evaluate(ShortWritable a, ShortWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0) {
       return null;
     }
 
@@ -85,7 +85,7 @@ public class UDFOPMod extends UDFBaseNum
   public IntWritable evaluate(IntWritable a, IntWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0) {
       return null;
     }
 
@@ -97,7 +97,7 @@ public class UDFOPMod extends UDFBaseNum
   public LongWritable evaluate(LongWritable a, LongWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0L) {
       return null;
     }
 
@@ -109,7 +109,7 @@ public class UDFOPMod extends UDFBaseNum
   public FloatWritable evaluate(FloatWritable a, FloatWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0.0f) {
       return null;
     }
 
@@ -121,7 +121,7 @@ public class UDFOPMod extends UDFBaseNum
   public DoubleWritable evaluate(DoubleWritable a, DoubleWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0.0) {
       return null;
     }
 

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPPlus.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPPlus.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPPlus.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFOPPlus.java Tue Oct 29 17:25:55 2013
@@ -139,12 +139,12 @@ public class UDFOPPlus extends UDFBaseNu
       return null;
     }
 
-      HiveDecimal dec = a.getHiveDecimal().add(b.getHiveDecimal());
-      if (dec == null) {
-        return null;
-      }
+    HiveDecimal dec = a.getHiveDecimal().add(b.getHiveDecimal());
+    if (dec == null) {
+      return null;
+    }
 
-      decimalWritable.set(dec);
+    decimalWritable.set(dec);
     return decimalWritable;
   }
 

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFPosMod.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFPosMod.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFPosMod.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFPosMod.java Tue Oct 29 17:25:55 2013
@@ -47,7 +47,7 @@ public class UDFPosMod extends UDFBaseNu
   public ByteWritable evaluate(ByteWritable a, ByteWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0) {
       return null;
     }
 
@@ -59,7 +59,7 @@ public class UDFPosMod extends UDFBaseNu
   public ShortWritable evaluate(ShortWritable a, ShortWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0) {
       return null;
     }
 
@@ -71,7 +71,7 @@ public class UDFPosMod extends UDFBaseNu
   public IntWritable evaluate(IntWritable a, IntWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0) {
       return null;
     }
 
@@ -83,7 +83,7 @@ public class UDFPosMod extends UDFBaseNu
   public LongWritable evaluate(LongWritable a, LongWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0L) {
       return null;
     }
 
@@ -95,7 +95,7 @@ public class UDFPosMod extends UDFBaseNu
   public FloatWritable evaluate(FloatWritable a, FloatWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0.0f) {
       return null;
     }
 
@@ -107,7 +107,7 @@ public class UDFPosMod extends UDFBaseNu
   public DoubleWritable evaluate(DoubleWritable a, DoubleWritable b) {
     // LOG.info("Get input " + a.getClass() + ":" + a + " " + b.getClass() + ":"
     // + b);
-    if ((a == null) || (b == null)) {
+    if (a == null || b == null || b.get() == 0.0) {
       return null;
     }
 

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java Tue Oct 29 17:25:55 2013
@@ -23,14 +23,17 @@ import java.lang.reflect.Method;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
+import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
@@ -43,21 +46,41 @@ import org.apache.hadoop.hive.serde2.typ
  *
  */
 public class GenericUDFBridge extends GenericUDF implements Serializable {
+  private static final long serialVersionUID = 4994861742809511113L;
+
   /**
    * The name of the UDF.
    */
-  String udfName;
+  private String udfName;
 
   /**
    * Whether the UDF is an operator or not. This controls how the display string
    * is generated.
    */
-  boolean isOperator;
+  private boolean isOperator;
 
   /**
    * The underlying UDF class Name.
    */
-  String udfClassName;
+  private String udfClassName;
+
+  /**
+   * The underlying method of the UDF class.
+   */
+  private transient Method udfMethod;
+
+  /**
+   * Helper to convert the parameters before passing to udfMethod.
+   */
+  private transient ConversionHelper conversionHelper;
+  /**
+   * The actual udf object.
+   */
+  private transient UDF udf;
+  /**
+   * The non-deferred real arguments for method invocation.
+   */
+  private transient Object[] realArguments;
 
   /**
    * Create a new GenericUDFBridge object.
@@ -73,7 +96,7 @@ public class GenericUDFBridge extends Ge
     this.isOperator = isOperator;
     this.udfClassName = udfClassName;
   }
-
+ 
   // For Java serialization only
   public GenericUDFBridge() {
   }
@@ -110,24 +133,6 @@ public class GenericUDFBridge extends Ge
     }
   }
 
-  /**
-   * The underlying method of the UDF class.
-   */
-  transient Method udfMethod;
-
-  /**
-   * Helper to convert the parameters before passing to udfMethod.
-   */
-  transient ConversionHelper conversionHelper;
-  /**
-   * The actual udf object.
-   */
-  transient UDF udf;
-  /**
-   * The non-deferred real arguments for method invocation.
-   */
-  transient Object[] realArguments;
-
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
 
@@ -176,6 +181,13 @@ public class GenericUDFBridge extends Ge
     Object result = FunctionRegistry.invoke(udfMethod, udf, conversionHelper
         .convertIfNecessary(realArguments));
 
+    // For non-generic UDF, type info isn't available. This poses a problem for Hive Decimal.
+    // If the returned value is HiveDecimal, we assume maximum precision/scale.
+    if (result != null && result instanceof HiveDecimalWritable) {
+      result = HiveDecimalUtils.enforcePrecisionScale((HiveDecimalWritable) result,
+          HiveDecimal.MAX_PRECISION, HiveDecimal.MAX_SCALE);
+    }
+
     return result;
   }
 

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToDecimal.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToDecimal.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToDecimal.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToDecimal.java Tue Oct 29 17:25:55 2013
@@ -21,17 +21,23 @@ import org.apache.hadoop.hive.ql.exec.De
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.SettableUDF;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter.HiveDecimalConverter;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableHiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 @Description(name = "decimal", value = "_FUNC_(a) - cast a to decimal")
-public class GenericUDFToDecimal extends GenericUDF {
+public class GenericUDFToDecimal extends GenericUDF implements SettableUDF {
 
   private transient PrimitiveObjectInspector argumentOI;
   private transient HiveDecimalConverter bdConverter;
 
+  private DecimalTypeInfo typeInfo;
+
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
     if (arguments.length < 1) {
@@ -46,9 +52,13 @@ public class GenericUDFToDecimal extends
           "The function DECIMAL takes only primitive types");
     }
 
-    bdConverter = new HiveDecimalConverter(argumentOI,
-        PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector);
-    return PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector;
+    // Check if this UDF has been provided with type params for the output varchar type
+    SettableHiveDecimalObjectInspector outputOI;
+    outputOI = (SettableHiveDecimalObjectInspector)
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(typeInfo);
+
+    bdConverter = new HiveDecimalConverter(argumentOI, outputOI);
+    return outputOI;
   }
 
   @Override
@@ -67,8 +77,23 @@ public class GenericUDFToDecimal extends
     StringBuilder sb = new StringBuilder();
     sb.append("CAST( ");
     sb.append(children[0]);
-    sb.append(" AS DECIMAL)");
+    sb.append(" AS ");
+    sb.append(typeInfo.getQualifiedName());
+    sb.append(")");
     return sb.toString();
   }
 
+  public DecimalTypeInfo getTypeInfo() {
+    return typeInfo;
+  }
+
+  public void setTypeInfo(DecimalTypeInfo typeInfo) {
+    this.typeInfo = typeInfo;
+  }
+
+  @Override
+  public void setTypeInfo(TypeInfo typeInfo) throws UDFArgumentException {
+    this.typeInfo = (DecimalTypeInfo) typeInfo;
+  }
+
 }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToVarchar.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToVarchar.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToVarchar.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToVarchar.java Tue Oct 29 17:25:55 2013
@@ -90,8 +90,8 @@ public class GenericUDFToVarchar extends
     StringBuilder sb = new StringBuilder();
     sb.append("CAST( ");
     sb.append(children[0]);
-    sb.append(" AS VARCHAR(");
-    sb.append("" + typeInfo.getLength());
+    sb.append(" AS ");
+    sb.append(typeInfo.getQualifiedName());
     sb.append(")");
     return sb.toString();
   }

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Tue Oct 29 17:25:55 2013
@@ -1087,10 +1087,6 @@ public class QTestUtil {
     in = new BufferedReader(new FileReader(fname));
     out = new BufferedWriter(new FileWriter(fname + ".orig"));
     while (null != (line = in.readLine())) {
-      // Ignore the empty lines on windows
-      if(line.isEmpty() && Shell.WINDOWS) {
-        continue;
-      }
       out.write(line);
       out.write('\n');
     }

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java Tue Oct 29 17:25:55 2013
@@ -32,6 +32,14 @@ import java.util.regex.Pattern;
  *  ignored.
  */
 public class TestLocationQueries extends BaseTestQueries {
+
+  public TestLocationQueries() {
+    File logDirFile = new File(logDir);
+    if (!(logDirFile.exists() || logDirFile.mkdirs())) {
+      fail("Could not create " + logDir);
+    }
+  }
+
   /**
    * Our own checker - validate the location of the partition.
    */

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java Tue Oct 29 17:25:55 2013
@@ -25,6 +25,13 @@ import java.io.File;
  */
 public class TestMTQueries extends BaseTestQueries {
 
+  public TestMTQueries() {
+    File logDirFile = new File(logDir);
+    if (!(logDirFile.exists() || logDirFile.mkdirs())) {
+      fail("Could not create " + logDir);
+    }
+  }
+
   public void testMTQueries1() throws Exception {
     String[] testNames = new String[] {"join1.q", "join2.q", "groupby1.q",
         "groupby2.q", "join3.q", "input1.q", "input19.q"};

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java Tue Oct 29 17:25:55 2013
@@ -131,8 +131,7 @@ public class TestExecDriver extends Test
       }
 
     } catch (Throwable e) {
-      e.printStackTrace();
-      throw new RuntimeException("Encountered throwable");
+      throw new RuntimeException("Encountered throwable", e);
     }
   }
 
@@ -472,118 +471,70 @@ public class TestExecDriver extends Test
   public void testMapPlan1() throws Exception {
 
     LOG.info("Beginning testMapPlan1");
-
-    try {
-      populateMapPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
-      executePlan();
-      fileDiff("lt100.txt.deflate", "mapplan1.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
+    executePlan();
+    fileDiff("lt100.txt.deflate", "mapplan1.out");
   }
 
   public void testMapPlan2() throws Exception {
 
     LOG.info("Beginning testMapPlan2");
-
-    try {
-      populateMapPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
-      executePlan();
-      fileDiff("lt100.txt", "mapplan2.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
+    executePlan();
+    fileDiff("lt100.txt", "mapplan2.out");
   }
 
   public void testMapRedPlan1() throws Exception {
 
     LOG.info("Beginning testMapRedPlan1");
-
-    try {
-      populateMapRedPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("kv1.val.sorted.txt", "mapredplan1.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("kv1.val.sorted.txt", "mapredplan1.out");
   }
 
   public void testMapRedPlan2() throws Exception {
 
     LOG.info("Beginning testMapPlan2");
-
-    try {
-      populateMapRedPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("lt100.sorted.txt", "mapredplan2.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("lt100.sorted.txt", "mapredplan2.out");
   }
 
   public void testMapRedPlan3() throws Exception {
 
     LOG.info("Beginning testMapPlan3");
-
-    try {
-      populateMapRedPlan3(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"), db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src2"));
-      executePlan();
-      fileDiff("kv1kv2.cogroup.txt", "mapredplan3.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan3(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"), db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src2"));
+    executePlan();
+    fileDiff("kv1kv2.cogroup.txt", "mapredplan3.out");
   }
 
   public void testMapRedPlan4() throws Exception {
 
     LOG.info("Beginning testMapPlan4");
-
-    try {
-      populateMapRedPlan4(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("kv1.string-sorted.txt", "mapredplan4.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan4(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("kv1.string-sorted.txt", "mapredplan4.out");
   }
 
   public void testMapRedPlan5() throws Exception {
 
     LOG.info("Beginning testMapPlan5");
-
-    try {
-      populateMapRedPlan5(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("kv1.string-sorted.txt", "mapredplan5.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan5(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("kv1.string-sorted.txt", "mapredplan5.out");
   }
 
   public void testMapRedPlan6() throws Exception {
 
     LOG.info("Beginning testMapPlan6");
-
-    try {
-      populateMapRedPlan6(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("lt100.sorted.txt", "mapredplan6.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan6(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("lt100.sorted.txt", "mapredplan6.out");
   }
 }

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java Tue Oct 29 17:25:55 2013
@@ -208,7 +208,7 @@ public class TestFunctionRegistry extend
     common(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.decimalTypeInfo,
            TypeInfoFactory.stringTypeInfo);
     common(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.decimalTypeInfo,
-           TypeInfoFactory.decimalTypeInfo);
+           TypeInfoFactory.getDecimalTypeInfo(65, 30));
     common(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.stringTypeInfo,
            TypeInfoFactory.stringTypeInfo);
 
@@ -226,7 +226,7 @@ public class TestFunctionRegistry extend
     comparison(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.decimalTypeInfo,
                TypeInfoFactory.decimalTypeInfo);
     comparison(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.decimalTypeInfo,
-               TypeInfoFactory.decimalTypeInfo);
+               TypeInfoFactory.getDecimalTypeInfo(65, 30));
     comparison(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.stringTypeInfo,
                TypeInfoFactory.doubleTypeInfo);
 
@@ -296,7 +296,7 @@ public class TestFunctionRegistry extend
     unionAll(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.decimalTypeInfo,
         TypeInfoFactory.decimalTypeInfo);
     unionAll(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.decimalTypeInfo,
-        TypeInfoFactory.decimalTypeInfo);
+        TypeInfoFactory.getDecimalTypeInfo(65, 30));
     unionAll(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.stringTypeInfo,
         TypeInfoFactory.stringTypeInfo);
 

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java Tue Oct 29 17:25:55 2013
@@ -51,7 +51,7 @@ public class TestMemoryManager {
     NullCallback callback = new NullCallback();
     long poolSize = mgr.getTotalMemoryPool();
     assertEquals(Math.round(ManagementFactory.getMemoryMXBean().
-        getHeapMemoryUsage().getMax() * 0.5f), poolSize);
+        getHeapMemoryUsage().getMax() * 0.5d), poolSize);
     assertEquals(1.0, mgr.getAllocationScale(), 0.00001);
     mgr.addWriter(new Path("p1"), 1000, callback);
     assertEquals(1.0, mgr.getAllocationScale(), 0.00001);

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java Tue Oct 29 17:25:55 2013
@@ -270,7 +270,7 @@ public class TestOrcFile {
         + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
         + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
         + "map:map<string,struct<int1:int,string1:string>>,ts:timestamp,"
-        + "decimal1:decimal>", readerInspector.getTypeName());
+        + "decimal1:decimal(65,30)>", readerInspector.getTypeName());
     List<? extends StructField> fields = readerInspector
         .getAllStructFieldRefs();
     BooleanObjectInspector bo = (BooleanObjectInspector) readerInspector
@@ -1008,8 +1008,8 @@ public class TestOrcFile {
       } else {
         union.set((byte) 1, new Text(new Integer(i*i).toString()));
       }
-      value = HiveDecimal.create(new BigInteger(118, rand),
-          rand.nextInt(36));
+      value = HiveDecimal.create(new BigInteger(104, rand),
+          rand.nextInt(28));
       row.setFieldValue(2, value);
       if (maxValue.compareTo(value) < 0) {
         maxValue = value;
@@ -1038,7 +1038,8 @@ public class TestOrcFile {
     assertEquals(303, stats.getNumberOfValues());
     assertEquals(HiveDecimal.create("-5643.234"), stats.getMinimum());
     assertEquals(maxValue, stats.getMaximum());
-    assertEquals(null, stats.getSum());
+    // TODO: fix this
+//    assertEquals(null,stats.getSum());
     int stripeCount = 0;
     int rowCount = 0;
     long currentOffset = -1;
@@ -1062,7 +1063,7 @@ public class TestOrcFile {
     row = (OrcStruct) rows.next(null);
     assertEquals(1, rows.getRowNumber());
     inspector = reader.getObjectInspector();
-    assertEquals("struct<time:timestamp,union:uniontype<int,string>,decimal:decimal>",
+    assertEquals("struct<time:timestamp,union:uniontype<int,string>,decimal:decimal(65,30)>",
         inspector.getTypeName());
     assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"),
         row.getFieldValue(0));
@@ -1110,8 +1111,8 @@ public class TestOrcFile {
         assertEquals(1, union.getTag());
         assertEquals(new Text(new Integer(i*i).toString()), union.getObject());
       }
-      assertEquals(HiveDecimal.create(new BigInteger(118, rand),
-                                   rand.nextInt(36)), row.getFieldValue(2));
+      assertEquals(HiveDecimal.create(new BigInteger(104, rand),
+                                   rand.nextInt(28)), row.getFieldValue(2));
     }
     for(int i=0; i < 5000; ++i) {
       row = (OrcStruct) rows.next(row);

Modified: hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java (original)
+++ hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java Tue Oct 29 17:25:55 2013
@@ -25,18 +25,18 @@ import java.util.Map;
 
 import junit.framework.Assert;
 
+import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFSumLong;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncAbsLongToLong;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.AggregationDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.ql.plan.GroupByDesc;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs;
+import org.apache.hadoop.hive.ql.plan.*;
+import org.apache.hadoop.hive.ql.udf.generic.*;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,9 +50,28 @@ public class TestVectorizer {
     Map<String, Integer> columnMap = new HashMap<String, Integer>();
     columnMap.put("col1", 0);
     columnMap.put("col2", 1);
+    columnMap.put("col3", 2);
 
     //Generate vectorized expression
-    vContext = new VectorizationContext(columnMap, 2);
+    vContext = new VectorizationContext(columnMap, 3);
+  }
+
+  @Description(name = "fake", value = "FAKE")
+  static class FakeGenericUDF extends GenericUDF {
+    @Override
+    public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+      return null;
+    }
+
+    @Override
+    public Object evaluate(DeferredObject[] arguments) throws HiveException {
+      return null;
+    }
+
+    @Override
+    public String getDisplayString(String[] children) {
+      return "fake";
+    }
   }
 
   @Test
@@ -96,4 +115,37 @@ public class TestVectorizer {
     VectorUDAFSumLong udaf = (VectorUDAFSumLong) vectorOp.getAggregators()[0];
     Assert.assertEquals(FuncAbsLongToLong.class, udaf.getInputExpression().getClass());
   }
+
+  @Test
+  public void testValidateNestedExpressions() {
+    ExprNodeColumnDesc col1Expr = new ExprNodeColumnDesc(Integer.class, "col1", "table", false);
+    ExprNodeConstantDesc constDesc = new ExprNodeConstantDesc(new Integer(10));
+
+    GenericUDFOPGreaterThan udf = new GenericUDFOPGreaterThan();
+    ExprNodeGenericFuncDesc greaterExprDesc = new ExprNodeGenericFuncDesc();
+    greaterExprDesc.setTypeInfo(TypeInfoFactory.booleanTypeInfo);
+    greaterExprDesc.setGenericUDF(udf);
+    List<ExprNodeDesc> children1 = new ArrayList<ExprNodeDesc>(2);
+    children1.add(col1Expr);
+    children1.add(constDesc);
+    greaterExprDesc.setChildren(children1);
+
+    FakeGenericUDF udf2 = new FakeGenericUDF();
+    ExprNodeGenericFuncDesc nonSupportedExpr = new ExprNodeGenericFuncDesc();
+    nonSupportedExpr.setTypeInfo(TypeInfoFactory.booleanTypeInfo);
+    nonSupportedExpr.setGenericUDF(udf2);
+
+    GenericUDFOPAnd andUdf = new GenericUDFOPAnd();
+    ExprNodeGenericFuncDesc andExprDesc = new ExprNodeGenericFuncDesc();
+    andExprDesc.setTypeInfo(TypeInfoFactory.booleanTypeInfo);
+    andExprDesc.setGenericUDF(andUdf);
+    List<ExprNodeDesc> children3 = new ArrayList<ExprNodeDesc>(2);
+    children3.add(greaterExprDesc);
+    children3.add(nonSupportedExpr);
+    andExprDesc.setChildren(children3);
+
+    Vectorizer v = new Vectorizer();
+    Assert.assertFalse(v.validateExprNodeDesc(andExprDesc, VectorExpressionDescriptor.Mode.FILTER));
+    Assert.assertFalse(v.validateExprNodeDesc(andExprDesc, VectorExpressionDescriptor.Mode.PROJECTION));
+  }
 }

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_1.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_1.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_1.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_1.q Tue Oct 29 17:25:55 2013
@@ -1,10 +1,10 @@
 drop table decimal_1;
 
-create table decimal_1 (t decimal);
+create table decimal_1 (t decimal(4,2));
 alter table decimal_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
 
 insert overwrite table decimal_1
-  select cast('17.29' as decimal) from src limit 1;
+  select cast('17.29' as decimal(4,2)) from src limit 1;
 select cast(t as boolean) from decimal_1 limit 1;
 select cast(t as tinyint) from decimal_1 limit 1;
 select cast(t as smallint) from decimal_1 limit 1;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_2.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_2.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_2.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_2.q Tue Oct 29 17:25:55 2013
@@ -1,10 +1,10 @@
 drop table decimal_2;
 
-create table decimal_2 (t decimal);
+create table decimal_2 (t decimal(18,9));
 alter table decimal_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe';
 
 insert overwrite table decimal_2
-  select cast('17.29' as decimal) from src limit 1;
+  select cast('17.29' as decimal(4,2)) from src limit 1;
 
 select cast(t as boolean) from decimal_2 limit 1;
 select cast(t as tinyint) from decimal_2 limit 1;
@@ -16,7 +16,7 @@ select cast(t as double) from decimal_2 
 select cast(t as string) from decimal_2 limit 1;
 
 insert overwrite table decimal_2
-  select cast('3404045.5044003' as decimal) from src limit 1;
+  select cast('3404045.5044003' as decimal(18,9)) from src limit 1;
 
 select cast(t as boolean) from decimal_2 limit 1;
 select cast(t as tinyint) from decimal_2 limit 1;
@@ -27,14 +27,14 @@ select cast(t as float) from decimal_2 l
 select cast(t as double) from decimal_2 limit 1;
 select cast(t as string) from decimal_2 limit 1;
 
-select cast(3.14 as decimal) from decimal_2 limit 1;
-select cast(cast(3.14 as float) as decimal) from decimal_2 limit 1;
-select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal) from decimal_2 limit 1;
+select cast(3.14 as decimal(4,2)) from decimal_2 limit 1;
+select cast(cast(3.14 as float) as decimal(4,2)) from decimal_2 limit 1;
+select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) from decimal_2 limit 1;
 select cast(true as decimal) from decimal_2 limit 1;
 select cast(3Y as decimal) from decimal_2 limit 1;
 select cast(3S as decimal) from decimal_2 limit 1;
 select cast(cast(3 as int) as decimal) from decimal_2 limit 1;
 select cast(3L as decimal) from decimal_2 limit 1;
-select cast(0.99999999999999999999 as decimal) from decimal_2 limit 1;
-select cast('0.99999999999999999999' as decimal) from decimal_2 limit 1;
+select cast(0.99999999999999999999 as decimal(20,19)) from decimal_2 limit 1;
+select cast('0.99999999999999999999' as decimal(20,20)) from decimal_2 limit 1;
 drop table decimal_2;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_3.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_3.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_3.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_3.q Tue Oct 29 17:25:55 2013
@@ -1,6 +1,6 @@
 DROP TABLE IF EXISTS DECIMAL_3;
 
-CREATE TABLE DECIMAL_3(key decimal, value int) 
+CREATE TABLE DECIMAL_3(key decimal(65,30), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_4.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_4.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_4.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_4.q Tue Oct 29 17:25:55 2013
@@ -1,12 +1,12 @@
 DROP TABLE IF EXISTS DECIMAL_4_1;
 DROP TABLE IF EXISTS DECIMAL_4_2;
 
-CREATE TABLE DECIMAL_4_1(key decimal, value int) 
+CREATE TABLE DECIMAL_4_1(key decimal(35,25), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
 
-CREATE TABLE DECIMAL_4_2(key decimal, value decimal) 
+CREATE TABLE DECIMAL_4_2(key decimal(35,25), value decimal(35,25)) 
 STORED AS ORC;
 
 LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_4_1;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_join.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_join.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_join.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_join.q Tue Oct 29 17:25:55 2013
@@ -1,6 +1,6 @@
 -- HIVE-5292 Join on decimal columns fails
 
-create table src_dec (key decimal, value string);
+create table src_dec (key decimal(3,0), value string);
 load data local inpath '../data/files/kv1.txt' into table src_dec;
 
 select * from src_dec a join src_dec b on a.key=b.key+450;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_precision.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_precision.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_precision.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_precision.q Tue Oct 29 17:25:55 2013
@@ -1,6 +1,6 @@
 DROP TABLE IF EXISTS DECIMAL_PRECISION;
 
-CREATE TABLE DECIMAL_PRECISION(dec decimal) 
+CREATE TABLE DECIMAL_PRECISION(dec decimal(60,30)) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
@@ -17,11 +17,11 @@ SELECT dec, dec * dec FROM DECIMAL_PRECI
 
 SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION;
 
-SELECT dec * cast('123456789012345678901234567890.123456789' as decimal) FROM DECIMAL_PRECISION LIMIT 1;
-SELECT * from DECIMAL_PRECISION WHERE dec > cast('123456789012345678901234567890.123456789' as decimal) LIMIT 1;
+SELECT dec * cast('123456789012345678901234567890.123456789' as decimal(39,9)) FROM DECIMAL_PRECISION LIMIT 1;
+SELECT * from DECIMAL_PRECISION WHERE dec > cast('123456789012345678901234567890.123456789' as decimal(39,9)) LIMIT 1;
 SELECT dec * 123456789012345678901234567890.123456789 FROM DECIMAL_PRECISION LIMIT 1;
 
-SELECT MIN(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION;
-SELECT COUNT(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION;
+SELECT MIN(cast('123456789012345678901234567890.123456789' as decimal(39,9))) FROM DECIMAL_PRECISION;
+SELECT COUNT(cast('123456789012345678901234567890.123456789' as decimal(39,9))) FROM DECIMAL_PRECISION;
 
 DROP TABLE DECIMAL_PRECISION;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/decimal_udf.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/decimal_udf.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/decimal_udf.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/decimal_udf.q Tue Oct 29 17:25:55 2013
@@ -1,6 +1,6 @@
 DROP TABLE IF EXISTS DECIMAL_UDF;
 
-CREATE TABLE DECIMAL_UDF (key decimal, value int) 
+CREATE TABLE DECIMAL_UDF (key decimal(65,30), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q Tue Oct 29 17:25:55 2013
@@ -7,7 +7,7 @@ CREATE TABLE orc_pred(t tinyint,
            bo boolean,
            s string,
            ts timestamp,
-           dec decimal,
+           dec decimal(4,2),
            bin binary)
 STORED AS ORC;
 
@@ -22,7 +22,7 @@ CREATE TABLE staging(t tinyint,
            bo boolean,
            s string,
            ts timestamp,
-           dec decimal,
+           dec decimal(4,2),
            bin binary)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
 STORED AS TEXTFILE;

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/ptf_decimal.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/ptf_decimal.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/ptf_decimal.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/ptf_decimal.q Tue Oct 29 17:25:55 2013
@@ -9,7 +9,7 @@ CREATE TABLE part( 
     p_type STRING,
     p_size INT,
     p_container STRING,
-    p_retailprice DECIMAL,
+    p_retailprice DECIMAL(6,2),
     p_comment STRING
 );
 

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/serde_regex.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/serde_regex.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/serde_regex.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/serde_regex.q Tue Oct 29 17:25:55 2013
@@ -42,7 +42,7 @@ DROP TABLE serde_regex;
 
 EXPLAIN
 CREATE TABLE serde_regex1(
-  key decimal,
+  key decimal(65,30),
   value int)
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe'
 WITH SERDEPROPERTIES (
@@ -51,7 +51,7 @@ WITH SERDEPROPERTIES (
 STORED AS TEXTFILE;
 
 CREATE TABLE serde_regex1(
-  key decimal,
+  key decimal(65,30),
   value int)
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe'
 WITH SERDEPROPERTIES (

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/udf_pmod.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/udf_pmod.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/udf_pmod.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/udf_pmod.q Tue Oct 29 17:25:55 2013
@@ -16,5 +16,5 @@ SELECT pmod(CAST(-100 AS BIGINT),CAST(9 
 
 SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src LIMIT 1;
 SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src LIMIT 1;
-SELECT pmod(CAST(-100.91 AS DECIMAL),CAST(9.8 AS DECIMAL)), pmod(CAST(-50.1 AS DECIMAL),CAST(101.8 AS DECIMAL)), pmod(CAST(-100.91 AS DECIMAL),CAST(29.75 AS DECIMAL)) FROM src LIMIT 1;
+SELECT pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(9.8 AS DECIMAL(2,1))), pmod(CAST(-50.1 AS DECIMAL(3,1)),CAST(101.8 AS DECIMAL(4,1))), pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(29.75 AS DECIMAL(4,2))) FROM src LIMIT 1;
 

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_double.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_double.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_double.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_double.q Tue Oct 29 17:25:55 2013
@@ -9,7 +9,7 @@ SELECT CAST(-129 AS DOUBLE) FROM src LIM
 SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src LIMIT 1;
 
 SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS DOUBLE) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS DOUBLE) FROM src LIMIT 1;
 
 SELECT CAST('-38.14' AS DOUBLE) FROM src LIMIT 1;
 

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_float.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_float.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_float.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_float.q Tue Oct 29 17:25:55 2013
@@ -9,7 +9,7 @@ SELECT CAST(-129 AS FLOAT) FROM src LIMI
 SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src LIMIT 1;
 
 SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS FLOAT) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS FLOAT) FROM src LIMIT 1;
 
 SELECT CAST('-38.14' AS FLOAT) FROM src LIMIT 1;
 

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_string.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_string.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_string.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/udf_to_string.q Tue Oct 29 17:25:55 2013
@@ -10,7 +10,7 @@ SELECT CAST(CAST(-1025 AS BIGINT) AS STR
 
 SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src LIMIT 1;
 SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS STRING) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS STRING) FROM src LIMIT 1;
 
 SELECT CAST('Foo' AS STRING) FROM src LIMIT 1;
 

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/windowing_expressions.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/windowing_expressions.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/windowing_expressions.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/windowing_expressions.q Tue Oct 29 17:25:55 2013
@@ -27,7 +27,7 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/windowing_multipartitioning.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/windowing_multipartitioning.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/windowing_multipartitioning.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/windowing_multipartitioning.q Tue Oct 29 17:25:55 2013
@@ -10,7 +10,7 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/windowing_navfn.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/windowing_navfn.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/windowing_navfn.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/windowing_navfn.q Tue Oct 29 17:25:55 2013
@@ -9,8 +9,8 @@ create table over10k(
            d double,
            bo boolean,
            s string,
-	   ts timestamp, 
-           dec decimal,  
+           ts timestamp, 
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/windowing_ntile.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/windowing_ntile.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/windowing_ntile.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/windowing_ntile.q Tue Oct 29 17:25:55 2013
@@ -10,7 +10,7 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';

Modified: hive/branches/tez/ql/src/test/queries/clientpositive/windowing_rank.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/queries/clientpositive/windowing_rank.q?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/queries/clientpositive/windowing_rank.q (original)
+++ hive/branches/tez/ql/src/test/queries/clientpositive/windowing_rank.q Tue Oct 29 17:25:55 2013
@@ -10,7 +10,7 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';

Modified: hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_1.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_1.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_1.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tbl (a binary)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@tbl
-FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToInteger with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(double)  _FUNC_(float)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  
+FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToInteger with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(double)  _FUNC_(float)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_2.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_2.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_2.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tbl (a binary)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@tbl
-FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToByte with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(double)  _FUNC_(float)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(void)  
+FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToByte with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(double)  _FUNC_(float)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_3.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_3.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_3.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tbl (a binary)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@tbl
-FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToShort with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(double)  _FUNC_(float)  _FUNC_(int)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  
+FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToShort with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(double)  _FUNC_(float)  _FUNC_(int)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_4.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_4.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_4.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_4.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tbl (a binary)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@tbl
-FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToLong with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(double)  _FUNC_(float)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  
+FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToLong with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(double)  _FUNC_(float)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_5.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_5.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_5.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_5.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tbl (a binary)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@tbl
-FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToFloat with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(double)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  
+FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToFloat with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(double)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_6.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_6.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_6.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/invalid_cast_from_binary_6.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: create table tbl (a binary)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@tbl
-FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToDouble with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(float)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  
+FAILED: SemanticException Line 0:-1 Wrong arguments 'a': No matching method for class org.apache.hadoop.hive.ql.udf.UDFToDouble with (binary). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(float)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientnegative/wrong_column_type.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientnegative/wrong_column_type.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientnegative/wrong_column_type.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientnegative/wrong_column_type.q.out Tue Oct 29 17:25:55 2013
@@ -3,4 +3,4 @@ PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE dest1(a float)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@dest1
-FAILED: NoMatchingMethodException No matching method for class org.apache.hadoop.hive.ql.udf.UDFToFloat with (array<double>). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal)  _FUNC_(double)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  
+FAILED: NoMatchingMethodException No matching method for class org.apache.hadoop.hive.ql.udf.UDFToFloat with (array<double>). Possible choices: _FUNC_(bigint)  _FUNC_(boolean)  _FUNC_(decimal(65,30))  _FUNC_(double)  _FUNC_(int)  _FUNC_(smallint)  _FUNC_(string)  _FUNC_(timestamp)  _FUNC_(tinyint)  _FUNC_(void)  

Modified: hive/branches/tez/ql/src/test/results/clientpositive/decimal_1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientpositive/decimal_1.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientpositive/decimal_1.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientpositive/decimal_1.q.out Tue Oct 29 17:25:55 2013
@@ -2,9 +2,9 @@ PREHOOK: query: drop table decimal_1
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table decimal_1
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: create table decimal_1 (t decimal)
+PREHOOK: query: create table decimal_1 (t decimal(4,2))
 PREHOOK: type: CREATETABLE
-POSTHOOK: query: create table decimal_1 (t decimal)
+POSTHOOK: query: create table decimal_1 (t decimal(4,2))
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@decimal_1
 PREHOOK: query: alter table decimal_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
@@ -16,12 +16,12 @@ POSTHOOK: type: ALTERTABLE_SERIALIZER
 POSTHOOK: Input: default@decimal_1
 POSTHOOK: Output: default@decimal_1
 PREHOOK: query: insert overwrite table decimal_1
-  select cast('17.29' as decimal) from src limit 1
+  select cast('17.29' as decimal(4,2)) from src limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@decimal_1
 POSTHOOK: query: insert overwrite table decimal_1
-  select cast('17.29' as decimal) from src limit 1
+  select cast('17.29' as decimal(4,2)) from src limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@decimal_1

Modified: hive/branches/tez/ql/src/test/results/clientpositive/decimal_2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientpositive/decimal_2.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientpositive/decimal_2.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientpositive/decimal_2.q.out Tue Oct 29 17:25:55 2013
@@ -2,9 +2,9 @@ PREHOOK: query: drop table decimal_2
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table decimal_2
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: create table decimal_2 (t decimal)
+PREHOOK: query: create table decimal_2 (t decimal(18,9))
 PREHOOK: type: CREATETABLE
-POSTHOOK: query: create table decimal_2 (t decimal)
+POSTHOOK: query: create table decimal_2 (t decimal(18,9))
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@decimal_2
 PREHOOK: query: alter table decimal_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'
@@ -16,12 +16,12 @@ POSTHOOK: type: ALTERTABLE_SERIALIZER
 POSTHOOK: Input: default@decimal_2
 POSTHOOK: Output: default@decimal_2
 PREHOOK: query: insert overwrite table decimal_2
-  select cast('17.29' as decimal) from src limit 1
+  select cast('17.29' as decimal(4,2)) from src limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@decimal_2
 POSTHOOK: query: insert overwrite table decimal_2
-  select cast('17.29' as decimal) from src limit 1
+  select cast('17.29' as decimal(4,2)) from src limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@decimal_2
@@ -107,12 +107,12 @@ POSTHOOK: Input: default@decimal_2
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 17.29
 PREHOOK: query: insert overwrite table decimal_2
-  select cast('3404045.5044003' as decimal) from src limit 1
+  select cast('3404045.5044003' as decimal(18,9)) from src limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@decimal_2
 POSTHOOK: query: insert overwrite table decimal_2
-  select cast('3404045.5044003' as decimal) from src limit 1
+  select cast('3404045.5044003' as decimal(18,9)) from src limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@decimal_2
@@ -206,33 +206,33 @@ POSTHOOK: Input: default@decimal_2
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 3404045.5044003
-PREHOOK: query: select cast(3.14 as decimal) from decimal_2 limit 1
+PREHOOK: query: select cast(3.14 as decimal(4,2)) from decimal_2 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-POSTHOOK: query: select cast(3.14 as decimal) from decimal_2 limit 1
+POSTHOOK: query: select cast(3.14 as decimal(4,2)) from decimal_2 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 3.14
-PREHOOK: query: select cast(cast(3.14 as float) as decimal) from decimal_2 limit 1
+PREHOOK: query: select cast(cast(3.14 as float) as decimal(4,2)) from decimal_2 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-POSTHOOK: query: select cast(cast(3.14 as float) as decimal) from decimal_2 limit 1
+POSTHOOK: query: select cast(cast(3.14 as float) as decimal(4,2)) from decimal_2 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 3.14
-PREHOOK: query: select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal) from decimal_2 limit 1
+PREHOOK: query: select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) from decimal_2 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-POSTHOOK: query: select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal) from decimal_2 limit 1
+POSTHOOK: query: select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) from decimal_2 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
@@ -294,22 +294,22 @@ POSTHOOK: Input: default@decimal_2
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 3
-PREHOOK: query: select cast(0.99999999999999999999 as decimal) from decimal_2 limit 1
+PREHOOK: query: select cast(0.99999999999999999999 as decimal(20,19)) from decimal_2 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-POSTHOOK: query: select cast(0.99999999999999999999 as decimal) from decimal_2 limit 1
+POSTHOOK: query: select cast(0.99999999999999999999 as decimal(20,19)) from decimal_2 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 POSTHOOK: Lineage: decimal_2.t EXPRESSION []
 1
-PREHOOK: query: select cast('0.99999999999999999999' as decimal) from decimal_2 limit 1
+PREHOOK: query: select cast('0.99999999999999999999' as decimal(20,20)) from decimal_2 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-POSTHOOK: query: select cast('0.99999999999999999999' as decimal) from decimal_2 limit 1
+POSTHOOK: query: select cast('0.99999999999999999999' as decimal(20,20)) from decimal_2 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####

Modified: hive/branches/tez/ql/src/test/results/clientpositive/decimal_3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientpositive/decimal_3.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientpositive/decimal_3.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientpositive/decimal_3.q.out Tue Oct 29 17:25:55 2013
@@ -2,12 +2,12 @@ PREHOOK: query: DROP TABLE IF EXISTS DEC
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: DROP TABLE IF EXISTS DECIMAL_3
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: CREATE TABLE DECIMAL_3(key decimal, value int) 
+PREHOOK: query: CREATE TABLE DECIMAL_3(key decimal(65,30), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
-POSTHOOK: query: CREATE TABLE DECIMAL_3(key decimal, value int) 
+POSTHOOK: query: CREATE TABLE DECIMAL_3(key decimal(65,30), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE
@@ -28,7 +28,6 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_3
 #### A masked pattern was here ####
 NULL	0
-NULL	0
 -1234567890.123456789	-1234567890
 -4400	4400
 -1255.49	-1255
@@ -40,6 +39,7 @@ NULL	0
 -0.3	0
 0	0
 0	0
+0	0
 0.01	0
 0.02	0
 0.1	0
@@ -100,6 +100,7 @@ POSTHOOK: Input: default@decimal_3
 0.01	0
 0	0
 0	0
+0	0
 -0.3	0
 -0.33	0
 -0.333	0
@@ -110,7 +111,6 @@ POSTHOOK: Input: default@decimal_3
 -4400	4400
 -1234567890.123456789	-1234567890
 NULL	0
-NULL	0
 PREHOOK: query: SELECT * FROM DECIMAL_3 ORDER BY key, value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_3
@@ -120,7 +120,6 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_3
 #### A masked pattern was here ####
 NULL	0
-NULL	0
 -1234567890.123456789	-1234567890
 -4400	4400
 -1255.49	-1255
@@ -132,6 +131,7 @@ NULL	0
 -0.3	0
 0	0
 0	0
+0	0
 0.01	0
 0.02	0
 0.1	0
@@ -281,6 +281,11 @@ POSTHOOK: Input: default@decimal_3
 0	0	0	0
 0	0	0	0
 0	0	0	0
+0	0	0	0
+0	0	0	0
+0	0	0	0
+0	0	0	0
+0	0	0	0
 0.01	0	0.01	0
 0.02	0	0.02	0
 0.1	0	0.1	0

Modified: hive/branches/tez/ql/src/test/results/clientpositive/decimal_4.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientpositive/decimal_4.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientpositive/decimal_4.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientpositive/decimal_4.q.out Tue Oct 29 17:25:55 2013
@@ -6,21 +6,21 @@ PREHOOK: query: DROP TABLE IF EXISTS DEC
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: DROP TABLE IF EXISTS DECIMAL_4_2
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: CREATE TABLE DECIMAL_4_1(key decimal, value int) 
+PREHOOK: query: CREATE TABLE DECIMAL_4_1(key decimal(35,25), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
-POSTHOOK: query: CREATE TABLE DECIMAL_4_1(key decimal, value int) 
+POSTHOOK: query: CREATE TABLE DECIMAL_4_1(key decimal(35,25), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@DECIMAL_4_1
-PREHOOK: query: CREATE TABLE DECIMAL_4_2(key decimal, value decimal) 
+PREHOOK: query: CREATE TABLE DECIMAL_4_2(key decimal(35,25), value decimal(35,25)) 
 STORED AS ORC
 PREHOOK: type: CREATETABLE
-POSTHOOK: query: CREATE TABLE DECIMAL_4_2(key decimal, value decimal) 
+POSTHOOK: query: CREATE TABLE DECIMAL_4_2(key decimal(35,25), value decimal(35,25)) 
 STORED AS ORC
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@DECIMAL_4_2
@@ -38,8 +38,8 @@ POSTHOOK: query: INSERT OVERWRITE TABLE 
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_4_1
 POSTHOOK: Output: default@decimal_4_2
-POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
 PREHOOK: query: SELECT * FROM DECIMAL_4_1 ORDER BY key, value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_4_1
@@ -48,9 +48,8 @@ POSTHOOK: query: SELECT * FROM DECIMAL_4
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_4_1
 #### A masked pattern was here ####
-POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-NULL	0
+POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
 NULL	0
 -1234567890.123456789	-1234567890
 -4400	4400
@@ -63,6 +62,7 @@ NULL	0
 -0.3	0
 0	0
 0	0
+0	0
 0.01	0
 0.02	0
 0.1	0
@@ -96,9 +96,8 @@ POSTHOOK: query: SELECT * FROM DECIMAL_4
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_4_2
 #### A masked pattern was here ####
-POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-NULL	NULL
+POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
 NULL	NULL
 -1234567890.123456789	-3703703670.370370367
 -4400	-13200
@@ -111,6 +110,7 @@ NULL	NULL
 -0.3	-0.9
 0	0
 0	0
+0	0
 0.01	0.03
 0.02	0.06
 0.1	0.3
@@ -144,8 +144,8 @@ POSTHOOK: query: DROP TABLE DECIMAL_4_1
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@decimal_4_1
 POSTHOOK: Output: default@decimal_4_1
-POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
 PREHOOK: query: DROP TABLE DECIMAL_4_2
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@decimal_4_2
@@ -154,5 +154,5 @@ POSTHOOK: query: DROP TABLE DECIMAL_4_2
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@decimal_4_2
 POSTHOOK: Output: default@decimal_4_2
-POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
-POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal, comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.key SIMPLE [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]
+POSTHOOK: Lineage: decimal_4_2.value EXPRESSION [(decimal_4_1)decimal_4_1.FieldSchema(name:key, type:decimal(35,25), comment:null), ]

Modified: hive/branches/tez/ql/src/test/results/clientpositive/decimal_join.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/results/clientpositive/decimal_join.q.out?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/results/clientpositive/decimal_join.q.out (original)
+++ hive/branches/tez/ql/src/test/results/clientpositive/decimal_join.q.out Tue Oct 29 17:25:55 2013
@@ -1,10 +1,10 @@
 PREHOOK: query: -- HIVE-5292 Join on decimal columns fails
 
-create table src_dec (key decimal, value string)
+create table src_dec (key decimal(3,0), value string)
 PREHOOK: type: CREATETABLE
 POSTHOOK: query: -- HIVE-5292 Join on decimal columns fails
 
-create table src_dec (key decimal, value string)
+create table src_dec (key decimal(3,0), value string)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@src_dec
 PREHOOK: query: load data local inpath '../data/files/kv1.txt' into table src_dec