You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ke...@apache.org on 2012/11/20 04:20:20 UTC
svn commit: r1411521 - in /hive/trunk:
common/src/java/org/apache/hadoop/hive/conf/ conf/
ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/
ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/
Author: kevinwilfong
Date: Tue Nov 20 03:20:18 2012
New Revision: 1411521
URL: http://svn.apache.org/viewvc?rev=1411521&view=rev
Log:
HIVE-3679. Unescape partition names returned by show partitions. (Wonho Kim via kevinwilfong)
Added:
hive/trunk/common/src/java/org/apache/hadoop/hive/conf/LoopingByteArrayInputStream.java
hive/trunk/ql/src/test/queries/clientpositive/partition_decode_name.q
hive/trunk/ql/src/test/results/clientpositive/partition_decode_name.q.out
Modified:
hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
hive/trunk/conf/hive-default.xml.template
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1411521&r1=1411520&r2=1411521&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Nov 20 03:20:18 2012
@@ -18,9 +18,10 @@
package org.apache.hadoop.hive.conf;
+import java.io.ByteArrayOutputStream;
import java.io.File;
-import java.io.FileOutputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.io.PrintStream;
import java.net.URL;
import java.util.HashMap;
@@ -52,7 +53,6 @@ public class HiveConf extends Configurat
protected String auxJars;
private static final Log l4j = LogFactory.getLog(HiveConf.class);
private static URL hiveSiteURL = null;
- private static URL confVarURL = null;
private static final Map<String, ConfVars> vars = new HashMap<String, ConfVars>();
@@ -672,6 +672,8 @@ public class HiveConf extends Configurat
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;
@@ -787,35 +789,34 @@ public class HiveConf extends Configurat
}
/**
- * Writes the default ConfVars out to a temporary File and returns
- * a URL pointing to the temporary file.
+ * Writes the default ConfVars out to a byte array and returns an input
+ * stream wrapping that byte array.
+ *
* We need this in order to initialize the ConfVar properties
- * in the underling Configuration object using the addResource(URL)
+ * in the underling Configuration object using the addResource(InputStream)
* method.
*
- * Using Configuration.addResource(InputStream) would be a preferable
- * approach, but it turns out that method is broken since Configuration
- * tries to read the entire contents of the same InputStream repeatedly.
+ * It is important to use a LoopingByteArrayInputStream because it turns out
+ * addResource(InputStream) is broken since Configuration tries to read the
+ * entire contents of the same InputStream repeatedly without resetting it.
+ * LoopingByteArrayInputStream has special logic to handle this.
*/
- private static synchronized URL getConfVarURL() {
- if (confVarURL == null) {
- try {
- Configuration conf = new Configuration();
- File confVarFile = File.createTempFile("hive-default-", ".xml");
- confVarFile.deleteOnExit();
-
- applyDefaultNonNullConfVars(conf);
-
- FileOutputStream fout = new FileOutputStream(confVarFile);
- conf.writeXml(fout);
- fout.close();
- confVarURL = confVarFile.toURI().toURL();
- } catch (Exception e) {
- // We're pretty screwed if we can't load the default conf vars
- throw new RuntimeException("Failed to initialize default Hive configuration variables!", e);
- }
+ private static synchronized InputStream getConfVarInputStream() {
+ byte[] confVarByteArray;
+ try {
+ Configuration conf = new Configuration();
+
+ applyDefaultNonNullConfVars(conf);
+
+ ByteArrayOutputStream confVarBaos = new ByteArrayOutputStream();
+ conf.writeXml(confVarBaos);
+ confVarByteArray = confVarBaos.toByteArray();
+ } catch (Exception e) {
+ // We're pretty screwed if we can't load the default conf vars
+ throw new RuntimeException("Failed to initialize default Hive configuration variables!", e);
}
- return confVarURL;
+
+ return new LoopingByteArrayInputStream(confVarByteArray);
}
public static int getIntVar(Configuration conf, ConfVars var) {
@@ -980,7 +981,7 @@ public class HiveConf extends Configurat
origProp = getAllProperties();
// Overlay the ConfVars. Note that this ignores ConfVars with null values
- addResource(getConfVarURL());
+ addResource(getConfVarInputStream());
// Overlay hive-site.xml if it exists
if (hiveSiteURL != null) {
Added: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/LoopingByteArrayInputStream.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/LoopingByteArrayInputStream.java?rev=1411521&view=auto
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/LoopingByteArrayInputStream.java (added)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/LoopingByteArrayInputStream.java Tue Nov 20 03:20:18 2012
@@ -0,0 +1,46 @@
+/**
+ * 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.conf;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+/**
+ * LoopingByteArrayInputStream.
+ *
+ * This was designed specifically to handle the problem in Hadoop's Configuration object that it
+ * tries to read the entire contents of the same InputStream repeatedly without resetting it.
+ *
+ * The Configuration object does attempt to close the InputStream though, so, since close does
+ * nothing for the ByteArrayInputStream object, override it to reset it.
+ */
+public class LoopingByteArrayInputStream extends ByteArrayInputStream {
+
+ public LoopingByteArrayInputStream(byte[] buf) {
+ super(buf);
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.reset();
+ // According to the Java documentation this does nothing, but just in case
+ super.close();
+ }
+
+}
Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1411521&r1=1411520&r2=1411521&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Tue Nov 20 03:20:18 2012
@@ -1565,6 +1565,11 @@
<description>Whether to enable TCP keepalive for the Hive server. Keepalive will prevent accumulation of half-open connections.</description>
</property>
+<property>
+ <name>hive.decode.partition.name</name>
+ <value>false</value>
+ <description>Whether to show the unquoted partition names in query results.</description>
+</property>
</configuration>
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java?rev=1411521&r1=1411520&r2=1411521&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java Tue Nov 20 03:20:18 2012
@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFac
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.metadat
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 @@ public class TextMetaDataFormatter imple
{
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) {
Added: hive/trunk/ql/src/test/queries/clientpositive/partition_decode_name.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/partition_decode_name.q?rev=1411521&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/partition_decode_name.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/partition_decode_name.q Tue Nov 20 03:20:18 2012
@@ -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;
Added: hive/trunk/ql/src/test/results/clientpositive/partition_decode_name.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/partition_decode_name.q.out?rev=1411521&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/partition_decode_name.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/partition_decode_name.q.out Tue Nov 20 03:20:18 2012
@@ -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