You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ji...@apache.org on 2014/03/17 19:56:12 UTC

svn commit: r1578512 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/exec/vector/expressions/ java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/ test/queries/clientpositive/ test/results/clientpositive/

Author: jitendra
Date: Mon Mar 17 18:56:11 2014
New Revision: 1578512

URL: http://svn.apache.org/r1578512
Log:
HIVE-6649: Vectorization: some date expressions throw exception. (jitendra, reviewed by Eric Hanson)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
    hive/trunk/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
    hive/trunk/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java Mon Mar 17 18:56:11 2014
@@ -126,11 +126,6 @@ public class ConstantVectorExpression ex
     return outputColumn;
   }
 
-  @Override
-  public String getOutputType() {
-    return getTypeString();
-  }
-
   public long getLongValue() {
     return longValue;
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringUnaryUDF.java Mon Mar 17 18:56:11 2014
@@ -105,7 +105,7 @@ public class StringUnaryUDF extends Vect
       } else {
 
         // Set all elements to not null. The setString call can override this.
-        Arrays.fill(outV.isNull, 0, n - 1, false);
+        Arrays.fill(outV.isNull, 0, n, false);
         for(int i = 0; i != n; i++) {
           s.set(vector[i], start[i], length[i]);
           t = func.evaluate(s);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java Mon Mar 17 18:56:11 2014
@@ -56,6 +56,11 @@ public class VectorUDFDateAddColCol exte
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     ColumnVector inputColVector1 = batch.cols[colNum1];
     LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
     int[] sel = batch.selected;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java Mon Mar 17 18:56:11 2014
@@ -56,6 +56,11 @@ public class VectorUDFDateAddColScalar e
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     BytesColumnVector outV = (BytesColumnVector) batch.cols[outputColumn];
     ColumnVector inputCol = batch.cols[this.colNum];
     /* every line below this is identical for evaluateLong & evaluateString */

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddScalarCol.java Mon Mar 17 18:56:11 2014
@@ -63,6 +63,11 @@ public class VectorUDFDateAddScalarCol e
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     LongColumnVector inputCol = (LongColumnVector) batch.cols[this.colNum];
     /* every line below this is identical for evaluateLong & evaluateString */
     final int n = inputCol.isRepeating ? 1 : batch.size;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java Mon Mar 17 18:56:11 2014
@@ -54,6 +54,11 @@ public class VectorUDFDateDiffColCol ext
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     ColumnVector inputColVector1 = batch.cols[colNum1];
     ColumnVector inputColVector2 = batch.cols[colNum2];
     int[] sel = batch.selected;
@@ -208,12 +213,14 @@ public class VectorUDFDateDiffColCol ext
       output.isNull[0] = input.isNull[0];
       output.isRepeating = true;
 
-      String string = new String(input.vector[0], input.start[0], input.length[0]);
-      try {
-        date.setTime(formatter.parse(string).getTime());
-        output.vector[0] = DateWritable.dateToDays(date);
-      } catch (ParseException e) {
-        output.isNull[0] = true;
+      if (!input.isNull[0]) {
+        String string = new String(input.vector[0], input.start[0], input.length[0]);
+        try {
+          date.setTime(formatter.parse(string).getTime());
+          output.vector[0] = DateWritable.dateToDays(date);
+        } catch (ParseException e) {
+          output.isNull[0] = true;
+        }
       }
       return;
     }
@@ -221,19 +228,18 @@ public class VectorUDFDateDiffColCol ext
     // Handle normal case
 
     // Copy data values over
-    if (selectedInUse) {
-      for (int j = 0; j < size; j++) {
-        int i = sel[j];
-        setDays(input, output, i);
+    if (input.noNulls) {
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          setDays(input, output, i);
+        }
+      } else {
+        for (int i = 0; i < size; i++) {
+          setDays(input, output, i);
+        }
       }
     } else {
-      for (int i = 0; i < size; i++) {
-        setDays(input, output, i);
-      }
-    }
-
-    // Copy nulls over if needed
-    if (!input.noNulls) {
       if (selectedInUse) {
         for (int j = 0; j < size; j++) {
           int i = sel[j];
@@ -243,20 +249,32 @@ public class VectorUDFDateDiffColCol ext
       else {
         System.arraycopy(input.isNull, 0, output.isNull, 0, size);
       }
+
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          if (!input.isNull[i]) {
+           setDays(input, output, i);
+          }
+        }
+      } else {
+        for (int i = 0; i < size; i++) {
+          if (!input.isNull[i]) {
+            setDays(input, output, i);
+          }
+        }
+      }
     }
   }
 
   private void setDays(BytesColumnVector input, LongColumnVector output, int i) {
-    if (input.isNull[i]) {
-      output.isNull[i] = true;
-      return;
-    }
     String string = new String(input.vector[i], input.start[i], input.length[i]);
     try {
       date.setTime(formatter.parse(string).getTime());
       output.vector[i] = DateWritable.dateToDays(date);
     } catch (ParseException e) {
       output.isNull[i] = true;
+      output.noNulls = false;
     }
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java Mon Mar 17 18:56:11 2014
@@ -60,6 +60,11 @@ public class VectorUDFDateDiffColScalar 
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     ColumnVector inputCol = batch.cols[this.colNum];
     /* every line below this is identical for evaluateLong & evaluateString */

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java Mon Mar 17 18:56:11 2014
@@ -60,6 +60,11 @@ public class VectorUDFDateDiffScalarCol 
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     ColumnVector inputCol = batch.cols[this.colNum];
     /* every line below this is identical for evaluateLong & evaluateString */

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java Mon Mar 17 18:56:11 2014
@@ -20,16 +20,13 @@ package org.apache.hadoop.hive.ql.exec.v
 
 import org.apache.hadoop.io.Text;
 
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
+import java.sql.Date;
 
 public class VectorUDFDateString extends StringUnaryUDF {
   private static final long serialVersionUID = 1L;
 
   public VectorUDFDateString(int colNum, int outputColumn) {
     super(colNum, outputColumn, new StringUnaryUDF.IUDFUnaryString() {
-      SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd");
       Text t = new Text();
 
       @Override
@@ -38,10 +35,11 @@ public class VectorUDFDateString extends
           return null;
         }
         try {
-          Date date = formatter.parse(s.toString());
-          t.set(formatter.format(date)) ;
+          Date date = Date.valueOf(s.toString());
+          t.set(date.toString());
           return t;
-        } catch (ParseException e) {
+        } catch (IllegalArgumentException e) {
+          e.printStackTrace();
           return null;
         }
       }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldString.java Mon Mar 17 18:56:11 2014
@@ -77,6 +77,11 @@ public abstract class VectorUDFTimestamp
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     BytesColumnVector inputCol = (BytesColumnVector)batch.cols[this.colNum];
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java Mon Mar 17 18:56:11 2014
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
@@ -475,8 +476,7 @@ public class VectorUDAFAvgDecimal extend
       else {
         assert(0 < myagg.count);
         resultCount.set (myagg.count);
-        int bufferIndex = myagg.sum.fastSerializeForHiveDecimal(scratch);
-        resultSum.set(scratch.getBytes(bufferIndex), (int) sumScale);
+        resultSum.set(HiveDecimal.create(myagg.sum.toBigDecimal()));
         return partialResult;
       }
     }

Modified: hive/trunk/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/vectorized_date_funcs.q?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/vectorized_date_funcs.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/vectorized_date_funcs.q Mon Mar 17 18:56:11 2014
@@ -102,3 +102,21 @@ SELECT
   date_sub(fl_time, 2) = date_sub(fl_date, 2),
   datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01")
 FROM date_udf_flight_orc;
+
+EXPLAIN SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10;
+
+SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10;

Modified: hive/trunk/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out?rev=1578512&r1=1578511&r2=1578512&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out Mon Mar 17 18:56:11 2014
@@ -884,3 +884,88 @@ true	true	true	true	true	true	true	true	
 true	true	true	true	true	true	true	true	true	true
 true	true	true	true	true	true	true	true	true	true
 true	true	true	true	true	true	true	true	true	true
+PREHOOK: query: EXPLAIN SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: date_udf_flight_orc.fl_date SIMPLE [(date_udf_flight)date_udf_flight.FieldSchema(name:fl_date, type:date, comment:null), ]
+POSTHOOK: Lineage: date_udf_flight_orc.fl_time EXPRESSION [(date_udf_flight)date_udf_flight.FieldSchema(name:fl_date, type:date, comment:null), ]
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: date_udf_flight_orc
+            Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: fl_date (type: date), to_date(date_add(fl_date, 2)) (type: string), to_date(date_sub(fl_date, 2)) (type: string), datediff(fl_date, date_add(fl_date, 2)) (type: int), datediff(fl_date, date_sub(fl_date, 2)) (type: int), datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) (type: int)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 10
+                Statistics: Num rows: 10 Data size: 960 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 10 Data size: 960 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+
+PREHOOK: query: SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_udf_flight_orc
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_udf_flight_orc
+#### A masked pattern was here ####
+POSTHOOK: Lineage: date_udf_flight_orc.fl_date SIMPLE [(date_udf_flight)date_udf_flight.FieldSchema(name:fl_date, type:date, comment:null), ]
+POSTHOOK: Lineage: date_udf_flight_orc.fl_time EXPRESSION [(date_udf_flight)date_udf_flight.FieldSchema(name:fl_date, type:date, comment:null), ]
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-20	2010-10-22	2010-10-18	-2	2	4
+2010-10-21	2010-10-23	2010-10-19	-2	2	4
+2010-10-21	2010-10-23	2010-10-19	-2	2	4