You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by px...@apache.org on 2015/09/05 02:33:02 UTC

[1/3] hive git commit: HIVE-10731 : NullPointerException in HiveParser.g (Pengcheng Xiong via John Pullokkaran)

Repository: hive
Updated Branches:
  refs/heads/branch-1.2 dc121c72b -> 1c80f2e5a


HIVE-10731 : NullPointerException in HiveParser.g (Pengcheng Xiong via John Pullokkaran)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/branch-1.2
Commit: 442f632e987673e9f4f6c846464ea062603ff05d
Parents: dc121c7
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Mon May 18 14:52:00 2015 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Sep 4 17:26:54 2015 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/442f632e/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index ef1a36a..9fb7825 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -650,6 +650,9 @@ import org.apache.hadoop.hive.conf.HiveConf;
     this.hiveConf = hiveConf;
   }
   protected boolean useSQL11ReservedKeywordsForIdentifier() {
+    if(hiveConf==null){
+      return false;
+    }
     return !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS);
   }
 }


[3/3] hive git commit: HIVE-10802 : Table join query with some constant field in select fails (Aihua Xu via Ashutosh Chauhan)

Posted by px...@apache.org.
HIVE-10802 : Table join query with some constant field in select fails (Aihua Xu via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/branch-1.2
Commit: 1c80f2e5ab6a03005bbaf9d465e9740f6fc2f8d9
Parents: e1b7bf7
Author: Aihua Xu <ai...@gmail.com>
Authored: Sat May 30 15:13:00 2015 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Sep 4 17:28:20 2015 -0700

----------------------------------------------------------------------
 .../optimizer/ConstantPropagateProcFactory.java |  57 +++++----
 .../clientpositive/udf_nondeterministic.q       |  11 ++
 .../clientpositive/spark/udf_in_file.q.out      |   2 +-
 .../results/clientpositive/udf_in_file.q.out    |   2 +-
 .../clientpositive/udf_nondeterministic.q.out   | 120 +++++++++++++++++++
 5 files changed, 164 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1c80f2e5/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 20e23d5..f9df8e5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -263,9 +263,13 @@ public final class ConstantPropagateProcFactory {
   /**
    * Fold input expression desc.
    *
-   * If desc is a UDF and all parameters are constants, evaluate it. If desc is a column expression,
-   * find it from propagated constants, and if there is, replace it with constant.
-   *
+   * This function recursively checks if any subexpression of a specified expression
+   * can be evaluated to be constant and replaces such subexpression with the constant.
+   * If the expression is a derterministic UDF and all the subexpressions are constants,
+   * the value will be calculated immediately (during compilation time vs. runtime).
+   * e.g.:
+   *   concat(year, month) => 200112 for year=2001, month=12 since concat is deterministic UDF
+   *   unix_timestamp(time) => unix_timestamp(123) for time=123 since unix_timestamp is nonderministic UDF
    * @param desc folding expression
    * @param constants current propagated constant map
    * @param cppCtx
@@ -280,12 +284,7 @@ public final class ConstantPropagateProcFactory {
     if (desc instanceof ExprNodeGenericFuncDesc) {
       ExprNodeGenericFuncDesc funcDesc = (ExprNodeGenericFuncDesc) desc;
 
-      // The function must be deterministic, or we can't fold it.
       GenericUDF udf = funcDesc.getGenericUDF();
-      if (!isDeterministicUdf(udf)) {
-        LOG.debug("Function " + udf.getClass() + " undeterministic, quit folding.");
-        return desc;
-      }
 
       boolean propagateNext = propagate && propagatableUdfs.contains(udf.getClass());
       List<ExprNodeDesc> newExprs = new ArrayList<ExprNodeDesc>();
@@ -293,27 +292,33 @@ public final class ConstantPropagateProcFactory {
         newExprs.add(foldExpr(childExpr, constants, cppCtx, op, tag, propagateNext));
       }
 
-      // If all child expressions are constants, evaluate UDF immediately
-      ExprNodeDesc constant = evaluateFunction(udf, newExprs, desc.getChildren());
-      if (constant != null) {
-        LOG.debug("Folding expression:" + desc + " -> " + constant);
-        return constant;
+      // Don't evalulate nondeterministic function since the value can only calculate during runtime.
+      if (!isDeterministicUdf(udf)) {
+        LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evalulating immediately.");
+        ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
+        return desc;
       } else {
-
-        // Check if the function can be short cut.
-        ExprNodeDesc shortcut = shortcutFunction(udf, newExprs, op);
-        if (shortcut != null) {
-          LOG.debug("Folding expression:" + desc + " -> " + shortcut);
-          return shortcut;
+        // If all child expressions of deterministic function are constants, evaluate such UDF immediately
+        ExprNodeDesc constant = evaluateFunction(udf, newExprs, desc.getChildren());
+        if (constant != null) {
+          LOG.debug("Folding expression:" + desc + " -> " + constant);
+          return constant;
+        } else {
+          // Check if the function can be short cut.
+          ExprNodeDesc shortcut = shortcutFunction(udf, newExprs, op);
+          if (shortcut != null) {
+            LOG.debug("Folding expression:" + desc + " -> " + shortcut);
+            return shortcut;
+          }
+          ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
         }
-        ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
-      }
 
-      // If in some selected binary operators (=, is null, etc), one of the
-      // expressions are
-      // constant, add them to colToConstatns as half-deterministic columns.
-      if (propagate) {
-        propagate(udf, newExprs, op.getSchema(), constants);
+        // If in some selected binary operators (=, is null, etc), one of the
+        // expressions are
+        // constant, add them to colToConstants as half-deterministic columns.
+        if (propagate) {
+          propagate(udf, newExprs, op.getSchema(), constants);
+        }
       }
 
       return desc;

http://git-wip-us.apache.org/repos/asf/hive/blob/1c80f2e5/ql/src/test/queries/clientpositive/udf_nondeterministic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_nondeterministic.q b/ql/src/test/queries/clientpositive/udf_nondeterministic.q
new file mode 100644
index 0000000..fcb5162
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udf_nondeterministic.q
@@ -0,0 +1,11 @@
+drop table if exists udf_tb1;
+drop table if exists udf_tb2;
+
+create table udf_tb1 (year int, month int);
+create table udf_tb2(month int);
+insert into udf_tb1 values(2001, 11);
+insert into udf_tb2 values(11);
+
+explain
+select unix_timestamp(concat(a.year, '-01-01 00:00:00')) from (select * from udf_tb1 where year=2001) a join udf_tb2 b on (a.month=b.month);
+select unix_timestamp(concat(a.year, '-01-01 00:00:00')) from (select * from udf_tb1 where year=2001) a join udf_tb2 b on (a.month=b.month);

http://git-wip-us.apache.org/repos/asf/hive/blob/1c80f2e5/ql/src/test/results/clientpositive/spark/udf_in_file.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/udf_in_file.q.out b/ql/src/test/results/clientpositive/spark/udf_in_file.q.out
index c5b6205..5ba62b0 100644
--- a/ql/src/test/results/clientpositive/spark/udf_in_file.q.out
+++ b/ql/src/test/results/clientpositive/spark/udf_in_file.q.out
@@ -59,7 +59,7 @@ STAGE PLANS:
           alias: value_src
           Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: in_file(str_val, '../../data/files/test2.dat') (type: boolean), in_file(ch_val, '../../data/files/test2.dat') (type: boolean), in_file(vch_val, '../../data/files/test2.dat') (type: boolean), in_file(str_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(ch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(vch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file('303', '../../data/files/test2.dat') (type: boolean), in_file('304', '../../data/files/test2.dat') (type: boolean), in_file(UDFToString(null), '../../data/files/test2.dat') (type: boolean)
+            expressions: in_file(str_val, '../../data/files/test2.dat') (type: boolean), in_file(ch_val, '../../data/files/test2.dat') (type: boolean), in_file(vch_val, '../../data/files/test2.dat') (type: boolean), in_file(str_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(ch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(vch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file('303', '../../data/files/test2.dat') (type: boolean), in_file('304', '../../data/files/test2.dat') (type: boolean), in_file(null, '../../data/files/test2.dat') (type: boolean)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
             Limit

http://git-wip-us.apache.org/repos/asf/hive/blob/1c80f2e5/ql/src/test/results/clientpositive/udf_in_file.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_in_file.q.out b/ql/src/test/results/clientpositive/udf_in_file.q.out
index b9d0d25..6d2c84b 100644
--- a/ql/src/test/results/clientpositive/udf_in_file.q.out
+++ b/ql/src/test/results/clientpositive/udf_in_file.q.out
@@ -59,7 +59,7 @@ STAGE PLANS:
             alias: value_src
             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: in_file(str_val, '../../data/files/test2.dat') (type: boolean), in_file(ch_val, '../../data/files/test2.dat') (type: boolean), in_file(vch_val, '../../data/files/test2.dat') (type: boolean), in_file(str_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(ch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(vch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file('303', '../../data/files/test2.dat') (type: boolean), in_file('304', '../../data/files/test2.dat') (type: boolean), in_file(UDFToString(null), '../../data/files/test2.dat') (type: boolean)
+              expressions: in_file(str_val, '../../data/files/test2.dat') (type: boolean), in_file(ch_val, '../../data/files/test2.dat') (type: boolean), in_file(vch_val, '../../data/files/test2.dat') (type: boolean), in_file(str_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(ch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(vch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file('303', '../../data/files/test2.dat') (type: boolean), in_file('304', '../../data/files/test2.dat') (type: boolean), in_file(null, '../../data/files/test2.dat') (type: boolean)
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
               Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Limit

http://git-wip-us.apache.org/repos/asf/hive/blob/1c80f2e5/ql/src/test/results/clientpositive/udf_nondeterministic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_nondeterministic.q.out b/ql/src/test/results/clientpositive/udf_nondeterministic.q.out
new file mode 100644
index 0000000..eef5555
--- /dev/null
+++ b/ql/src/test/results/clientpositive/udf_nondeterministic.q.out
@@ -0,0 +1,120 @@
+PREHOOK: query: drop table if exists udf_tb1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists udf_tb1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists udf_tb2
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists udf_tb2
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table udf_tb1 (year int, month int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@udf_tb1
+POSTHOOK: query: create table udf_tb1 (year int, month int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@udf_tb1
+PREHOOK: query: create table udf_tb2(month int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@udf_tb2
+POSTHOOK: query: create table udf_tb2(month int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@udf_tb2
+PREHOOK: query: insert into udf_tb1 values(2001, 11)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@udf_tb1
+POSTHOOK: query: insert into udf_tb1 values(2001, 11)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@udf_tb1
+POSTHOOK: Lineage: udf_tb1.month EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: udf_tb1.year EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: insert into udf_tb2 values(11)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@udf_tb2
+POSTHOOK: query: insert into udf_tb2 values(11)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@udf_tb2
+POSTHOOK: Lineage: udf_tb2.month EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: explain
+select unix_timestamp(concat(a.year, '-01-01 00:00:00')) from (select * from udf_tb1 where year=2001) a join udf_tb2 b on (a.month=b.month)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select unix_timestamp(concat(a.year, '-01-01 00:00:00')) from (select * from udf_tb1 where year=2001) a join udf_tb2 b on (a.month=b.month)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: udf_tb1
+            Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((year = 2001) and month is not null) (type: boolean)
+              Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: month (type: int)
+                outputColumnNames: _col1
+                Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+          TableScan
+            alias: b
+            Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: month is not null (type: boolean)
+              Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: month (type: int)
+                sort order: +
+                Map-reduce partition columns: month (type: int)
+                Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col1 (type: int)
+            1 month (type: int)
+          Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: unix_timestamp('2001-01-01 00:00:00') (type: bigint)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select unix_timestamp(concat(a.year, '-01-01 00:00:00')) from (select * from udf_tb1 where year=2001) a join udf_tb2 b on (a.month=b.month)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@udf_tb1
+PREHOOK: Input: default@udf_tb2
+#### A masked pattern was here ####
+POSTHOOK: query: select unix_timestamp(concat(a.year, '-01-01 00:00:00')) from (select * from udf_tb1 where year=2001) a join udf_tb2 b on (a.month=b.month)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@udf_tb1
+POSTHOOK: Input: default@udf_tb2
+#### A masked pattern was here ####
+978336000


[2/3] hive git commit: HIVE-10771: 'separatorChar' has no effect in 'CREATE TABLE AS SELECT' statement (Yongzhi via Xuefu)

Posted by px...@apache.org.
HIVE-10771: 'separatorChar' has no effect in 'CREATE TABLE AS SELECT' statement (Yongzhi via Xuefu)


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

Branch: refs/heads/branch-1.2
Commit: e1b7bf741799ead1729c8e47ed23907fb02abd9d
Parents: 442f632
Author: Xuefu Zhang <xz...@Cloudera.com>
Authored: Wed May 27 11:15:05 2015 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Sep 4 17:27:36 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |  3 ++
 .../clientpositive/crtseltbl_serdeprops.q       | 11 ++++++
 .../clientpositive/crtseltbl_serdeprops.q.out   | 35 ++++++++++++++++++++
 3 files changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e1b7bf74/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 5d41c69..cb0b680 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -355,6 +355,9 @@ public final class PlanUtils {
       if (crtTblDesc.getTblProps() != null) {
         properties.putAll(crtTblDesc.getTblProps());
       }
+      if (crtTblDesc.getSerdeProps() != null) {
+        properties.putAll(crtTblDesc.getSerdeProps());
+      }
 
       // replace the default input & output file format with those found in
       // crtTblDesc

http://git-wip-us.apache.org/repos/asf/hive/blob/e1b7bf74/ql/src/test/queries/clientpositive/crtseltbl_serdeprops.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/crtseltbl_serdeprops.q b/ql/src/test/queries/clientpositive/crtseltbl_serdeprops.q
new file mode 100644
index 0000000..6e73bc2
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/crtseltbl_serdeprops.q
@@ -0,0 +1,11 @@
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/tmpsepatest;
+CREATE TABLE separator_test 
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde'
+WITH SERDEPROPERTIES ("separatorChar" = "|","quoteChar"="\"","escapeChar"="
+") 
+STORED AS TEXTFILE
+LOCATION 'file:${system:test.tmp.dir}/tmpsepatest'
+AS
+SELECT * FROM src where key = 100 limit 1; 
+dfs -cat ${system:test.tmp.dir}/tmpsepatest/000000_0;
+drop table separator_test;

http://git-wip-us.apache.org/repos/asf/hive/blob/e1b7bf74/ql/src/test/results/clientpositive/crtseltbl_serdeprops.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/crtseltbl_serdeprops.q.out b/ql/src/test/results/clientpositive/crtseltbl_serdeprops.q.out
new file mode 100644
index 0000000..3af9150
--- /dev/null
+++ b/ql/src/test/results/clientpositive/crtseltbl_serdeprops.q.out
@@ -0,0 +1,35 @@
+PREHOOK: query: CREATE TABLE separator_test 
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde'
+WITH SERDEPROPERTIES ("separatorChar" = "|","quoteChar"="\"","escapeChar"="
+") 
+STORED AS TEXTFILE
+#### A masked pattern was here ####
+AS
+SELECT * FROM src where key = 100 limit 1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@separator_test
+POSTHOOK: query: CREATE TABLE separator_test 
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde'
+WITH SERDEPROPERTIES ("separatorChar" = "|","quoteChar"="\"","escapeChar"="
+") 
+STORED AS TEXTFILE
+#### 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: Output: database:default
+POSTHOOK: Output: default@separator_test
+"100"|"val_100"
+PREHOOK: query: drop table separator_test
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@separator_test
+PREHOOK: Output: default@separator_test
+POSTHOOK: query: drop table separator_test
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@separator_test
+POSTHOOK: Output: default@separator_test