You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/03/20 02:40:58 UTC

svn commit: r1579501 - in /hive/branches/branch-0.13/ql/src: java/org/apache/hadoop/hive/ql/udf/generic/ test/org/apache/hadoop/hive/ql/udf/ test/org/apache/hadoop/hive/ql/udf/generic/

Author: hashutosh
Date: Thu Mar 20 01:40:57 2014
New Revision: 1579501

URL: http://svn.apache.org/r1579501
Log:
HIVE-6645 : to_date()/to_unix_timestamp() fail with NPE if input is null (Jason Dere via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java
Modified:
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToUnixTimeStamp.java
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/TestGenericUDFDate.java

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java?rev=1579501&r1=1579500&r2=1579501&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java Thu Mar 20 01:40:57 2014
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.serde2.io.
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
@@ -63,14 +64,20 @@ public class GenericUDFDate extends Gene
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
     if (arguments.length != 1) {
       throw new UDFArgumentLengthException(
-          "to_date() requires 1 argument, got " + arguments.length);
+        "to_date() requires 1 argument, got " + arguments.length);
+    }
+    if (arguments[0].getCategory() != Category.PRIMITIVE) {
+      throw new UDFArgumentException("to_date() only accepts STRING/TIMESTAMP/DATEWRITABLE types, got "
+          + arguments[0].getTypeName());
     }
     argumentOI = (PrimitiveObjectInspector) arguments[0];
     inputType = argumentOI.getPrimitiveCategory();
     ObjectInspector outputOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
     switch (inputType) {
+    case CHAR:
+    case VARCHAR:
     case STRING:
-      // textConverter = new TextConverter(argumentOI);
+      inputType = PrimitiveCategory.STRING;
       textConverter = ObjectInspectorConverters.getConverter(
         argumentOI, PrimitiveObjectInspectorFactory.writableStringObjectInspector);
       break;
@@ -91,6 +98,10 @@ public class GenericUDFDate extends Gene
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    if (arguments[0].get() == null) {
+      return null;
+    }
+
     switch (inputType) {
     case STRING:
       Date date;

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToUnixTimeStamp.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToUnixTimeStamp.java?rev=1579501&r1=1579500&r2=1579501&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToUnixTimeStamp.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToUnixTimeStamp.java Thu Mar 20 01:40:57 2014
@@ -31,11 +31,18 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampString;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
 
 /**
  * deterministic version of UDFUnixTimeStamp. enforces argument
@@ -46,10 +53,10 @@ import org.apache.hadoop.io.LongWritable
 @VectorizedExpressions({VectorUDFUnixTimeStampLong.class, VectorUDFUnixTimeStampString.class})
 public class GenericUDFToUnixTimeStamp extends GenericUDF {
 
-  private transient StringObjectInspector intputTextOI;
   private transient DateObjectInspector inputDateOI;
   private transient TimestampObjectInspector inputTimestampOI;
-  private transient StringObjectInspector patternOI;
+  private transient Converter inputTextConverter;
+  private transient Converter patternConverter;
 
   private transient String lasPattern = "yyyy-MM-dd HH:mm:ss";
   private transient final SimpleDateFormat formatter = new SimpleDateFormat(lasPattern);
@@ -62,26 +69,44 @@ public class GenericUDFToUnixTimeStamp e
 
   protected void initializeInput(ObjectInspector[] arguments) throws UDFArgumentException {
     if (arguments.length < 1) {
-      throw new UDFArgumentLengthException("The function TO_UNIX_TIMESTAMP " +
+      throw new UDFArgumentLengthException("The function " + getName().toUpperCase() +
           "requires at least one argument");
     }
+    for (ObjectInspector argument : arguments) {
+      if (arguments[0].getCategory() != Category.PRIMITIVE) {
+        throw new UDFArgumentException(getName().toUpperCase() +
+            " only takes string/date/timestamp types, got " + argument.getTypeName());
+      }
+    }
 
-    if (arguments[0] instanceof StringObjectInspector) {
-      intputTextOI = (StringObjectInspector) arguments[0];
-      if (arguments.length > 1) {
-        if (!(arguments[1] instanceof StringObjectInspector)) {
-          throw new UDFArgumentException(
-            "The time pattern for " + getName().toUpperCase() + " should be string type");
+    PrimitiveObjectInspector arg1OI = (PrimitiveObjectInspector) arguments[0];
+    switch (arg1OI.getPrimitiveCategory()) {
+      case CHAR:
+      case VARCHAR:
+      case STRING:
+        inputTextConverter = ObjectInspectorConverters.getConverter(arg1OI,
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+        if (arguments.length > 1) {
+          PrimitiveObjectInspector arg2OI = (PrimitiveObjectInspector) arguments[1];
+          if (PrimitiveObjectInspectorUtils.getPrimitiveGrouping(arg2OI.getPrimitiveCategory())
+              != PrimitiveGrouping.STRING_GROUP) {
+            throw new UDFArgumentException(
+              "The time pattern for " + getName().toUpperCase() + " should be string type");
+          }
+          patternConverter = ObjectInspectorConverters.getConverter(arg2OI,
+              PrimitiveObjectInspectorFactory.javaStringObjectInspector);
         }
-        patternOI = (StringObjectInspector) arguments[1];
-      }
-    } else if (arguments[0] instanceof DateObjectInspector) {
-      inputDateOI = (DateObjectInspector) arguments[0];
-    } else if (arguments[0] instanceof TimestampObjectInspector) {
-      inputTimestampOI = (TimestampObjectInspector) arguments[0];
-    } else {
-      throw new UDFArgumentException(
-          "The function " + getName().toUpperCase() + " takes only string or timestamp types");
+        break;
+
+      case DATE:
+        inputDateOI = (DateObjectInspector) arguments[0];
+        break;
+      case TIMESTAMP:
+        inputTimestampOI = (TimestampObjectInspector) arguments[0];
+        break;
+      default:
+        throw new UDFArgumentException(
+            "The function " + getName().toUpperCase() + " takes only string/date/timestamp types");
     }
   }
 
@@ -93,13 +118,20 @@ public class GenericUDFToUnixTimeStamp e
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    if (intputTextOI != null) {
-      String textVal = intputTextOI.getPrimitiveJavaObject(arguments[0].get());
+    if (arguments[0].get() == null) {
+      return null;
+    }
+
+    if (inputTextConverter != null) {
+      String textVal = (String) inputTextConverter.convert(arguments[0].get());
       if (textVal == null) {
         return null;
       }
-      if (patternOI != null) {
-        String patternVal = patternOI.getPrimitiveJavaObject(arguments[1].get());
+      if (patternConverter != null) {
+        if (arguments[1].get() == null) {
+          return null;
+        }
+        String patternVal = (String) patternConverter.convert(arguments[1].get());
         if (patternVal == null) {
           return null;
         }
@@ -118,7 +150,7 @@ public class GenericUDFToUnixTimeStamp e
       retValue.set(inputDateOI.getPrimitiveWritableObject(arguments[0].get())
                    .getTimeInSeconds());
       return retValue;
-	}
+    }
     Timestamp timestamp = inputTimestampOI.getPrimitiveJavaObject(arguments[0].get());
     retValue.set(timestamp.getTime() / 1000);
     return retValue;

Modified: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/TestGenericUDFDate.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/TestGenericUDFDate.java?rev=1579501&r1=1579500&r2=1579501&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/TestGenericUDFDate.java (original)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/TestGenericUDFDate.java Thu Mar 20 01:40:57 2014
@@ -45,6 +45,11 @@ public class TestGenericUDFDate extends 
     Text output = (Text) udf.evaluate(args);
 
     assertEquals("to_date() test for STRING failed ", "2009-07-30", output.toString());
+
+    // Try with null args
+    DeferredObject[] nullArgs = { new DeferredJavaObject(null) };
+    output = (Text) udf.evaluate(nullArgs);
+    assertNull("to_date() with null STRING", output);
   }
 
   public void testTimestampToDate() throws HiveException {
@@ -59,6 +64,11 @@ public class TestGenericUDFDate extends 
     Text output = (Text) udf.evaluate(args);
 
     assertEquals("to_date() test for TIMESTAMP failed ", "2009-07-30", output.toString());
+
+    // Try with null args
+    DeferredObject[] nullArgs = { new DeferredJavaObject(null) };
+    output = (Text) udf.evaluate(nullArgs);
+    assertNull("to_date() with null TIMESTAMP", output);
   }
 
   public void testDateWritablepToDate() throws HiveException {
@@ -72,6 +82,11 @@ public class TestGenericUDFDate extends 
     Text output = (Text) udf.evaluate(args);
 
     assertEquals("to_date() test for DATEWRITABLE failed ", "2009-07-30", output.toString());
+
+    // Try with null args
+    DeferredObject[] nullArgs = { new DeferredJavaObject(null) };
+    output = (Text) udf.evaluate(nullArgs);
+    assertNull("to_date() with null DATE", output);
   }
 
 }

Added: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java?rev=1579501&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java (added)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java Thu Mar 20 01:40:57 2014
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+import junit.framework.TestCase;
+
+public class TestGenericUDFToUnixTimestamp extends TestCase {
+
+  public static void runAndVerify(GenericUDFToUnixTimeStamp udf,
+      Object arg, Object expected) throws HiveException {
+    DeferredObject[] args = { new DeferredJavaObject(arg) };
+    Object result = udf.evaluate(args);
+    if (expected == null) {
+      assertNull(result);
+    } else {
+      assertEquals(expected.toString(), result.toString());
+    }
+  }
+
+  public static void runAndVerify(GenericUDFToUnixTimeStamp udf,
+      Object arg1, Object arg2, Object expected) throws HiveException {
+    DeferredObject[] args = { new DeferredJavaObject(arg1), new DeferredJavaObject(arg2) };
+    Object result = udf.evaluate(args);
+    if (expected == null) {
+      assertNull(result);
+    } else {
+      assertEquals(expected.toString(), result.toString());
+    }
+  }
+
+  public void testTimestamp() throws HiveException {
+    GenericUDFToUnixTimeStamp udf = new GenericUDFToUnixTimeStamp();
+    ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
+    ObjectInspector[] arguments = {valueOI};
+    udf.initialize(arguments);
+
+    Timestamp ts = Timestamp.valueOf("1970-01-01 00:00:00");
+    runAndVerify(udf,
+        new TimestampWritable(ts),
+        new LongWritable(ts.getTime() / 1000));
+
+    ts = Timestamp.valueOf("2001-02-03 01:02:03");
+    runAndVerify(udf,
+        new TimestampWritable(ts),
+        new LongWritable(ts.getTime() / 1000));
+
+    // test null values
+    runAndVerify(udf, null, null);
+  }
+
+  public void testDate() throws HiveException {
+    GenericUDFToUnixTimeStamp udf = new GenericUDFToUnixTimeStamp();
+    ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
+    ObjectInspector[] arguments = {valueOI};
+    udf.initialize(arguments);
+
+    Date date = Date.valueOf("1970-01-01");
+    runAndVerify(udf,
+        new DateWritable(date),
+        new LongWritable(date.getTime() / 1000));
+
+    // test null values
+    runAndVerify(udf, null, null);
+  }
+
+  public void testString() throws HiveException {
+    GenericUDFToUnixTimeStamp udf1 = new GenericUDFToUnixTimeStamp();
+    ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    ObjectInspector[] arguments = {valueOI};
+    udf1.initialize(arguments);
+
+    String val = "2001-01-01 01:02:03";
+    runAndVerify(udf1,
+        new Text(val),
+        new LongWritable(Timestamp.valueOf(val).getTime() / 1000));
+
+    // test null values
+    runAndVerify(udf1, null, null);
+
+    // Try 2-arg version
+    GenericUDFToUnixTimeStamp udf2 = new GenericUDFToUnixTimeStamp();
+    ObjectInspector[] args2 = {valueOI, valueOI};
+    udf2.initialize(args2);
+
+    val = "2001-01-01";
+    String format = "yyyy-MM-dd";
+    runAndVerify(udf2,
+        new Text(val),
+        new Text(format),
+        new LongWritable(Date.valueOf(val).getTime() / 1000));
+
+    // test null values
+    runAndVerify(udf2, null, null, null);
+    runAndVerify(udf2, null, new Text(format), null);
+    runAndVerify(udf2, new Text(val), null, null);
+  }
+}