Index: conf/hive-default.xml.template =================================================================== --- conf/hive-default.xml.template (revision 1408455) +++ conf/hive-default.xml.template (working copy) @@ -1565,6 +1565,11 @@ Whether to enable TCP keepalive for the Hive server. Keepalive will prevent accumulation of half-open connections. + + hive.decode.partition.name + false + Whether to show the unquoted partition names in query results. + Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1408455) +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy) @@ -672,6 +672,8 @@ SERVER_READ_SOCKET_TIMEOUT("hive.server.read.socket.timeout", 10), SERVER_TCP_KEEP_ALIVE("hive.server.tcp.keepalive", true), + // Whether to show the unquoted partition names in query results. + HIVE_DECODE_PARTITION_NAME("hive.decode.partition.name", false), ; public final String varname; Index: ql/src/test/results/clientpositive/partition_decode_name.q.out =================================================================== --- ql/src/test/results/clientpositive/partition_decode_name.q.out (revision 0) +++ ql/src/test/results/clientpositive/partition_decode_name.q.out (working copy) @@ -0,0 +1,109 @@ +PREHOOK: query: create table sc as select * +from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +POSTHOOK: query: create table sc as select * +from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: default@sc +PREHOOK: query: create table sc_part (key string) partitioned by (ts string) stored as rcfile +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table sc_part (key string) partitioned by (ts string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@sc_part +PREHOOK: query: insert overwrite table sc_part partition(ts) select * from sc +PREHOOK: type: QUERY +PREHOOK: Input: default@sc +PREHOOK: Output: default@sc_part +POSTHOOK: query: insert overwrite table sc_part partition(ts) select * from sc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@sc +POSTHOOK: Output: default@sc_part@ts=2011-01-11+14%3A18%3A26 +POSTHOOK: Output: default@sc_part@ts=2011-01-11+15%3A18%3A26 +POSTHOOK: Output: default@sc_part@ts=2011-01-11+16%3A18%3A26 +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +PREHOOK: query: show partitions sc_part +PREHOOK: type: SHOWPARTITIONS +POSTHOOK: query: show partitions sc_part +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +ts=2011-01-11+14%3A18%3A26 +ts=2011-01-11+15%3A18%3A26 +ts=2011-01-11+16%3A18%3A26 +PREHOOK: query: select count(*) from sc_part where ts is not null +PREHOOK: type: QUERY +PREHOOK: Input: default@sc_part@ts=2011-01-11+14%3A18%3A26 +PREHOOK: Input: default@sc_part@ts=2011-01-11+15%3A18%3A26 +PREHOOK: Input: default@sc_part@ts=2011-01-11+16%3A18%3A26 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from sc_part where ts is not null +POSTHOOK: type: QUERY +POSTHOOK: Input: default@sc_part@ts=2011-01-11+14%3A18%3A26 +POSTHOOK: Input: default@sc_part@ts=2011-01-11+15%3A18%3A26 +POSTHOOK: Input: default@sc_part@ts=2011-01-11+16%3A18%3A26 +#### A masked pattern was here #### +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +3 +PREHOOK: query: insert overwrite table sc_part partition(ts) select * from sc +PREHOOK: type: QUERY +PREHOOK: Input: default@sc +PREHOOK: Output: default@sc_part +POSTHOOK: query: insert overwrite table sc_part partition(ts) select * from sc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@sc +POSTHOOK: Output: default@sc_part@ts=2011-01-11+14%3A18%3A26 +POSTHOOK: Output: default@sc_part@ts=2011-01-11+15%3A18%3A26 +POSTHOOK: Output: default@sc_part@ts=2011-01-11+16%3A18%3A26 +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +PREHOOK: query: show partitions sc_part +PREHOOK: type: SHOWPARTITIONS +POSTHOOK: query: show partitions sc_part +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +ts=2011-01-11+14:18:26 +ts=2011-01-11+15:18:26 +ts=2011-01-11+16:18:26 +PREHOOK: query: select count(*) from sc_part where ts is not null +PREHOOK: type: QUERY +PREHOOK: Input: default@sc_part@ts=2011-01-11+14%3A18%3A26 +PREHOOK: Input: default@sc_part@ts=2011-01-11+15%3A18%3A26 +PREHOOK: Input: default@sc_part@ts=2011-01-11+16%3A18%3A26 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from sc_part where ts is not null +POSTHOOK: type: QUERY +POSTHOOK: Input: default@sc_part@ts=2011-01-11+14%3A18%3A26 +POSTHOOK: Input: default@sc_part@ts=2011-01-11+15%3A18%3A26 +POSTHOOK: Input: default@sc_part@ts=2011-01-11+16%3A18%3A26 +#### A masked pattern was here #### +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+14:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+15:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +POSTHOOK: Lineage: sc_part PARTITION(ts=2011-01-11+16:18:26).key SIMPLE [(sc)sc.FieldSchema(name:_c0, type:string, comment:null), ] +3 Index: ql/src/test/queries/clientpositive/partition_decode_name.q =================================================================== --- ql/src/test/queries/clientpositive/partition_decode_name.q (revision 0) +++ ql/src/test/queries/clientpositive/partition_decode_name.q (working copy) @@ -0,0 +1,21 @@ +create table sc as select * +from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + +create table sc_part (key string) partitioned by (ts string) stored as rcfile; + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +set hive.decode.partition.name=false; +insert overwrite table sc_part partition(ts) select * from sc; +show partitions sc_part; +select count(*) from sc_part where ts is not null; + +set hive.decode.partition.name=true; +insert overwrite table sc_part partition(ts) select * from sc; +show partitions sc_part; +select count(*) from sc_part where ts is not null; Index: ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java (revision 1408455) +++ ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java (working copy) @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -39,6 +40,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.hadoop.hive.shims.ShimLoader; @@ -403,7 +405,15 @@ { try { for (String part : parts) { - outStream.writeBytes(part); + // Partition names are URL encoded. We decode the names unless Hive + // is configured to use the encoded names. + SessionState ss = SessionState.get(); + if (ss != null && ss.getConf() != null && + !ss.getConf().getBoolVar(HiveConf.ConfVars.HIVE_DECODE_PARTITION_NAME)) { + outStream.writeBytes(part); + } else { + outStream.writeBytes(FileUtils.unescapePathName(part)); + } outStream.write(terminator); } } catch (IOException e) {