You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2013/09/17 22:02:23 UTC

svn commit: r1524198 [5/5] - in /hive/branches/branch-0.12: common/src/java/org/apache/hadoop/hive/common/type/ common/src/test/org/apache/hadoop/hive/common/type/ data/files/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hi...

Modified: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java?rev=1524198&r1=1524197&r2=1524198&view=diff
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java (original)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java Tue Sep 17 20:02:21 2013
@@ -29,12 +29,14 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDeException;
 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.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.lazy.LazyInteger;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.BaseTypeParams;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeSpec;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeParams;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -172,7 +175,7 @@ public final class PrimitiveObjectInspec
         return result;
       } catch (Exception err) {
         LOG.error("Error while setting type parameters: " + err);
-        return null;
+        throw new RuntimeException(err);
       }
     }
 
@@ -212,7 +215,8 @@ public final class PrimitiveObjectInspec
           return null;
         }
       } catch (Exception err) {
-        throw new SerDeException("Error creating type params for " + typeName, err);
+        throw new SerDeException("Error creating type params for " + typeName
+            + ": " + err, err);
       }
     }
 
@@ -296,6 +300,9 @@ public final class PrimitiveObjectInspec
   public static final PrimitiveTypeEntry decimalTypeEntry = new PrimitiveTypeEntry(
       PrimitiveCategory.DECIMAL, serdeConstants.DECIMAL_TYPE_NAME, null,
       HiveDecimal.class, HiveDecimalWritable.class, null);
+  public static final PrimitiveTypeEntry varcharTypeEntry = new PrimitiveTypeEntry(
+      PrimitiveCategory.VARCHAR, serdeConstants.VARCHAR_TYPE_NAME, null, HiveVarchar.class,
+      HiveVarcharWritable.class, VarcharTypeParams.class);
 
   // The following is a complex type for special handling
   public static final PrimitiveTypeEntry unknownTypeEntry = new PrimitiveTypeEntry(
@@ -304,6 +311,7 @@ public final class PrimitiveObjectInspec
   static {
     registerType(binaryTypeEntry);
     registerType(stringTypeEntry);
+    registerType(varcharTypeEntry);
     registerType(booleanTypeEntry);
     registerType(intTypeEntry);
     registerType(longTypeEntry);
@@ -428,14 +436,24 @@ public final class PrimitiveObjectInspec
   public static PrimitiveTypeEntry getTypeEntryFromTypeSpecs(
       PrimitiveCategory primitiveCategory,
       BaseTypeParams typeParams) {
-    String typeString = primitiveCategory.toString().toLowerCase();
-    if (typeParams != null) {
-      typeString += typeParams.toString();
+    if (typeParams == null) {
+      // No type params, can just use the primitive category
+      return getTypeEntryFromPrimitiveCategory(primitiveCategory);
     }
+
+    // Type params were passed in. First check for cached version
+    String typeString = primitiveCategory.toString().toLowerCase();
+    typeString += typeParams.toString();
     PrimitiveTypeEntry typeEntry = getTypeEntryFromTypeName(typeString);
     if (typeEntry == null) {
       // Parameterized type doesn't exist yet, create now.
-      typeEntry = (PrimitiveTypeEntry)getTypeEntryFromTypeSpecs(primitiveCategory, null).clone();
+      typeEntry = 
+          (PrimitiveTypeEntry) getTypeEntryFromPrimitiveCategory(primitiveCategory).clone();
+      if (!typeEntry.isParameterized()) {
+        throw new IllegalArgumentException(
+            primitiveCategory + " type was being used with type parameters "
+            + typeParams + ", which should not be allowed");
+      }
       typeEntry.typeParams = typeParams;
       addParameterizedType(typeEntry);
     }
@@ -491,6 +509,10 @@ public final class PrimitiveObjectInspec
           .getPrimitiveWritableObject(o2);
       return t1.equals(t2);
     }
+    case VARCHAR: {
+      return ((HiveVarcharObjectInspector)oi1).getPrimitiveWritableObject(o1)
+          .equals(((HiveVarcharObjectInspector)oi2).getPrimitiveWritableObject(o2));
+    }
     case DATE: {
       return ((DateObjectInspector) oi1).getPrimitiveWritableObject(o1)
           .equals(((DateObjectInspector) oi2).getPrimitiveWritableObject(o2));
@@ -694,6 +716,10 @@ public final class PrimitiveObjectInspec
       }
       break;
     }
+    case VARCHAR: {
+      result = Integer.parseInt(getString(o, oi));
+      break;
+    }
     case TIMESTAMP:
       result = (int) (((TimestampObjectInspector) oi)
           .getPrimitiveWritableObject(o).getSeconds());
@@ -753,6 +779,10 @@ public final class PrimitiveObjectInspec
         result = Long.parseLong(s);
       }
       break;
+    case VARCHAR: {
+      result = Long.parseLong(getString(o, oi));
+      break;
+    }
     case TIMESTAMP:
       result = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o)
           .getSeconds();
@@ -806,6 +836,9 @@ public final class PrimitiveObjectInspec
       String s = soi.getPrimitiveJavaObject(o);
       result = Double.parseDouble(s);
       break;
+    case VARCHAR:
+      result = Double.parseDouble(getString(o, oi));
+      break;
     case TIMESTAMP:
       result = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o).getDouble();
       break;
@@ -871,6 +904,10 @@ public final class PrimitiveObjectInspec
       StringObjectInspector soi = (StringObjectInspector) oi;
       result = soi.getPrimitiveJavaObject(o);
       break;
+    case VARCHAR:
+      HiveVarcharObjectInspector hcoi = (HiveVarcharObjectInspector) oi;
+      result = hcoi.getPrimitiveJavaObject(o).toString();
+      break;
     case DATE:
       result = ((DateObjectInspector) oi).getPrimitiveWritableObject(o).toString();
       break;
@@ -888,6 +925,28 @@ public final class PrimitiveObjectInspec
     return result;
   }
 
+  public static HiveVarchar getHiveVarchar(Object o, PrimitiveObjectInspector oi) {
+
+    if (o == null) {
+      return null;
+    }
+
+    HiveVarchar result = null;
+    switch (oi.getPrimitiveCategory()) {
+      case VARCHAR:
+        result = ((HiveVarcharObjectInspector)oi).getPrimitiveJavaObject(o);
+        break;
+      default:
+        // Is there a way to provide char length here?
+        // It might actually be ok as long as there is an object inspector (with char length)
+        // receiving this value.
+        result = new HiveVarchar();
+        result.setValue(getString(o, oi));
+        break;
+    }
+    return result;
+  }
+
   public static BytesWritable getBinary(Object o, PrimitiveObjectInspector oi) {
 
     if (null == o) {
@@ -951,6 +1010,9 @@ public final class PrimitiveObjectInspec
     case STRING:
       result = new HiveDecimal(((StringObjectInspector) oi).getPrimitiveJavaObject(o));
       break;
+    case VARCHAR:
+      result = new HiveDecimal(getString(o, oi));
+      break;
     case TIMESTAMP:
       Double ts = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o)
         .getDouble();
@@ -986,6 +1048,15 @@ public final class PrimitiveObjectInspec
         result = null;
       }
       break;
+    case VARCHAR: {
+      try {
+        String val = getString(o, oi).trim();
+        result = Date.valueOf(val);
+      } catch (IllegalArgumentException e) {
+        result = null;
+      }
+      break;
+    }
     case DATE:
       result = ((DateObjectInspector) oi).getPrimitiveWritableObject(o).get();
       break;
@@ -1041,6 +1112,9 @@ public final class PrimitiveObjectInspec
       String s = soi.getPrimitiveJavaObject(o);
       result = getTimestampFromString(s);
       break;
+    case VARCHAR:
+      result = getTimestampFromString(getString(o, oi));
+      break;
     case DATE:
       result = new Timestamp(
           ((DateObjectInspector) oi).getPrimitiveWritableObject(o).get().getTime());
@@ -1109,6 +1183,7 @@ public final class PrimitiveObjectInspec
       case DECIMAL:
         return PrimitiveGrouping.NUMERIC_GROUP;
       case STRING:
+      case VARCHAR:
         return PrimitiveGrouping.STRING_GROUP;
       case BOOLEAN:
         return PrimitiveGrouping.BOOLEAN_GROUP;

Added: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableHiveVarcharObjectInspector.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableHiveVarcharObjectInspector.java?rev=1524198&view=auto
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableHiveVarcharObjectInspector.java (added)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableHiveVarcharObjectInspector.java Tue Sep 17 20:02:21 2013
@@ -0,0 +1,30 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+
+public interface SettableHiveVarcharObjectInspector extends HiveVarcharObjectInspector {
+  Object set(Object o, HiveVarchar value);
+
+  Object set(Object o, String value);
+
+  Object create(HiveVarchar value);
+
+}

Added: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveVarcharObjectInspector.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveVarcharObjectInspector.java?rev=1524198&view=auto
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveVarcharObjectInspector.java (added)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveVarcharObjectInspector.java Tue Sep 17 20:02:21 2013
@@ -0,0 +1,59 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeParams;
+
+/**
+ * A WritableConstantHiveVarcharObjectInspector is a WritableHiveVarcharObjectInspector
+ * that implements ConstantObjectInspector.
+ */
+public class WritableConstantHiveVarcharObjectInspector extends
+    WritableHiveVarcharObjectInspector implements
+    ConstantObjectInspector {
+
+  protected HiveVarcharWritable value;
+
+  WritableConstantHiveVarcharObjectInspector(HiveVarcharWritable value) {
+    this(value, null);
+  }
+
+  WritableConstantHiveVarcharObjectInspector(HiveVarcharWritable value,
+      VarcharTypeParams typeParams) {
+    super(PrimitiveObjectInspectorUtils.varcharTypeEntry);
+    this.value = value;
+
+    // If we have been provided with type params, then use them.
+    // Otherwise determine character length and update type params/typeinfo accordingly.
+    if (typeParams == null) {
+      typeParams = new VarcharTypeParams();
+      typeParams.length = this.value.getCharacterLength();
+    }
+    setTypeParams(typeParams);
+    this.typeEntry = PrimitiveObjectInspectorUtils.getTypeEntryFromTypeSpecs(
+        PrimitiveCategory.VARCHAR,
+        typeParams);
+  }
+
+  @Override
+  public HiveVarcharWritable getWritableConstantValue() {
+    return value;
+  }
+}

Added: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableHiveVarcharObjectInspector.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableHiveVarcharObjectInspector.java?rev=1524198&view=auto
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableHiveVarcharObjectInspector.java (added)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableHiveVarcharObjectInspector.java Tue Sep 17 20:02:21 2013
@@ -0,0 +1,131 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
+import org.apache.hadoop.hive.serde2.typeinfo.BaseTypeParams;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeParams;
+import org.apache.hadoop.hive.serde2.typeinfo.ParameterizedPrimitiveTypeUtils;
+
+public class WritableHiveVarcharObjectInspector
+    extends AbstractPrimitiveWritableObjectInspector
+    implements SettableHiveVarcharObjectInspector {
+
+  private static final Log LOG = LogFactory.getLog(WritableHiveVarcharObjectInspector.class);
+
+  public WritableHiveVarcharObjectInspector(PrimitiveTypeEntry typeEntry) {
+    super(typeEntry);
+    if (typeEntry.primitiveCategory != PrimitiveCategory.VARCHAR) {
+      throw new RuntimeException(
+          "TypeEntry of type varchar expected, got " + typeEntry.primitiveCategory);
+    }
+  }
+
+  @Override
+  public HiveVarchar getPrimitiveJavaObject(Object o) {
+    // check input object's length, if it doesn't match
+    // then output a new primitive with the correct params.
+    if (o == null) {
+      return null;
+    }
+    HiveVarcharWritable writable = ((HiveVarcharWritable)o);
+    if (doesWritableMatchTypeParams(writable)) {
+      return writable.getHiveVarchar();
+    }
+    return getPrimitiveWithParams(writable);
+  }
+
+  public HiveVarcharWritable getPrimitiveWritableObject(Object o) {
+    // check input object's length, if it doesn't match
+    // then output new writable with correct params.
+    if (o == null) {
+      return null;
+    }
+    HiveVarcharWritable writable = ((HiveVarcharWritable)o);
+    if (doesWritableMatchTypeParams((HiveVarcharWritable)o)) {
+      return writable;
+    }
+
+    return getWritableWithParams(writable);
+  }
+
+  private HiveVarchar getPrimitiveWithParams(HiveVarcharWritable val) {
+    HiveVarchar hv = new HiveVarchar();
+    hv.setValue(val.getHiveVarchar(), getMaxLength());
+    return hv;
+  }
+
+  private HiveVarcharWritable getWritableWithParams(HiveVarcharWritable val) {
+    HiveVarcharWritable newValue = new HiveVarcharWritable();
+    newValue.set(val, getMaxLength());
+    return newValue;
+  }
+
+  private boolean doesWritableMatchTypeParams(HiveVarcharWritable writable) {
+    return ParameterizedPrimitiveTypeUtils.doesWritableMatchTypeParams(
+        writable, (VarcharTypeParams) typeParams);
+  }
+
+  private boolean doesPrimitiveMatchTypeParams(HiveVarchar value) {
+    return ParameterizedPrimitiveTypeUtils.doesPrimitiveMatchTypeParams(
+        value, (VarcharTypeParams) typeParams);
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    if (o == null) {
+      return null;
+    }
+    HiveVarcharWritable writable = (HiveVarcharWritable)o;
+    if (doesWritableMatchTypeParams((HiveVarcharWritable)o)) {
+      return new HiveVarcharWritable(writable);
+    }
+    return getWritableWithParams(writable);
+  }
+
+  @Override
+  public Object set(Object o, HiveVarchar value) {
+    HiveVarcharWritable writable = (HiveVarcharWritable)o;
+    writable.set(value, getMaxLength());
+    return o;
+  }
+
+  @Override
+  public Object set(Object o, String value) {
+    HiveVarcharWritable writable = (HiveVarcharWritable)o;
+    writable.set(value, getMaxLength());
+    return o;
+  }
+
+  @Override
+  public Object create(HiveVarchar value) {
+    HiveVarcharWritable ret;
+    ret = new HiveVarcharWritable();
+    ret.set(value, getMaxLength());
+    return ret;
+  }
+
+  public int getMaxLength() {
+    return typeParams != null ? ((VarcharTypeParams) typeParams).length : -1;
+  }
+}

Modified: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ParameterizedPrimitiveTypeUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ParameterizedPrimitiveTypeUtils.java?rev=1524198&r1=1524197&r2=1524198&view=diff
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ParameterizedPrimitiveTypeUtils.java (original)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/ParameterizedPrimitiveTypeUtils.java Tue Sep 17 20:02:21 2013
@@ -1,5 +1,9 @@
 package org.apache.hadoop.hive.serde2.typeinfo;
 
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
 
@@ -40,4 +44,28 @@ public class ParameterizedPrimitiveTypeU
     return oi.getTypeParams();
   }
 
+  /**
+   * Utils for varchar type
+   */
+  public static class HiveVarcharSerDeHelper {
+    public int maxLength;
+    public HiveVarcharWritable writable = new HiveVarcharWritable();
+
+    public HiveVarcharSerDeHelper(VarcharTypeParams typeParams) {
+      if (typeParams == null) {
+        throw new RuntimeException("varchar type used without type params");
+      }
+      maxLength = typeParams.getLength();
+    }
+  }
+
+  public static boolean doesWritableMatchTypeParams(HiveVarcharWritable writable,
+      VarcharTypeParams typeParams) {
+    return (typeParams == null || typeParams.length >= writable.getCharacterLength());
+  }
+
+  public static boolean doesPrimitiveMatchTypeParams(HiveVarchar value,
+      VarcharTypeParams typeParams) {
+    return (typeParams == null || typeParams.length == value.getCharacterLength());
+  }
 }

Modified: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java?rev=1524198&r1=1524197&r2=1524198&view=diff
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java (original)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfo.java Tue Sep 17 20:02:21 2013
@@ -49,6 +49,15 @@ public abstract class TypeInfo implement
    */
   public abstract String getTypeName();
 
+  /**
+   * String representing the qualified type name.
+   * Qualified types should override this method.
+   * @return
+   */
+  public String getQualifiedName() {
+    return getTypeName();
+  }
+
   @Override
   public String toString() {
     return getTypeName();

Modified: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java?rev=1524198&r1=1524197&r2=1524198&view=diff
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java (original)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java Tue Sep 17 20:02:21 2013
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentHa
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
 
@@ -67,6 +68,16 @@ public final class TypeInfoFactory {
         }
       } else {
         // No type params
+
+        // Prevent creation of varchar TypeInfo with no length specification.
+        // This can happen if an old-style UDF uses a varchar type either as an
+        // argument or return type in an evaluate() function, or other instances
+        // of using reflection-based methods for retrieving a TypeInfo.
+        if (typeEntry.primitiveCategory == PrimitiveCategory.VARCHAR) {
+          LOG.error("varchar type used with no type params");
+          throw new RuntimeException("varchar type used with no type params");
+        }
+
         result = new PrimitiveTypeInfo(parts.typeName);
       }
 

Modified: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java?rev=1524198&r1=1524197&r2=1524198&view=diff
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java (original)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java Tue Sep 17 20:02:21 2013
@@ -29,6 +29,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
@@ -736,4 +738,24 @@ public final class TypeInfoUtils {
     }
     return true;
   }
+
+  /**
+   * Return the character length of the type
+   * @param typeInfo
+   * @return
+   */
+  public static int getCharacterLengthForType(PrimitiveTypeInfo typeInfo) {
+    switch (typeInfo.getPrimitiveCategory()) {
+      case STRING:
+        return HiveVarchar.MAX_VARCHAR_LENGTH;
+      case VARCHAR:
+        VarcharTypeParams varcharParams = (VarcharTypeParams) typeInfo.getTypeParams();
+        if (varcharParams == null) {
+          throw new RuntimeException("varchar type used without type params");
+        }
+        return varcharParams.getLength();
+      default:
+        return 0;
+    }
+  }
 }

Added: hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeParams.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeParams.java?rev=1524198&view=auto
==============================================================================
--- hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeParams.java (added)
+++ hive/branches/branch-0.12/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/VarcharTypeParams.java Tue Sep 17 20:02:21 2013
@@ -0,0 +1,97 @@
+/**
+ * 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.serde2.typeinfo;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.io.WritableUtils;
+
+public class VarcharTypeParams extends BaseTypeParams implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  public int length;
+
+  @Override
+  public void validateParams() throws SerDeException {
+    if (length < 1) {
+      throw new SerDeException("VARCHAR length must be positive");
+    }
+    if (length > HiveVarchar.MAX_VARCHAR_LENGTH) {
+      throw new SerDeException("Length " + length
+          + " exceeds max varchar length of " + HiveVarchar.MAX_VARCHAR_LENGTH);
+    }
+  }
+
+  @Override
+  public void populateParams(String[] params) throws SerDeException {
+    if (params.length != 1) {
+      throw new SerDeException("Invalid number of parameters for VARCHAR");
+    }
+    try {
+      length = Integer.valueOf(params[0]);
+    } catch (NumberFormatException err) {
+      throw new SerDeException("Error setting VARCHAR length: " + err);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("(");
+    sb.append(length);
+    sb.append(")");
+    return sb.toString();
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    length = WritableUtils.readVInt(in);
+    try {
+      validateParams();
+    } catch (SerDeException err) {
+      throw new IOException(err);
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out,  length);
+  }
+
+  public int getLength() {
+    return length;
+  }
+
+  public void setLength(int len) {
+    length = len;
+  }
+
+  @Override
+  public boolean hasCharacterMaximumLength() {
+    return true;
+  }
+  @Override
+  public int getCharacterMaximumLength() {
+    return length;
+  }
+}

Added: hive/branches/branch-0.12/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java?rev=1524198&view=auto
==============================================================================
--- hive/branches/branch-0.12/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java (added)
+++ hive/branches/branch-0.12/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java Tue Sep 17 20:02:21 2013
@@ -0,0 +1,71 @@
+/**
+ * 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.serde2.typeinfo;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+
+public class TestTypeInfoUtils extends TestCase {
+
+  static void parseTypeString(String typeString, boolean exceptionExpected) {
+    boolean caughtException = false;
+    try {
+      TypeInfoUtils.getTypeInfoFromTypeString(typeString);
+    } catch (IllegalArgumentException err) {
+      caughtException = true;
+    }
+    assertEquals("parsing typestring " + typeString, exceptionExpected, caughtException);
+  }
+
+  public void testTypeInfoParser() {
+    String[] validTypeStrings = {
+        "int",
+        "string",
+        "varchar(10)",
+        "array<int>"
+    };
+
+    String[] invalidTypeStrings = {
+        "array<",
+        "varchar(123",
+        "varchar(123,",
+        "varchar()",
+        "varchar("
+    };
+
+    for (String typeString : validTypeStrings) {
+      parseTypeString(typeString, false);
+    }
+    for (String typeString : invalidTypeStrings) {
+      parseTypeString(typeString, true);
+    }
+  }
+
+  public void testVarcharNoParams() {
+    boolean caughtException = false;
+    try {
+      TypeInfoUtils.getTypeInfoFromTypeString("varchar");
+    } catch (Exception err) {
+      caughtException = true;
+    }
+    assertEquals("varchar TypeInfo with no params should fail", true, caughtException);
+  }
+}