You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/12/13 20:15:57 UTC

hive git commit: HIVE-15397 : metadata-only queries may return incorrect results with empty tables (Sergey Shelukhin, reviewed by Ashutosh Chauhan, Lefty Leverenz)

Repository: hive
Updated Branches:
  refs/heads/master 8e3c369c1 -> d96da70de


HIVE-15397 : metadata-only queries may return incorrect results with empty tables (Sergey Shelukhin, reviewed by Ashutosh Chauhan, Lefty Leverenz)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d96da70d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d96da70d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d96da70d

Branch: refs/heads/master
Commit: d96da70de84830ea84e4d7c4667f80eea982bdf9
Parents: 8e3c369
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Dec 13 12:08:24 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Dec 13 12:08:24 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  5 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  4 +-
 .../hadoop/hive/ql/io/NullRowsInputFormat.java  |  3 +-
 .../hive/ql/io/OneNullRowInputFormat.java       |  2 +-
 .../hadoop/hive/ql/io/ZeroRowsInputFormat.java  | 54 +++++++++++++
 .../physical/NullScanTaskDispatcher.java        | 30 ++++++--
 .../limit_partition_metadataonly.q              |  1 +
 .../clientpositive/metadata_empty_table.q       | 22 ++++++
 .../test/queries/clientpositive/metadataonly1.q |  1 +
 ql/src/test/queries/clientpositive/partInit.q   |  5 ++
 .../queries/clientpositive/partition_date2.q    |  2 +
 .../clientpositive/partition_timestamp2.q       |  1 +
 .../clientpositive/metadata_empty_table.q.out   | 81 ++++++++++++++++++++
 .../test/results/clientpositive/partInit.q.out  | 13 +++-
 .../clientpositive/partition_date2.q.out        |  2 -
 .../clientpositive/partition_timestamp2.q.out   |  5 --
 16 files changed, 210 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 9064e49..3fd07e2 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1560,7 +1560,10 @@ public class HiveConf extends Configuration {
     // Constant propagation optimizer
     HIVEOPTCONSTANTPROPAGATION("hive.optimize.constant.propagation", true, "Whether to enable constant propagation optimizer"),
     HIVEIDENTITYPROJECTREMOVER("hive.optimize.remove.identity.project", true, "Removes identity project from operator tree"),
-    HIVEMETADATAONLYQUERIES("hive.optimize.metadataonly", true, ""),
+    HIVEMETADATAONLYQUERIES("hive.optimize.metadataonly", false,
+        "Whether to eliminate scans of the tables from which no columns are selected. Note\n" +
+        "that, when selecting from empty tables with data files, this can produce incorrect\n" +
+        "results, so it's disabled by default. It works correctly for normal tables."),
     HIVENULLSCANOPTIMIZE("hive.optimize.null.scan", true, "Dont scan relations which are guaranteed to not generate any rows"),
     HIVEOPTPPD_STORAGE("hive.optimize.ppd.storage", true,
         "Whether to push predicates down to storage handlers"),

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 98c6c76..8db833e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -2289,7 +2289,7 @@ public final class Utilities {
     return isEmptyPath(job, dirPath);
   }
 
-  public static boolean isEmptyPath(JobConf job, Path dirPath) throws Exception {
+  public static boolean isEmptyPath(Configuration job, Path dirPath) throws IOException {
     FileSystem inpFs = dirPath.getFileSystem(job);
     try {
       FileStatus[] fStats = inpFs.listStatus(dirPath, FileUtils.HIDDEN_FILES_PATH_FILTER);
@@ -3073,7 +3073,7 @@ public final class Utilities {
         props, oneRow);
 
     if (LOG.isInfoEnabled()) {
-      LOG.info("Changed input file " + strPath + " to empty file " + newPath);
+      LOG.info("Changed input file " + strPath + " to empty file " + newPath + " (" + oneRow + ")");
     }
 
     // update the work

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/java/org/apache/hadoop/hive/ql/io/NullRowsInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/NullRowsInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/NullRowsInputFormat.java
index bca5096..f6eaed1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/NullRowsInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/NullRowsInputFormat.java
@@ -73,7 +73,8 @@ public class NullRowsInputFormat implements InputFormat<NullWritable, NullWritab
     public NullRowsRecordReader(Configuration conf, InputSplit split) throws IOException {
       boolean isVectorMode = Utilities.getUseVectorizedInputFileFormat(conf);
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Null record reader in " + (isVectorMode ? "" : "non-") + "vector mode");
+        LOG.debug(getClass().getSimpleName() + " in "
+            + (isVectorMode ? "" : "non-") + "vector mode");
       }
       if (isVectorMode) {
         rbCtx = Utilities.getVectorizedRowBatchCtx(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java
index 01f48e4..4957edf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java
@@ -48,7 +48,7 @@ public class OneNullRowInputFormat extends NullRowsInputFormat
       super(conf, split);
     }
 
-    private boolean processed;
+    protected boolean processed;
 
     @Override
     public long getPos() throws IOException {

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/java/org/apache/hadoop/hive/ql/io/ZeroRowsInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/ZeroRowsInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/ZeroRowsInputFormat.java
new file mode 100644
index 0000000..84eb7af
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/ZeroRowsInputFormat.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import org.apache.hadoop.hive.ql.io.OneNullRowInputFormat.OneNullRowRecordReader;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Same as OneNullRowInputFormat, but with 0 rows. There's no way to store smth like OperatorDesc
+ * in InputFormat, so this is how it is. We could perhaps encode the number of null rows in the
+ * null path. However, NullIF can be used without using NullFS, so that would not be possible.
+ */
+public class ZeroRowsInputFormat extends NullRowsInputFormat
+  implements VectorizedInputFormatInterface {
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public RecordReader<NullWritable, NullWritable> getRecordReader(InputSplit split,
+      JobConf conf, Reporter arg2) throws IOException {
+    return new ZeroRowsRecordReader(conf, split);
+  }
+
+  public static class ZeroRowsRecordReader extends OneNullRowRecordReader {
+    public ZeroRowsRecordReader(Configuration conf, InputSplit split) throws IOException {
+      super(conf, split);
+      processed = true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
index 90cd41d..0882ae2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
+import org.apache.hadoop.mapred.InputFormat;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.ql.exec.Utilities;
+
+import org.apache.hadoop.hive.ql.io.ZeroRowsInputFormat;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -86,13 +94,19 @@ public class NullScanTaskDispatcher implements Dispatcher {
     return null;
   }
 
-  private PartitionDesc changePartitionToMetadataOnly(PartitionDesc desc) {
-    if (desc != null) {
-      desc.setInputFileFormatClass(OneNullRowInputFormat.class);
-      desc.setOutputFileFormatClass(HiveIgnoreKeyTextOutputFormat.class);
-      desc.getProperties().setProperty(serdeConstants.SERIALIZATION_LIB,
-        NullStructSerDe.class.getName());
+  private PartitionDesc changePartitionToMetadataOnly(PartitionDesc desc, Path path) {
+    if (desc == null) return null;
+    boolean isEmpty = false;
+    try {
+      isEmpty = Utilities.isEmptyPath(physicalContext.getConf(), path);
+    } catch (IOException e) {
+      LOG.error("Cannot determine if the table is empty", e);
     }
+    desc.setInputFileFormatClass(
+        isEmpty ? ZeroRowsInputFormat.class : OneNullRowInputFormat.class);
+    desc.setOutputFileFormatClass(HiveIgnoreKeyTextOutputFormat.class);
+    desc.getProperties().setProperty(serdeConstants.SERIALIZATION_LIB,
+      NullStructSerDe.class.getName());
     return desc;
   }
 
@@ -107,7 +121,7 @@ public class NullScanTaskDispatcher implements Dispatcher {
     }
     if (allowed.size() > 0) {
       PartitionDesc partDesc = work.getPathToPartitionInfo().get(path).clone();
-      PartitionDesc newPartition = changePartitionToMetadataOnly(partDesc);
+      PartitionDesc newPartition = changePartitionToMetadataOnly(partDesc, path);
       // Prefix partition with something to avoid it being a hidden file.
       Path fakePath = new Path(NullScanFileSystem.getBase() + newPartition.getTableName()
           + "/part" + encode(newPartition.getPartSpec()));
@@ -126,7 +140,7 @@ public class NullScanTaskDispatcher implements Dispatcher {
     for (TableScanOperator tso : tableScans) {
       // use LinkedHashMap<String, Operator<? extends OperatorDesc>>
       // getAliasToWork()
-	  // should not apply this for non-native table
+      // should not apply this for non-native table
       if (tso.getConf().getTableMetadata().getStorageHandler() != null) {
         continue;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q b/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q
index 42c5feb..3e0a749 100644
--- a/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q
+++ b/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q
@@ -1,5 +1,6 @@
 set hive.mapred.mode=nonstrict;
 set hive.limit.query.max.table.partition=1;
+set hive.optimize.metadataonly=true;
 
 -- SORT_QUERY_RESULTS
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/queries/clientpositive/metadata_empty_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/metadata_empty_table.q b/ql/src/test/queries/clientpositive/metadata_empty_table.q
new file mode 100644
index 0000000..73e23dd
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/metadata_empty_table.q
@@ -0,0 +1,22 @@
+set hive.compute.query.using.stats=false;
+set hive.fetch.task.conversion=none;
+
+drop table nullscan_table;
+drop table onerow_table;
+create table nullscan_table(i int); 
+create table onerow_table(i int); 
+insert into table onerow_table values(0);
+
+set hive.optimize.metadataonly=true;
+select (1=1) from nullscan_table group by (1=1); 
+select (1=1) from onerow_table group by (1=1); 
+
+
+set hive.optimize.metadataonly=false;
+select (1=1) from nullscan_table group by (1=1); 
+select (1=1) from onerow_table group by (1=1); 
+
+
+drop table nullscan_table;
+drop table onerow_table;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/queries/clientpositive/metadataonly1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/metadataonly1.q b/ql/src/test/queries/clientpositive/metadataonly1.q
index 5420ea9..409109a 100644
--- a/ql/src/test/queries/clientpositive/metadataonly1.q
+++ b/ql/src/test/queries/clientpositive/metadataonly1.q
@@ -1,4 +1,5 @@
 set hive.mapred.mode=nonstrict;
+set hive.optimize.metadataonly=true;
 CREATE TABLE TEST1(A INT, B DOUBLE) partitioned by (ds string);
 explain extended select max(ds) from TEST1;
 select max(ds) from TEST1;

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/queries/clientpositive/partInit.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/partInit.q b/ql/src/test/queries/clientpositive/partInit.q
index 2b2b342..62299b2 100644
--- a/ql/src/test/queries/clientpositive/partInit.q
+++ b/ql/src/test/queries/clientpositive/partInit.q
@@ -4,5 +4,10 @@ SELECT MAX(c) FROM empty;
 SELECT MAX(p) FROM empty;
 
 ALTER TABLE empty ADD PARTITION (p=1);
+
+set hive.optimize.metadataonly=true;
+SELECT MAX(p) FROM empty;
+
+set hive.optimize.metadataonly=false;
 SELECT MAX(p) FROM empty;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/queries/clientpositive/partition_date2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/partition_date2.q b/ql/src/test/queries/clientpositive/partition_date2.q
index 7883db6..cb8e5db 100644
--- a/ql/src/test/queries/clientpositive/partition_date2.q
+++ b/ql/src/test/queries/clientpositive/partition_date2.q
@@ -1,4 +1,6 @@
 set hive.mapred.mode=nonstrict;
+set hive.optimize.metadataonly=true;
+
 -- SORT_QUERY_RESULTS
 
 drop table partition_date2_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/queries/clientpositive/partition_timestamp2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/partition_timestamp2.q b/ql/src/test/queries/clientpositive/partition_timestamp2.q
index 236b88e..8460a9f 100644
--- a/ql/src/test/queries/clientpositive/partition_timestamp2.q
+++ b/ql/src/test/queries/clientpositive/partition_timestamp2.q
@@ -1,4 +1,5 @@
 set hive.mapred.mode=nonstrict;
+set hive.optimize.metadataonly=true;
 -- Exclude test on Windows due to space character being escaped in Hive paths on Windows.
 -- EXCLUDE_OS_WINDOWS
 drop table partition_timestamp2_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/results/clientpositive/metadata_empty_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/metadata_empty_table.q.out b/ql/src/test/results/clientpositive/metadata_empty_table.q.out
new file mode 100644
index 0000000..b51eaec
--- /dev/null
+++ b/ql/src/test/results/clientpositive/metadata_empty_table.q.out
@@ -0,0 +1,81 @@
+PREHOOK: query: drop table nullscan_table
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table nullscan_table
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table onerow_table
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table onerow_table
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table nullscan_table(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nullscan_table
+POSTHOOK: query: create table nullscan_table(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nullscan_table
+PREHOOK: query: create table onerow_table(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@onerow_table
+POSTHOOK: query: create table onerow_table(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@onerow_table
+PREHOOK: query: insert into table onerow_table values(0)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@onerow_table
+POSTHOOK: query: insert into table onerow_table values(0)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@onerow_table
+POSTHOOK: Lineage: onerow_table.i EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: select (1=1) from nullscan_table group by (1=1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@nullscan_table
+#### A masked pattern was here ####
+POSTHOOK: query: select (1=1) from nullscan_table group by (1=1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@nullscan_table
+#### A masked pattern was here ####
+PREHOOK: query: select (1=1) from onerow_table group by (1=1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@onerow_table
+#### A masked pattern was here ####
+POSTHOOK: query: select (1=1) from onerow_table group by (1=1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@onerow_table
+#### A masked pattern was here ####
+true
+PREHOOK: query: select (1=1) from nullscan_table group by (1=1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@nullscan_table
+#### A masked pattern was here ####
+POSTHOOK: query: select (1=1) from nullscan_table group by (1=1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@nullscan_table
+#### A masked pattern was here ####
+PREHOOK: query: select (1=1) from onerow_table group by (1=1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@onerow_table
+#### A masked pattern was here ####
+POSTHOOK: query: select (1=1) from onerow_table group by (1=1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@onerow_table
+#### A masked pattern was here ####
+true
+PREHOOK: query: drop table nullscan_table
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@nullscan_table
+PREHOOK: Output: default@nullscan_table
+POSTHOOK: query: drop table nullscan_table
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@nullscan_table
+POSTHOOK: Output: default@nullscan_table
+PREHOOK: query: drop table onerow_table
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@onerow_table
+PREHOOK: Output: default@onerow_table
+POSTHOOK: query: drop table onerow_table
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@onerow_table
+POSTHOOK: Output: default@onerow_table

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/results/clientpositive/partInit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/partInit.q.out b/ql/src/test/results/clientpositive/partInit.q.out
index da125cf..ab3830b 100644
--- a/ql/src/test/results/clientpositive/partInit.q.out
+++ b/ql/src/test/results/clientpositive/partInit.q.out
@@ -41,4 +41,15 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@empty
 POSTHOOK: Input: default@empty@p=1
 #### A masked pattern was here ####
-1
+NULL
+PREHOOK: query: SELECT MAX(p) FROM empty
+PREHOOK: type: QUERY
+PREHOOK: Input: default@empty
+PREHOOK: Input: default@empty@p=1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT MAX(p) FROM empty
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@empty
+POSTHOOK: Input: default@empty@p=1
+#### A masked pattern was here ####
+NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/results/clientpositive/partition_date2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/partition_date2.q.out b/ql/src/test/results/clientpositive/partition_date2.q.out
index 3bd1f49..f4fdb47 100644
--- a/ql/src/test/results/clientpositive/partition_date2.q.out
+++ b/ql/src/test/results/clientpositive/partition_date2.q.out
@@ -170,7 +170,6 @@ POSTHOOK: Input: default@partition_date2_1@dt=1999-01-01/region=2
 POSTHOOK: Input: default@partition_date2_1@dt=2000-01-01/region=1
 POSTHOOK: Input: default@partition_date2_1@dt=2000-01-01/region=2
 #### A masked pattern was here ####
-1980-01-02
 1999-01-01
 2000-01-01
 PREHOOK: query: select * from partition_date2_1
@@ -215,7 +214,6 @@ POSTHOOK: Input: default@partition_date2_1@dt=1980-01-02/region=3
 POSTHOOK: Input: default@partition_date2_1@dt=2000-01-01/region=1
 POSTHOOK: Input: default@partition_date2_1@dt=2000-01-01/region=2
 #### A masked pattern was here ####
-1980-01-02
 2000-01-01
 PREHOOK: query: select * from partition_date2_1
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/d96da70d/ql/src/test/results/clientpositive/partition_timestamp2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/partition_timestamp2.q.out b/ql/src/test/results/clientpositive/partition_timestamp2.q.out
index 4222f2e..f32538f 100644
--- a/ql/src/test/results/clientpositive/partition_timestamp2.q.out
+++ b/ql/src/test/results/clientpositive/partition_timestamp2.q.out
@@ -147,7 +147,6 @@ POSTHOOK: Input: default@partition_timestamp2_1@dt=2000-01-01 01%3A00%3A00.0/reg
 #### A masked pattern was here ####
 1999-01-01 00:00:00
 1999-01-01 01:00:00
-2000-01-01 00:00:00
 2000-01-01 01:00:00
 PREHOOK: query: select * from partition_timestamp2_1
 PREHOOK: type: QUERY
@@ -195,10 +194,8 @@ POSTHOOK: Input: default@partition_timestamp2_1@dt=1999-01-01 01%3A00%3A00.0/reg
 POSTHOOK: Input: default@partition_timestamp2_1@dt=2000-01-01 00%3A00%3A00.0/region=1
 POSTHOOK: Input: default@partition_timestamp2_1@dt=2000-01-01 01%3A00%3A00.0/region=1
 #### A masked pattern was here ####
-1980-01-02 00:00:00
 1999-01-01 00:00:00
 1999-01-01 01:00:00
-2000-01-01 00:00:00
 2000-01-01 01:00:00
 PREHOOK: query: select * from partition_timestamp2_1
 PREHOOK: type: QUERY
@@ -247,9 +244,7 @@ POSTHOOK: Input: default@partition_timestamp2_1@dt=1999-01-01 00%3A00%3A00.0/reg
 POSTHOOK: Input: default@partition_timestamp2_1@dt=2000-01-01 00%3A00%3A00.0/region=1
 POSTHOOK: Input: default@partition_timestamp2_1@dt=2000-01-01 01%3A00%3A00.0/region=1
 #### A masked pattern was here ####
-1980-01-02 00:00:00
 1999-01-01 00:00:00
-2000-01-01 00:00:00
 2000-01-01 01:00:00
 PREHOOK: query: select * from partition_timestamp2_1
 PREHOOK: type: QUERY