You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2018/01/22 08:01:20 UTC

[17/17] hive git commit: HIVE-18061: q.outs: be more selective with masking hdfs paths (Laszlo Bodor via Zoltan Haindrich)

HIVE-18061: q.outs: be more selective with masking hdfs paths (Laszlo Bodor via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/master
Commit: 425271896a834b3eb695a0cfd23334d08565bc9a
Parents: a59cb88
Author: Laszlo Bodor <bo...@gmail.com>
Authored: Mon Jan 22 08:44:56 2018 +0100
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon Jan 22 08:59:05 2018 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   31 +-
 .../apache/hadoop/hive/ql/TestQTestUtil.java    |   88 ++
 .../alter_table_wrong_location.q.out            |    2 +-
 .../cluster_tasklog_retrieval.q.out             |    1 +
 .../results/clientnegative/ct_noperm_loc.q.out  |   10 +-
 .../clientnegative/ctas_noperm_loc.q.out        |    2 +-
 .../file_with_header_footer_negative.q.out      |    8 +-
 .../local_mapred_error_cache.q.out              |    1 +
 .../clientnegative/mapreduce_stack_trace.q.out  |    1 +
 .../mapreduce_stack_trace_turnoff.q.out         |    1 +
 .../clientnegative/minimr_broken_pipe.q.out     |    1 +
 .../table_nonprintable_negative.q.out           |   10 +-
 .../clientnegative/udf_local_resource.q.out     |    2 +-
 .../bucket_num_reducers_acid.q.out              |    2 +-
 .../bucket_num_reducers_acid2.q.out             |    2 +-
 .../clientpositive/database_properties.q.out    |    8 +-
 .../encryption_auto_purge_tables.q.out          |    8 +-
 .../encrypted/encryption_ctas.q.out             |    4 +-
 .../encrypted/encryption_drop_partition.q.out   |   12 +-
 .../encrypted/encryption_drop_table.q.out       |   12 +-
 .../encryption_drop_table_in_encrypted_db.q.out |    7 +-
 .../encrypted/encryption_drop_view.q.out        |    4 +-
 .../encryption_insert_partition_dynamic.q.out   |    8 +-
 .../encryption_insert_partition_static.q.out    |    8 +-
 .../encrypted/encryption_insert_values.q.out    |    4 +-
 .../encryption_join_unencrypted_tbl.q.out       |   19 +-
 ...on_join_with_different_encryption_keys.q.out |   27 +-
 ...cryption_load_data_to_encrypted_tables.q.out |    8 +-
 .../encrypted/encryption_move_tbl.q.out         |   14 +-
 ...ryption_select_read_only_encrypted_tbl.q.out |    4 +-
 ...ption_select_read_only_unencrypted_tbl.q.out |    8 +-
 ...on_unencrypted_nonhdfs_external_tables.q.out |    4 +-
 .../results/clientpositive/index_bitmap3.q.out  | 1016 ++++++++++++++++-
 .../clientpositive/index_bitmap_auto.q.out      | 1020 +++++++++++++++++-
 .../infer_bucket_sort_dyn_part.q.out            |    4 +-
 .../infer_bucket_sort_map_operators.q.out       |    8 +-
 .../llap/acid_bucket_pruning.q.out              |   18 +-
 .../results/clientpositive/llap/bucket5.q.out   |   74 +-
 .../results/clientpositive/llap/bucket6.q.out   |    4 +-
 .../results/clientpositive/llap/cte_2.q.out     |   24 +-
 .../results/clientpositive/llap/cte_4.q.out     |   38 +-
 .../llap/dynamic_partition_pruning_2.q.out      |   36 +-
 .../llap/dynamic_semijoin_user_level.q.out      |   86 +-
 .../llap/empty_dir_in_table.q.out               |   20 +-
 .../clientpositive/llap/except_distinct.q.out   |   52 +-
 .../clientpositive/llap/explainuser_2.q.out     |   12 +-
 ...rnal_table_with_space_in_location_path.q.out |   24 +-
 .../llap/file_with_header_footer.q.out          |   56 +-
 .../clientpositive/llap/global_limit.q.out      |   84 +-
 .../llap/import_exported_table.q.out            |    8 +-
 .../clientpositive/llap/insert_into1.q.out      |   32 +-
 .../clientpositive/llap/insert_into2.q.out      |   32 +-
 .../clientpositive/llap/intersect_all.q.out     |   44 +-
 .../llap/intersect_distinct.q.out               |   44 +-
 .../clientpositive/llap/llap_nullscan.q.out     |   22 +-
 .../results/clientpositive/llap/llap_smb.q.out  |    8 +-
 .../clientpositive/llap/llap_stats.q.out        |    8 +-
 .../clientpositive/llap/llapdecider.q.out       |    4 +-
 .../results/clientpositive/llap/load_fs2.q.out  |   15 +-
 .../load_hdfs_file_with_space_in_the_name.q.out |   16 +-
 .../clientpositive/llap/mapreduce1.q.out        |    4 +-
 .../clientpositive/llap/mapreduce2.q.out        |    4 +-
 .../results/clientpositive/llap/mm_all.q.out    |  128 +--
 .../llap/multi_count_distinct_null.q.out        |   36 +-
 .../clientpositive/llap/orc_llap_counters.q.out |   84 +-
 .../llap/orc_llap_counters1.q.out               |    4 +-
 .../clientpositive/llap/orc_merge1.q.out        |   43 +-
 .../clientpositive/llap/orc_merge10.q.out       |   58 +-
 .../clientpositive/llap/orc_merge2.q.out        |   10 +-
 .../clientpositive/llap/orc_merge3.q.out        |   10 +-
 .../clientpositive/llap/orc_merge4.q.out        |   14 +-
 .../clientpositive/llap/orc_merge_diff_fs.q.out |   24 +-
 .../clientpositive/llap/orc_ppd_basic.q.out     |   92 +-
 .../llap/orc_ppd_schema_evol_3a.q.out           |  100 +-
 .../clientpositive/llap/parallel_colstats.q.out |   16 +-
 .../parquet_complex_types_vectorization.q.out   |   36 +-
 .../llap/parquet_map_type_vectorization.q.out   |   36 +-
 .../llap/parquet_types_vectorization.q.out      |   32 +-
 .../clientpositive/llap/rcfile_createas1.q.out  |   10 +-
 .../clientpositive/llap/rcfile_merge2.q.out     |    8 +-
 .../clientpositive/llap/rcfile_merge3.q.out     |    8 +-
 .../clientpositive/llap/rcfile_merge4.q.out     |    8 +-
 .../llap/reduce_deduplicate.q.out               |   42 +-
 .../llap/reduce_deduplicate_distinct.q.out      |   20 +-
 .../clientpositive/llap/remote_script.q.out     |    4 +-
 .../clientpositive/llap/schemeAuthority.q.out   |   24 +-
 .../clientpositive/llap/schemeAuthority2.q.out  |   12 +-
 .../llap/table_nonprintable.q.out               |   22 +-
 .../llap/temp_table_external.q.out              |   14 +-
 .../llap/tez_union_dynamic_partition.q.out      |    8 +-
 .../llap/tez_union_dynamic_partition_2.q.out    |   12 +-
 .../clientpositive/llap/unionDistinct_1.q.out   |  266 +++--
 .../clientpositive/parallel_orderby.q.out       |   10 +-
 .../clientpositive/perf/tez/query39.q.out       |    4 +-
 .../root_dir_external_table.q.out               |   14 +-
 .../results/clientpositive/scriptfile1.q.out    |    4 +-
 .../spark/auto_sortmerge_join_16.q.out          |    4 +-
 .../spark/auto_sortmerge_join_16.q.out_spark    |  252 +++++
 .../results/clientpositive/spark/bucket4.q.out  |   20 +-
 .../clientpositive/spark/bucket4.q.out_spark    |  465 ++++++++
 .../results/clientpositive/spark/bucket5.q.out  |   48 +-
 .../results/clientpositive/spark/bucket6.q.out  |    4 +-
 .../spark/bucketizedhiveinputformat.q.out       |   16 +-
 .../clientpositive/spark/bucketmapjoin6.q.out   |    4 +-
 .../clientpositive/spark/bucketmapjoin7.q.out   |   30 +-
 .../spark/bucketmapjoin7.q.out_spark            |  315 ++++++
 .../spark/constprog_semijoin.q.out              |   32 +-
 .../spark/disable_merge_for_bucketing.q.out     |   20 +-
 .../disable_merge_for_bucketing.q.out_spark     |  493 +++++++++
 .../spark/dynamic_rdd_cache.q.out               |   12 +-
 .../spark/empty_dir_in_table.q.out              |   20 +-
 ...rnal_table_with_space_in_location_path.q.out |   24 +-
 .../spark/file_with_header_footer.q.out         |   56 +-
 .../spark/gen_udf_example_add10.q.out           |    4 +-
 .../spark/import_exported_table.q.out           |    8 +-
 .../clientpositive/spark/index_bitmap3.q.out    | 1016 ++++++++++++++++-
 .../spark/index_bitmap_auto.q.out               | 1020 +++++++++++++++++-
 .../infer_bucket_sort_bucketed_table.q.out      |   16 +-
 .../clientpositive/spark/input16_cc.q.out       |    4 +-
 .../spark/insert_overwrite_directory2.q.out     |   12 +-
 .../clientpositive/spark/leftsemijoin_mr.q.out  |   16 +-
 .../spark/leftsemijoin_mr.q.out_spark           |   94 ++
 .../spark/list_bucket_dml_10.q.out              |   18 +-
 .../results/clientpositive/spark/load_fs2.q.out |   15 +-
 .../load_hdfs_file_with_space_in_the_name.q.out |   16 +-
 .../clientpositive/spark/orc_merge1.q.out       |   39 +-
 .../clientpositive/spark/orc_merge2.q.out       |   14 +-
 .../clientpositive/spark/orc_merge3.q.out       |   14 +-
 .../clientpositive/spark/orc_merge4.q.out       |   18 +-
 .../clientpositive/spark/orc_merge5.q.out       |   32 +-
 .../clientpositive/spark/orc_merge6.q.out       |   44 +-
 .../clientpositive/spark/orc_merge7.q.out       |   38 +-
 .../clientpositive/spark/orc_merge8.q.out       |    7 +-
 .../clientpositive/spark/orc_merge9.q.out       |   42 +-
 .../spark/orc_merge_diff_fs.q.out               |   24 +-
 .../spark/orc_merge_incompat1.q.out             |   20 +-
 .../spark/orc_merge_incompat2.q.out             |   26 +-
 .../clientpositive/spark/parallel_orderby.q.out |   10 +-
 .../clientpositive/spark/quotedid_smb.q.out     |    4 +-
 .../spark/reduce_deduplicate.q.out              |   34 +-
 .../clientpositive/spark/remote_script.q.out    |    4 +-
 .../spark/root_dir_external_table.q.out         |   14 +-
 .../clientpositive/spark/schemeAuthority.q.out  |   24 +-
 .../clientpositive/spark/schemeAuthority2.q.out |   12 +-
 .../clientpositive/spark/scriptfile1.q.out      |    4 +-
 .../spark/scriptfile1.q.out_spark               |   47 +
 .../spark/spark_combine_equivalent_work.q.out   |    8 +-
 .../spark/spark_dynamic_partition_pruning.q.out |  210 ++--
 .../spark_dynamic_partition_pruning_2.q.out     |   36 +-
 .../spark_dynamic_partition_pruning_3.q.out     |   32 +-
 .../spark_dynamic_partition_pruning_4.q.out     |   40 +-
 ...ic_partition_pruning_recursive_mapjoin.q.out |   16 +-
 .../spark/spark_explainuser_1.q.out             |    6 +-
 .../spark_multi_insert_parallel_orderby.q.out   |   28 +-
 .../spark/spark_opt_shuffle_serde.q.out         |    8 +-
 .../spark/spark_use_op_stats.q.out              |    8 +-
 .../spark/spark_use_ts_stats_for_mapjoin.q.out  |   12 +-
 ...k_vectorized_dynamic_partition_pruning.q.out |  202 ++--
 .../spark/temp_table_external.q.out             |   14 +-
 .../spark/truncate_column_buckets.q.out         |    8 +-
 .../clientpositive/spark/uber_reduce.q.out      |    4 +-
 .../spark/vector_inner_join.q.out               |   36 +-
 .../spark/vector_outer_join0.q.out              |   16 +-
 .../spark/vector_outer_join1.q.out              |   36 +-
 .../spark/vector_outer_join2.q.out              |   28 +-
 .../spark/vector_outer_join3.q.out              |   36 +-
 .../spark/vector_outer_join4.q.out              |   36 +-
 .../spark/vector_outer_join5.q.out              |   56 +-
 .../tez/acid_vectorization_original_tez.q.out   |   54 +-
 .../clientpositive/tez/delete_orig_table.q.out  |   12 +-
 .../clientpositive/tez/explainanalyze_1.q.out   |   60 +-
 .../clientpositive/tez/explainanalyze_3.q.out   |   29 +-
 .../clientpositive/tez/explainanalyze_4.q.out   |   48 +-
 .../clientpositive/tez/explainanalyze_5.q.out   |   36 +-
 .../clientpositive/tez/explainuser_3.q.out      |   11 +-
 .../tez/hybridgrace_hashjoin_1.q.out            |   52 +-
 .../tez/hybridgrace_hashjoin_2.q.out            |   52 +-
 .../tez/multi_count_distinct.q.out              |   28 +-
 .../clientpositive/tez/orc_merge12.q.out        |   20 +-
 .../tez/orc_vectorization_ppd.q.out             |   32 +-
 .../results/clientpositive/tez/tez-tag.q.out    |    8 +-
 .../clientpositive/tez/tez_union_with_udf.q.out |    8 +-
 .../clientpositive/tez/update_orig_table.q.out  |    8 +-
 .../tez/vector_join_part_col_char.q.out         |    8 +-
 .../tez/vector_non_string_partition.q.out       |    8 +-
 .../clientpositive/tez/vectorization_div0.q.out |   16 +-
 .../tez/vectorization_limit.q.out               |   28 +-
 .../test/results/clientpositive/udf_using.q.out |   11 +-
 188 files changed, 8128 insertions(+), 2153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index c4437f2..3f377f9 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -70,16 +70,13 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Stream;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileStatus;
@@ -138,6 +135,8 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 
 import junit.framework.TestSuite;
@@ -163,9 +162,13 @@ public class QTestUtil {
   private static final String TEST_TMP_DIR_PROPERTY = "test.tmp.dir"; // typically target/tmp
   private static final String BUILD_DIR_PROPERTY = "build.dir"; // typically target
 
+  public static final String PATH_HDFS_REGEX = "(hdfs://)([a-zA-Z0-9:/_\\-\\.=])+";
+  public static final String PATH_HDFS_WITH_DATE_USER_GROUP_REGEX = "([a-z]+) ([a-z]+)([ ]+)([0-9]+) ([0-9]{4}-[0-9]{2}-[0-9]{2} [0-9]{2}:[0-9]{2}) " + PATH_HDFS_REGEX;
+
   private String testWarehouse;
   private final String testFiles;
   protected final String outDir;
+  protected String overrideResultsDir;
   protected final String logDir;
   private final TreeMap<String, String> qMap;
   private final Set<String> qSkipSet;
@@ -535,7 +538,7 @@ public class QTestUtil {
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
-      String confDir, String hadzoopVer, String initScript, String cleanupScript,
+      String confDir, String hadoopVer, String initScript, String cleanupScript,
       boolean withLlapIo, FsType fsType)
     throws Exception {
     LOG.info("Setting up QTestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +
@@ -1541,6 +1544,7 @@ public class QTestUtil {
     String ret = (new File(outDir, testName)).getPath();
     // List of configurations. Currently the list consists of hadoop version and execution mode only
     List<String> configs = new ArrayList<String>();
+    configs.add(this.clusterType.toString());
     configs.add(this.hadoopVer);
 
     Deque<String> stack = new LinkedList<String>();
@@ -1548,7 +1552,7 @@ public class QTestUtil {
     sb.append(testName);
     stack.push(sb.toString());
 
-    // example file names are input1.q.out_0.20.0_minimr or input2.q.out_0.17
+    // example file names are input1.q.out_mr_0.17 or input2.q.out_0.17
     for (String s: configs) {
       sb.append('_');
       sb.append(s);
@@ -1653,7 +1657,6 @@ public class QTestUtil {
   private final Pattern[] planMask = toPattern(new String[] {
       ".*file:.*",
       ".*pfile:.*",
-      ".*hdfs:.*",
       ".*/tmp/.*",
       ".*invalidscheme:.*",
       ".*lastUpdateTime.*",
@@ -1725,9 +1728,16 @@ public class QTestUtil {
     partialPlanMask = ppm.toArray(new PatternReplacementPair[ppm.size()]);
   }
   /* This list may be modified by specific cli drivers to mask strings that change on every test */
-  private final List<Pair<Pattern, String>> patternsWithMaskComments = new ArrayList<Pair<Pattern, String>>() {{
-    add(toPatternPair("(pblob|s3.?|swift|wasb.?).*hive-staging.*","### BLOBSTORE_STAGING_PATH ###"));
-  }};
+  private final List<Pair<Pattern, String>> patternsWithMaskComments =
+      new ArrayList<Pair<Pattern, String>>() {
+        {
+          add(toPatternPair("(pblob|s3.?|swift|wasb.?).*hive-staging.*",
+              "### BLOBSTORE_STAGING_PATH ###"));
+          add(toPatternPair(PATH_HDFS_WITH_DATE_USER_GROUP_REGEX,
+              "### USER ### ### GROUP ###$3$4 ### HDFS DATE ### $6### HDFS PATH ###"));
+          add(toPatternPair(PATH_HDFS_REGEX, "$1### HDFS PATH ###"));
+        }
+      };
 
   private Pair<Pattern, String> toPatternPair(String patternStr, String maskComment) {
     return ImmutablePair.of(Pattern.compile(patternStr), maskComment);
@@ -2497,5 +2507,4 @@ public class QTestUtil {
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/itests/util/src/test/java/org/apache/hadoop/hive/ql/TestQTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/test/java/org/apache/hadoop/hive/ql/TestQTestUtil.java b/itests/util/src/test/java/org/apache/hadoop/hive/ql/TestQTestUtil.java
new file mode 100644
index 0000000..c01d87b
--- /dev/null
+++ b/itests/util/src/test/java/org/apache/hadoop/hive/ql/TestQTestUtil.java
@@ -0,0 +1,88 @@
+/**
+ * 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;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class contains unit tests for QTestUtil
+ */
+public class TestQTestUtil {
+  private static final String TEST_HDFS_MASK = "###HDFS###";
+  private static final String TEST_HDFS_DATE_MASK = "###HDFS_DATE###";
+  private static final String TEST_HDFS_USER_MASK = "###USER###";
+  private static final String TEST_HDFS_GROUP_MASK = "###GROUP###";
+
+  @Test
+  public void testSelectiveHdfsPatternMaskOnlyHdfsPath() {
+    Assert.assertEquals("nothing to be masked", maskHdfs("nothing to be masked"));
+    Assert.assertEquals("hdfs://", maskHdfs("hdfs://"));
+    Assert.assertEquals(String.format("hdfs://%s", TEST_HDFS_MASK), maskHdfs("hdfs://a"));
+    Assert.assertEquals(String.format("hdfs://%s other text", TEST_HDFS_MASK),
+        maskHdfs("hdfs://tmp.dfs.com:50029/tmp other text"));
+    Assert.assertEquals(String.format("hdfs://%s", TEST_HDFS_MASK), maskHdfs(
+        "hdfs://localhost:51594/build/ql/test/data/warehouse/default/encrypted_table_dp/p=2014-09-23"));
+
+    String line = maskHdfs("hdfs://localhost:11111/tmp/ct_noperm_loc_foo1");
+    Assert.assertEquals(String.format("hdfs://%s", TEST_HDFS_MASK), line);
+
+    line = maskHdfs("hdfs://one hdfs://two");
+    Assert.assertEquals(String.format("hdfs://%s hdfs://%s", TEST_HDFS_MASK, TEST_HDFS_MASK), line);
+
+    line = maskHdfs(
+        "some text before [name=hdfs://localhost:11111/tmp/ct_noperm_loc_foo1]] some text between hdfs://localhost:22222/tmp/ct_noperm_loc_foo2 some text after");
+    Assert.assertEquals(String.format(
+        "some text before [name=hdfs://%s]] some text between hdfs://%s some text after",
+        TEST_HDFS_MASK, TEST_HDFS_MASK), line);
+
+    line = maskHdfsWithDateUserGroup(
+        "-rw-r--r--   3 hiveptest supergroup       2557 2018-01-11 17:09 hdfs://hello_hdfs_path");
+    Assert.assertEquals(String.format("-rw-r--r--   3 %s %s       2557 %s hdfs://%s",
+        TEST_HDFS_USER_MASK, TEST_HDFS_GROUP_MASK, TEST_HDFS_DATE_MASK, TEST_HDFS_MASK), line);
+
+    line = maskHdfs(maskHdfsWithDateUserGroup(
+        "-rw-r--r--   3 hiveptest supergroup       2557 2018-01-11 17:09 hdfs://hello_hdfs_path"));
+    Assert.assertEquals(String.format("-rw-r--r--   3 %s %s       2557 %s hdfs://%s",
+        TEST_HDFS_USER_MASK, TEST_HDFS_GROUP_MASK, TEST_HDFS_DATE_MASK, TEST_HDFS_MASK), line);
+  }
+
+  private String maskHdfs(String line) {
+    Matcher matcher = Pattern.compile(QTestUtil.PATH_HDFS_REGEX).matcher(line);
+
+    if (matcher.find()) {
+      line = matcher.replaceAll(String.format("$1%s", TEST_HDFS_MASK));
+    }
+
+    return line;
+  }
+
+  private String maskHdfsWithDateUserGroup(String line) {
+    Matcher matcher = Pattern.compile(QTestUtil.PATH_HDFS_WITH_DATE_USER_GROUP_REGEX).matcher(line);
+
+    if (matcher.find()) {
+      line = matcher.replaceAll(String.format("%s %s$3$4 %s $6%s", TEST_HDFS_USER_MASK,
+          TEST_HDFS_GROUP_MASK, TEST_HDFS_DATE_MASK, TEST_HDFS_MASK));
+    }
+
+    return line;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/alter_table_wrong_location.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_wrong_location.q.out b/ql/src/test/results/clientnegative/alter_table_wrong_location.q.out
index d788d55..8808490 100644
--- a/ql/src/test/results/clientnegative/alter_table_wrong_location.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_wrong_location.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create table testwrongloc(id int)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@testwrongloc
-#### A masked pattern was here ####
+FAILED: SemanticException Cannot connect to namenode, please check if host/port pair for hdfs://### HDFS PATH ### is valid

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out b/ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out
index b5ccfed..7db7bfe 100644
--- a/ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out
+++ b/ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out
@@ -8,5 +8,6 @@ PREHOOK: query: FROM src
 SELECT evaluate_npe(src.key) LIMIT 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/ct_noperm_loc.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/ct_noperm_loc.q.out b/ql/src/test/results/clientnegative/ct_noperm_loc.q.out
index a3f7aec..ac59863 100644
--- a/ql/src/test/results/clientnegative/ct_noperm_loc.q.out
+++ b/ql/src/test/results/clientnegative/ct_noperm_loc.q.out
@@ -1,11 +1,11 @@
-#### A masked pattern was here ####
+PREHOOK: query: create table foo0(id int) location 'hdfs://### HDFS PATH ###'
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@foo0
-#### A masked pattern was here ####
+POSTHOOK: query: create table foo0(id int) location 'hdfs://### HDFS PATH ###'
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@foo0
-#### A masked pattern was here ####
+FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation CREATETABLE [[INSERT, DELETE] on Object [type=DFS_URI, name=hdfs://### HDFS PATH ###]]

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/ctas_noperm_loc.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/ctas_noperm_loc.q.out b/ql/src/test/results/clientnegative/ctas_noperm_loc.q.out
index 0b8182a..990c7eb 100644
--- a/ql/src/test/results/clientnegative/ctas_noperm_loc.q.out
+++ b/ql/src/test/results/clientnegative/ctas_noperm_loc.q.out
@@ -1 +1 @@
-#### A masked pattern was here ####
+FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation CREATETABLE_AS_SELECT [[INSERT, DELETE] on Object [type=DFS_URI, name=hdfs://### HDFS PATH ###]]

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out b/ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out
index 1794ae6..f273009 100644
--- a/ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out
+++ b/ql/src/test/results/clientnegative/file_with_header_footer_negative.q.out
@@ -1,19 +1,19 @@
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@header_footer_table_1
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@header_footer_table_1
 PREHOOK: query: SELECT * FROM header_footer_table_1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@header_footer_table_1
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: SELECT * FROM header_footer_table_1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@header_footer_table_1
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 Failed with exception java.io.IOException:java.io.IOException: footer number exceeds the limit defined in hive.file.max.footer

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out b/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
index f5cf1ea..238f342 100644
--- a/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
+++ b/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
@@ -1,5 +1,6 @@
 PREHOOK: query: FROM src SELECT TRANSFORM(key, value) USING 'python ../../data/scripts/cat_error.py' AS (key, value)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out b/ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out
index dfc8f54..423df30 100644
--- a/ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out
+++ b/ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out
@@ -1,5 +1,6 @@
 PREHOOK: query: FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script.

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out b/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out
index dfc8f54..423df30 100644
--- a/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out
+++ b/ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out
@@ -1,5 +1,6 @@
 PREHOOK: query: FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script.

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/minimr_broken_pipe.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/minimr_broken_pipe.q.out b/ql/src/test/results/clientnegative/minimr_broken_pipe.q.out
index 553e94a..d0d83de 100644
--- a/ql/src/test/results/clientnegative/minimr_broken_pipe.q.out
+++ b/ql/src/test/results/clientnegative/minimr_broken_pipe.q.out
@@ -1,5 +1,6 @@
 PREHOOK: query: SELECT TRANSFORM(*) USING 'true' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 FAILED: Execution Error, return code 20003 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. An error occurred when trying to close the Operator running your custom script.

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/table_nonprintable_negative.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/table_nonprintable_negative.q.out b/ql/src/test/results/clientnegative/table_nonprintable_negative.q.out
index 8b22480..3891747 100644
--- a/ql/src/test/results/clientnegative/table_nonprintable_negative.q.out
+++ b/ql/src/test/results/clientnegative/table_nonprintable_negative.q.out
@@ -1,17 +1,17 @@
 Found 1 items
-#### A masked pattern was here ####
+-rw-r--r--   3 ### USER ### ### GROUP ###         16 ### HDFS DATE ### hdfs://### HDFS PATH ###Foo/in1.txt
 PREHOOK: query: create external table table_external (c1 int, c2 int)
 partitioned by (day string)
-#### A masked pattern was here ####
+location 'hdfs://### HDFS PATH ###'
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table_external
 POSTHOOK: query: create external table table_external (c1 int, c2 int)
 partitioned by (day string)
-#### A masked pattern was here ####
+location 'hdfs://### HDFS PATH ###'
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table_external
 PREHOOK: query: msck repair table table_external

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientnegative/udf_local_resource.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/udf_local_resource.q.out b/ql/src/test/results/clientnegative/udf_local_resource.q.out
index 6a89955..62664c9 100644
--- a/ql/src/test/results/clientnegative/udf_local_resource.q.out
+++ b/ql/src/test/results/clientnegative/udf_local_resource.q.out
@@ -2,5 +2,5 @@ PREHOOK: query: create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFile
 PREHOOK: type: CREATEFUNCTION
 PREHOOK: Output: database:default
 PREHOOK: Output: default.lookup
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask. Hive warehouse is non-local, but ../../data/files/sales.txt specifies file on local filesystem. Resources on non-local warehouse should specify a non-local scheme/path

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/bucket_num_reducers_acid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_num_reducers_acid.q.out b/ql/src/test/results/clientpositive/bucket_num_reducers_acid.q.out
index 81ba5ae..7234731 100644
--- a/ql/src/test/results/clientpositive/bucket_num_reducers_acid.q.out
+++ b/ql/src/test/results/clientpositive/bucket_num_reducers_acid.q.out
@@ -25,7 +25,7 @@ PREHOOK: Output: default@bucket_nr_acid
 PREHOOK: query: select * from bucket_nr_acid order by a, b
 PREHOOK: type: QUERY
 PREHOOK: Input: default@bucket_nr_acid
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 0	-1
 1	-1
 3	-1

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/bucket_num_reducers_acid2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_num_reducers_acid2.q.out b/ql/src/test/results/clientpositive/bucket_num_reducers_acid2.q.out
index 94a2884..add89f4 100644
--- a/ql/src/test/results/clientpositive/bucket_num_reducers_acid2.q.out
+++ b/ql/src/test/results/clientpositive/bucket_num_reducers_acid2.q.out
@@ -33,7 +33,7 @@ PREHOOK: Output: default@bucket_nr_acid2
 PREHOOK: query: select * from bucket_nr_acid2 order by a, b
 PREHOOK: type: QUERY
 PREHOOK: Input: default@bucket_nr_acid2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 0	-1
 0	-1
 1	-1

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/database_properties.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/database_properties.q.out b/ql/src/test/results/clientpositive/database_properties.q.out
index 685ed25..c401a6e 100644
--- a/ql/src/test/results/clientpositive/database_properties.q.out
+++ b/ql/src/test/results/clientpositive/database_properties.q.out
@@ -13,13 +13,13 @@ default
 PREHOOK: query: create database db2 with dbproperties (
   'mapred.jobtracker.url'='http://my.jobtracker.com:53000',
   'hive.warehouse.dir' = '/user/hive/warehouse',
-#### A masked pattern was here ####
+  'mapred.scratch.dir' = 'hdfs://### HDFS PATH ###')
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:db2
 POSTHOOK: query: create database db2 with dbproperties (
   'mapred.jobtracker.url'='http://my.jobtracker.com:53000',
   'hive.warehouse.dir' = '/user/hive/warehouse',
-#### A masked pattern was here ####
+  'mapred.scratch.dir' = 'hdfs://### HDFS PATH ###')
 POSTHOOK: type: CREATEDATABASE
 POSTHOOK: Output: database:db2
 PREHOOK: query: describe database db2
@@ -35,7 +35,7 @@ PREHOOK: Input: database:db2
 POSTHOOK: query: describe database extended db2
 POSTHOOK: type: DESCDATABASE
 POSTHOOK: Input: database:db2
-#### A masked pattern was here ####
+db2		location/in/test	hive_test_user	USER	{hive.warehouse.dir=/user/hive/warehouse, mapred.jobtracker.url=http://my.jobtracker.com:53000, mapred.scratch.dir=hdfs://### HDFS PATH ###}
 PREHOOK: query: alter database db2 set dbproperties (
   'new.property' = 'some new props',
   'hive.warehouse.dir' = 'new/warehouse/dir')
@@ -52,4 +52,4 @@ PREHOOK: Input: database:db2
 POSTHOOK: query: describe database extended db2
 POSTHOOK: type: DESCDATABASE
 POSTHOOK: Input: database:db2
-#### A masked pattern was here ####
+db2		location/in/test	hive_test_user	USER	{hive.warehouse.dir=new/warehouse/dir, mapred.jobtracker.url=http://my.jobtracker.com:53000, mapred.scratch.dir=hdfs://### HDFS PATH ###, new.property=some new props}

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_auto_purge_tables.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_auto_purge_tables.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_auto_purge_tables.q.out
index a2d7dd2..129abc8 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_auto_purge_tables.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_auto_purge_tables.q.out
@@ -8,12 +8,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_ext_table PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'
@@ -89,12 +89,12 @@ POSTHOOK: Input: default@encrypted_table
 500
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_ext_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_ext_table
 PREHOOK: query: ALTER TABLE encrypted_ext_table SET TBLPROPERTIES("auto.purge"="true")

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
index b8464e9..d8d30b4 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_ctas.q.out
@@ -18,14 +18,14 @@ Encryption zone created: '/build/ql/test/data/warehouse/default/encrypted_tablec
 AS SELECT * from src where key = 100 limit 1
 PREHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: Input: default@src
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:testct
 PREHOOK: Output: testCT@encrypted_tablectas
 #### A masked pattern was here ####
 AS SELECT * from src where key = 100 limit 1
 POSTHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: Input: default@src
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:testct
 POSTHOOK: Output: testCT@encrypted_tablectas
 PREHOOK: query: select * from testCT.encrypted_tablectas

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
index 081fdd5..f8d7b06 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_drop_partition.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table_dp
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table_dp
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table_dp
 Encryption key created: 'key_128'
@@ -43,21 +43,21 @@ POSTHOOK: Input: default@encrypted_table_dp@p=2014-09-24
 2	bar	2014-09-24
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_ext_table_dp
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_ext_table_dp
 #### A masked pattern was here ####
 PREHOOK: type: ALTERTABLE_ADDPARTS
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: default@encrypted_ext_table_dp
 #### A masked pattern was here ####
 POSTHOOK: type: ALTERTABLE_ADDPARTS
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: default@encrypted_ext_table_dp
 POSTHOOK: Output: default@encrypted_ext_table_dp@p=2014-09-23
 PREHOOK: query: SELECT * FROM encrypted_ext_table_dp

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
index d3e0282..996c33a 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_drop_table.q.out
@@ -8,12 +8,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_ext_table
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'
@@ -30,12 +30,12 @@ POSTHOOK: Lineage: encrypted_table.key EXPRESSION [(src)src.FieldSchema(name:key
 POSTHOOK: Lineage: encrypted_table.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_ext_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_ext_table
 PREHOOK: query: SHOW TABLES
@@ -84,12 +84,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table1
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table1
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table1
 Encryption zone created: '/build/ql/test/data/warehouse/default/encrypted_table1' using key: 'key_128'

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_drop_table_in_encrypted_db.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_drop_table_in_encrypted_db.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_drop_table_in_encrypted_db.q.out
index 1287d01..7cf3d46 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_drop_table_in_encrypted_db.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_drop_table_in_encrypted_db.q.out
@@ -9,20 +9,21 @@ POSTHOOK: type: DROPDATABASE
 #### A masked pattern was here ####
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:encrypted_db
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 POSTHOOK: type: CREATEDATABASE
 POSTHOOK: Output: database:encrypted_db
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 Encryption key created: 'key_128'
 Encryption zone created: '/build/ql/test/data/warehouse/encrypted_db.db' using key: 'key_128'
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:encrypted_db
 PREHOOK: Output: encrypted_db@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:encrypted_db
 POSTHOOK: Output: encrypted_db@encrypted_table
 PREHOOK: query: INSERT OVERWRITE TABLE encrypted_db.encrypted_table SELECT * FROM src

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_drop_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_drop_view.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_drop_view.q.out
index 97c7275..a13de4e 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_drop_view.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_drop_view.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS dve_encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dve_encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dve_encrypted_table
 Encryption key created: 'key_128'

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out
index e673e5b..431a834 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out
@@ -10,14 +10,14 @@ PREHOOK: query: create table encryptedTable(value string)
     partitioned by (key string) clustered by (value) into 2 buckets stored as orc
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encryptedTable
 POSTHOOK: query: create table encryptedTable(value string)
     partitioned by (key string) clustered by (value) into 2 buckets stored as orc
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encryptedTable
 Encryption key created: 'key_1'
@@ -130,7 +130,7 @@ PREHOOK: Input: default@unencryptedtable@key=238
 PREHOOK: Input: default@unencryptedtable@key=501
 PREHOOK: Input: default@unencryptedtable@key=502
 PREHOOK: Input: default@unencryptedtable@key=86
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select * from unencryptedTable order by key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@unencryptedtable
@@ -138,7 +138,7 @@ POSTHOOK: Input: default@unencryptedtable@key=238
 POSTHOOK: Input: default@unencryptedtable@key=501
 POSTHOOK: Input: default@unencryptedtable@key=502
 POSTHOOK: Input: default@unencryptedtable@key=86
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 val_238	238
 val_501	501
 val_502	502

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
index c0da979..500b769 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
@@ -10,14 +10,14 @@ PREHOOK: query: create table encryptedTable(key string,
     value string) partitioned by (ds string) clustered by (key) into 2 buckets stored as orc
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encryptedTable
 POSTHOOK: query: create table encryptedTable(key string,
     value string) partitioned by (ds string) clustered by (key) into 2 buckets stored as orc
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encryptedTable
 Encryption key created: 'key_1'
@@ -122,13 +122,13 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@unencryptedtable
 PREHOOK: Input: default@unencryptedtable@ds=today
 PREHOOK: Input: default@unencryptedtable@ds=yesterday
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: select * from unencryptedTable order by key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@unencryptedtable
 POSTHOOK: Input: default@unencryptedtable@ds=today
 POSTHOOK: Input: default@unencryptedtable@ds=yesterday
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 238	val_238	yesterday
 238	val_238	yesterday
 501	val_501	today

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
index a10b60d..8cd4660 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
index 20e753e..5bdc072 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'
@@ -587,9 +587,10 @@ STAGE PLANS:
                   value expressions: _col0 (type: int), _col1 (type: string)
                   auto parallelism: false
       Path -> Alias:
-#### A masked pattern was here ####
+        hdfs://### HDFS PATH ### [$hdt$_1:t2]
+        hdfs://### HDFS PATH ### [$hdt$_0:t1]
       Path -> Partition:
-#### A masked pattern was here ####
+        hdfs://### HDFS PATH ### 
           Partition
             base file name: encrypted_table
             input format: org.apache.hadoop.mapred.TextInputFormat
@@ -602,6 +603,7 @@ STAGE PLANS:
               columns.comments 
               columns.types int:string
 #### A masked pattern was here ####
+              location hdfs://### HDFS PATH ###
               name default.encrypted_table
               numFiles 1
               numRows 500
@@ -623,6 +625,7 @@ STAGE PLANS:
                 columns.comments 
                 columns.types int:string
 #### A masked pattern was here ####
+                location hdfs://### HDFS PATH ###
                 name default.encrypted_table
                 numFiles 1
                 numRows 500
@@ -635,7 +638,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.encrypted_table
             name: default.encrypted_table
-#### A masked pattern was here ####
+        hdfs://### HDFS PATH ### 
           Partition
             base file name: src
             input format: org.apache.hadoop.mapred.TextInputFormat
@@ -648,6 +651,7 @@ STAGE PLANS:
               columns.comments 'default','default'
               columns.types string:string
 #### A masked pattern was here ####
+              location hdfs://### HDFS PATH ###
               name default.src
               numFiles 1
               numRows 500
@@ -669,6 +673,7 @@ STAGE PLANS:
                 columns.comments 'default','default'
                 columns.types string:string
 #### A masked pattern was here ####
+                location hdfs://### HDFS PATH ###
                 name default.src
                 numFiles 1
                 numRows 500
@@ -697,10 +702,10 @@ STAGE PLANS:
           File Output Operator
             compressed: false
             GlobalTableId: 0
-#### A masked pattern was here ####
+            directory: hdfs://### HDFS PATH ###
             NumFilesPerFileSink: 1
             Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-#### A masked pattern was here ####
+            Stats Publishing Key Prefix: hdfs://### HDFS PATH ###
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
index 54f6b27..a01169c 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS table_key_1 PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table_key_1
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table_key_1
 Encryption key created: 'key_1'
@@ -20,12 +20,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS table_key_2 PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table_key_2
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table_key_2
 Encryption key created: 'key_2'
@@ -105,9 +105,10 @@ STAGE PLANS:
                   value expressions: _col1 (type: string)
                   auto parallelism: false
       Path -> Alias:
-#### A masked pattern was here ####
+        hdfs://### HDFS PATH ### [$hdt$_0:t1]
+        hdfs://### HDFS PATH ### [$hdt$_1:t2]
       Path -> Partition:
-#### A masked pattern was here ####
+        hdfs://### HDFS PATH ### 
           Partition
             base file name: table_key_1
             input format: org.apache.hadoop.mapred.TextInputFormat
@@ -120,6 +121,7 @@ STAGE PLANS:
               columns.comments 
               columns.types int:string
 #### A masked pattern was here ####
+              location hdfs://### HDFS PATH ###
               name default.table_key_1
               numFiles 1
               numRows 500
@@ -141,6 +143,7 @@ STAGE PLANS:
                 columns.comments 
                 columns.types int:string
 #### A masked pattern was here ####
+                location hdfs://### HDFS PATH ###
                 name default.table_key_1
                 numFiles 1
                 numRows 500
@@ -153,7 +156,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.table_key_1
             name: default.table_key_1
-#### A masked pattern was here ####
+        hdfs://### HDFS PATH ### 
           Partition
             base file name: table_key_2
             input format: org.apache.hadoop.mapred.TextInputFormat
@@ -166,6 +169,7 @@ STAGE PLANS:
               columns.comments 
               columns.types int:string
 #### A masked pattern was here ####
+              location hdfs://### HDFS PATH ###
               name default.table_key_2
               numFiles 1
               numRows 500
@@ -187,6 +191,7 @@ STAGE PLANS:
                 columns.comments 
                 columns.types int:string
 #### A masked pattern was here ####
+                location hdfs://### HDFS PATH ###
                 name default.table_key_2
                 numFiles 1
                 numRows 500
@@ -215,10 +220,10 @@ STAGE PLANS:
           File Output Operator
             compressed: false
             GlobalTableId: 0
-#### A masked pattern was here ####
+            directory: hdfs://### HDFS PATH ###
             NumFilesPerFileSink: 1
             Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
-#### A masked pattern was here ####
+            Stats Publishing Key Prefix: hdfs://### HDFS PATH ###
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -245,12 +250,12 @@ PREHOOK: query: SELECT * FROM table_key_1 t1 JOIN table_key_2 t2 WHERE (t1.key =
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table_key_1
 PREHOOK: Input: default@table_key_2
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: SELECT * FROM table_key_1 t1 JOIN table_key_2 t2 WHERE (t1.key = t2.key)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@table_key_1
 POSTHOOK: Input: default@table_key_2
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 0	val_0	0	val_0
 0	val_0	0	val_0
 0	val_0	0	val_0

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_load_data_to_encrypted_tables.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_load_data_to_encrypted_tables.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_load_data_to_encrypted_tables.q.out
index 0d1ae19..c250ebb 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_load_data_to_encrypted_tables.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_load_data_to_encrypted_tables.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key1'
@@ -532,11 +532,11 @@ POSTHOOK: Input: default@encrypted_table
 97	val_97
 #### A masked pattern was here ####
 PREHOOK: type: LOAD
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: LOAD
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: default@encrypted_table
 PREHOOK: query: SELECT * FROM encrypted_table
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
index 4eee575..5af6d5d 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
@@ -8,12 +8,12 @@ POSTHOOK: query: DROP DATABASE IF EXISTS encrypted_db
 POSTHOOK: type: DROPDATABASE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'
@@ -21,10 +21,11 @@ Encryption zone created: '/build/ql/test/data/warehouse/encrypted_table' using k
 #### A masked pattern was here ####
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:encrypted_db
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 POSTHOOK: type: CREATEDATABASE
 POSTHOOK: Output: database:encrypted_db
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 Encryption key created: 'key_128_2'
 Encryption zone created: '/build/ql/test/data/warehouse/encrypted_db.db' using key: 'key_128_2'
 PREHOOK: query: INSERT OVERWRITE TABLE encrypted_table SELECT * FROM src
@@ -163,12 +164,12 @@ plain_table
 src
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table_outloc
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table_outloc
 Encryption key created: 'key_128_3'
@@ -194,10 +195,11 @@ src
 #### A masked pattern was here ####
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:encrypted_db_outloc
+PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
 POSTHOOK: type: CREATEDATABASE
 POSTHOOK: Output: database:encrypted_db_outloc
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 Encryption key created: 'key_128_4'
 Encryption zone created: '/build/ql/test/data/specified_db_location' using key: 'key_128_4'
 PREHOOK: query: USE encrypted_db_outloc

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
index 758a8fa..ceb592a 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_encrypted_tbl.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@encrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_unencrypted_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_unencrypted_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_unencrypted_tbl.q.out
index 4c3b853..33380a0 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_unencrypted_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_select_read_only_unencrypted_tbl.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE IF EXISTS unencrypted_table
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+PREHOOK: Input: hdfs://### HDFS PATH ###
 PREHOOK: Output: database:default
 PREHOOK: Output: default@unencrypted_table
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-#### A masked pattern was here ####
+POSTHOOK: Input: hdfs://### HDFS PATH ###
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@unencrypted_table
 PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE unencrypted_table
@@ -23,11 +23,11 @@ POSTHOOK: Output: default@unencrypted_table
 PREHOOK: query: SELECT count(*) FROM unencrypted_table
 PREHOOK: type: QUERY
 PREHOOK: Input: default@unencrypted_table
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: SELECT count(*) FROM unencrypted_table
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@unencrypted_table
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 500
 PREHOOK: query: drop table unencrypted_table
 PREHOOK: type: DROPTABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/42527189/ql/src/test/results/clientpositive/encrypted/encryption_unencrypted_nonhdfs_external_tables.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_unencrypted_nonhdfs_external_tables.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_unencrypted_nonhdfs_external_tables.q.out
index 966f06e..92ac36e 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_unencrypted_nonhdfs_external_tables.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_unencrypted_nonhdfs_external_tables.q.out
@@ -25,11 +25,11 @@ POSTHOOK: Output: default@mydata
 PREHOOK: query: SELECT * from mydata
 PREHOOK: type: QUERY
 PREHOOK: Input: default@mydata
-#### A masked pattern was here ####
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: SELECT * from mydata
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@mydata
-#### A masked pattern was here ####
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 238val_238	NULL
 86val_86	NULL
 311val_311	NULL