Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id CE12818BEF for ; Sat, 30 May 2015 00:08:08 +0000 (UTC) Received: (qmail 8786 invoked by uid 500); 30 May 2015 00:08:08 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 8743 invoked by uid 500); 30 May 2015 00:08:08 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 8731 invoked by uid 99); 30 May 2015 00:08:08 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 30 May 2015 00:08:08 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 31927E03C7; Sat, 30 May 2015 00:08:08 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: sershe@apache.org To: commits@hive.apache.org Message-Id: <1cef5026563648e1bb226e56e004e39d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hive git commit: HIVE-10722 : external table creation with msck in Hive can create unusable partition (Sergey Shelukhin, reviewed by Sushanth Sowmyan) Date: Sat, 30 May 2015 00:08:08 +0000 (UTC) Repository: hive Updated Branches: refs/heads/master 95aad5410 -> 83cc691c5 HIVE-10722 : external table creation with msck in Hive can create unusable partition (Sergey Shelukhin, reviewed by Sushanth Sowmyan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/83cc691c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/83cc691c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/83cc691c Branch: refs/heads/master Commit: 83cc691c5ac1ef5009cc3270f5fc1618dee14d61 Parents: 95aad54 Author: Sergey Shelukhin Authored: Fri May 29 17:07:57 2015 -0700 Committer: Sergey Shelukhin Committed: Fri May 29 17:07:57 2015 -0700 ---------------------------------------------------------------------- .../org/apache/hadoop/hive/conf/HiveConf.java | 8 ++- .../test/resources/testconfiguration.properties | 2 + .../apache/hadoop/hive/metastore/Warehouse.java | 10 ++- .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 37 +++++++++- .../hive/ql/exec/DefaultFetchFormatter.java | 1 - .../hadoop/hive/ql/io/HiveInputFormat.java | 1 - .../formatting/JsonMetaDataFormatter.java | 2 +- .../metadata/formatting/MetaDataFormatter.java | 2 +- .../formatting/TextMetaDataFormatter.java | 76 ++++++++++---------- .../table_nonprintable_negative.q | 11 +++ .../queries/clientpositive/table_nonprintable.q | 30 ++++++++ .../table_nonprintable_negative.q.out | 19 +++++ .../clientpositive/table_nonprintable.q.out | 72 +++++++++++++++++++ 13 files changed, 225 insertions(+), 46 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java ---------------------------------------------------------------------- diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 1a6b534..f495620 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2174,7 +2174,13 @@ public class HiveConf extends Configuration { NWAYJOINREORDER("hive.reorder.nway.joins", true, "Runs reordering of tables within single n-way join (i.e.: picks streamtable)"), HIVE_LOG_N_RECORDS("hive.log.every.n.records", 0L, new RangeValidator(0L, null), - "If value is greater than 0 logs in fixed intervals of size n rather than exponentially."); + "If value is greater than 0 logs in fixed intervals of size n rather than exponentially."), + HIVE_MSCK_PATH_VALIDATION("hive.msck.path.validation", "throw", + new StringSet("throw", "skip", "ignore"), "The approach msck should take with HDFS " + + "directories that are partition-like but contain unsupported characters. 'throw' (an " + + "exception) is the default; 'skip' will skip the invalid directories and still repair the" + + " others; 'ignore' will skip the validation (legacy behavior, causes bugs in many cases)"); + public final String varname; private final String defaultExpr; http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/itests/src/test/resources/testconfiguration.properties ---------------------------------------------------------------------- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index bdb96e8..fe6ee17 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -47,6 +47,7 @@ minimr.query.files=auto_sortmerge_join_16.q,\ smb_mapjoin_8.q,\ stats_counter.q,\ stats_counter_partitioned.q,\ + table_nonprintable.q,\ temp_table_external.q,\ truncate_column_buckets.q,\ uber_reduce.q,\ @@ -517,6 +518,7 @@ minimr.query.negative.files=cluster_tasklog_retrieval.q,\ mapreduce_stack_trace_turnoff.q,\ mapreduce_stack_trace_turnoff_hadoop20.q,\ minimr_broken_pipe.q,\ + table_nonprintable_negative.q,\ udf_local_resource.q # tests are sorted use: perl -pe 's@\\\s*\n@ @g' testconfiguration.properties \ http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java ---------------------------------------------------------------------- diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java index 25119ab..fc0e738 100755 --- a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java @@ -372,11 +372,16 @@ public class Warehouse { * @param name Partition name. * @param result The result. Must be pre-sized to the expected number of columns. */ - public static void makeValsFromName( + public static AbstractList makeValsFromName( String name, AbstractList result) throws MetaException { assert name != null; String[] parts = slash.split(name, 0); - if (parts.length != result.size()) { + if (result == null) { + result = new ArrayList<>(parts.length); + for (int i = 0; i < parts.length; ++i) { + result.add(null); + } + } else if (parts.length != result.size()) { throw new MetaException( "Expected " + result.size() + " components, got " + parts.length + " (" + name + ")"); } @@ -387,6 +392,7 @@ public class Warehouse { } result.set(i, unescapePathName(parts[i].substring(eq + 1))); } + return result; } public static LinkedHashMap makeSpecFromName(String name) http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java index 04c89ae..8bcf860 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java @@ -31,6 +31,7 @@ import java.io.Writer; import java.net.URI; import java.net.URISyntaxException; import java.sql.SQLException; +import java.util.AbstractList; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -1728,6 +1729,40 @@ public class DDLTask extends Task implements Serializable { checker.checkMetastore(names[0], names[1], msckDesc.getPartSpecs(), result); List partsNotInMs = result.getPartitionsNotInMs(); if (msckDesc.isRepairPartitions() && !partsNotInMs.isEmpty()) { + AbstractList vals = null; + String settingStr = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_MSCK_PATH_VALIDATION); + boolean doValidate = !("ignore".equals(settingStr)); + boolean doSkip = doValidate && "skip".equals(settingStr); + // The default setting is "throw"; assume doValidate && !doSkip means throw. + if (doValidate) { + // Validate that we can add partition without escaping. Escaping was originally intended + // to avoid creating invalid HDFS paths; however, if we escape the HDFS path (that we + // deem invalid but HDFS actually supports - it is possible to create HDFS paths with + // unprintable characters like ASCII 7), metastore will create another directory instead + // of the one we are trying to "repair" here. + Iterator iter = partsNotInMs.iterator(); + while (iter.hasNext()) { + CheckResult.PartitionResult part = iter.next(); + try { + vals = Warehouse.makeValsFromName(part.getPartitionName(), vals); + } catch (MetaException ex) { + throw new HiveException(ex); + } + for (String val : vals) { + String escapedPath = FileUtils.escapePathName(val); + assert escapedPath != null; + if (escapedPath.equals(val)) continue; + String errorMsg = "Repair: Cannot add partition " + msckDesc.getTableName() + + ':' + part.getPartitionName() + " due to invalid characters in the name"; + if (doSkip) { + repairOutput.add(errorMsg); + iter.remove(); + } else { + throw new HiveException(errorMsg); + } + } + } + } Table table = db.getTable(msckDesc.getTableName()); AddPartitionDesc apd = new AddPartitionDesc( table.getDbName(), table.getTableName(), false); @@ -1863,7 +1898,7 @@ public class DDLTask extends Task implements Serializable { FileSystem fs = resFile.getFileSystem(conf); outStream = fs.create(resFile); - formatter.showTablePartitons(outStream, parts); + formatter.showTablePartitions(outStream, parts); outStream.close(); outStream = null; http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultFetchFormatter.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultFetchFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultFetchFormatter.java index 37852ef..b8be3a5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultFetchFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultFetchFormatter.java @@ -50,7 +50,6 @@ public class DefaultFetchFormatter implements FetchFormatter { } } -//TODO#: THIS private SerDe initializeSerde(Configuration conf, Properties props) throws Exception { String serdeName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE); Class serdeClass = Class.forName(serdeName, true, http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java index 20d0304..54e1d4e 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -155,7 +155,6 @@ public class HiveInputFormat return inputSplit.getLocations(); } - // TODO#: THIS @Override public void readFields(DataInput in) throws IOException { String inputSplitClassName = in.readUTF(); http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java index 818e7ca..92dc81c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java @@ -313,7 +313,7 @@ public class JsonMetaDataFormatter implements MetaDataFormatter { * Show the table partitions. */ @Override - public void showTablePartitons(DataOutputStream out, List parts) + public void showTablePartitions(DataOutputStream out, List parts) throws HiveException { asJson(out, MapBuilder.create().put("partitions", makeTablePartions(parts)).build()); http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java index 2504e47..55e1b3b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java @@ -93,7 +93,7 @@ public interface MetaDataFormatter { /** * Show the table partitions. */ - public void showTablePartitons(DataOutputStream out, + public void showTablePartitions(DataOutputStream out, List parts) throws HiveException; http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java index 8fabea9..a9e500a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java @@ -105,7 +105,7 @@ class TextMetaDataFormatter implements MetaDataFormatter { try { while (iterTbls.hasNext()) { // create a row per table name - out.writeBytes(iterTbls.next()); + out.write(iterTbls.next().getBytes("UTF-8")); out.write(terminator); } } catch (IOException e) { @@ -148,7 +148,7 @@ class TextMetaDataFormatter implements MetaDataFormatter { outStream.write(terminator); if (part != null) { // show partition information - outStream.writeBytes("Detailed Partition Information"); + outStream.write(("Detailed Partition Information").getBytes("UTF-8")); outStream.write(separator); outStream.write(part.getTPartition().toString().getBytes("UTF-8")); outStream.write(separator); @@ -156,7 +156,7 @@ class TextMetaDataFormatter implements MetaDataFormatter { outStream.write(terminator); } else { // show table information - outStream.writeBytes("Detailed Table Information"); + outStream.write(("Detailed Table Information").getBytes("UTF-8")); outStream.write(separator); outStream.write(tbl.getTTable().toString().getBytes("UTF-8")); outStream.write(separator); @@ -213,21 +213,21 @@ class TextMetaDataFormatter implements MetaDataFormatter { "partition_columns", tbl.getPartCols()); } - outStream.writeBytes("tableName:" + tableName); + outStream.write(("tableName:" + tableName).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("owner:" + owner); + outStream.write(("owner:" + owner).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("location:" + tblLoc); + outStream.write(("location:" + tblLoc).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("inputformat:" + inputFormattCls); + outStream.write(("inputformat:" + inputFormattCls).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("outputformat:" + outputFormattCls); + outStream.write(("outputformat:" + outputFormattCls).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("columns:" + ddlCols); + outStream.write(("columns:" + ddlCols).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("partitioned:" + isPartitioned); + outStream.write(("partitioned:" + isPartitioned).getBytes("UTF-8")); outStream.write(terminator); - outStream.writeBytes("partitionColumns:" + partitionCols); + outStream.write(("partitionColumns:" + partitionCols).getBytes("UTF-8")); outStream.write(terminator); // output file system information Path tblPath = tbl.getPath(); @@ -338,50 +338,50 @@ class TextMetaDataFormatter implements MetaDataFormatter { String unknownString = "unknown"; for (int k = 0; k < indent; k++) { - outStream.writeBytes(Utilities.INDENT); + outStream.write(Utilities.INDENT.getBytes("UTF-8")); } - outStream.writeBytes("totalNumberFiles:"); - outStream.writeBytes(unknown ? unknownString : "" + numOfFiles); + outStream.write("totalNumberFiles:".getBytes("UTF-8")); + outStream.write((unknown ? unknownString : "" + numOfFiles).getBytes("UTF-8")); outStream.write(terminator); for (int k = 0; k < indent; k++) { - outStream.writeBytes(Utilities.INDENT); + outStream.write(Utilities.INDENT.getBytes("UTF-8")); } - outStream.writeBytes("totalFileSize:"); - outStream.writeBytes(unknown ? unknownString : "" + totalFileSize); + outStream.write("totalFileSize:".getBytes("UTF-8")); + outStream.write((unknown ? unknownString : "" + totalFileSize).getBytes("UTF-8")); outStream.write(terminator); for (int k = 0; k < indent; k++) { - outStream.writeBytes(Utilities.INDENT); + outStream.write(Utilities.INDENT.getBytes("UTF-8")); } - outStream.writeBytes("maxFileSize:"); - outStream.writeBytes(unknown ? unknownString : "" + maxFileSize); + outStream.write("maxFileSize:".getBytes("UTF-8")); + outStream.write((unknown ? unknownString : "" + maxFileSize).getBytes("UTF-8")); outStream.write(terminator); for (int k = 0; k < indent; k++) { - outStream.writeBytes(Utilities.INDENT); + outStream.write(Utilities.INDENT.getBytes("UTF-8")); } - outStream.writeBytes("minFileSize:"); + outStream.write("minFileSize:".getBytes("UTF-8")); if (numOfFiles > 0) { - outStream.writeBytes(unknown ? unknownString : "" + minFileSize); + outStream.write((unknown ? unknownString : "" + minFileSize).getBytes("UTF-8")); } else { - outStream.writeBytes(unknown ? unknownString : "" + 0); + outStream.write((unknown ? unknownString : "" + 0).getBytes("UTF-8")); } outStream.write(terminator); for (int k = 0; k < indent; k++) { - outStream.writeBytes(Utilities.INDENT); + outStream.write(Utilities.INDENT.getBytes("UTF-8")); } - outStream.writeBytes("lastAccessTime:"); + outStream.write("lastAccessTime:".getBytes("UTF-8")); outStream.writeBytes((unknown || lastAccessTime < 0) ? unknownString : "" + lastAccessTime); outStream.write(terminator); for (int k = 0; k < indent; k++) { - outStream.writeBytes(Utilities.INDENT); + outStream.write(Utilities.INDENT.getBytes("UTF-8")); } - outStream.writeBytes("lastUpdateTime:"); - outStream.writeBytes(unknown ? unknownString : "" + lastUpdateTime); + outStream.write("lastUpdateTime:".getBytes("UTF-8")); + outStream.write((unknown ? unknownString : "" + lastUpdateTime).getBytes("UTF-8")); outStream.write(terminator); } @@ -389,7 +389,7 @@ class TextMetaDataFormatter implements MetaDataFormatter { * Show the table partitions. */ @Override - public void showTablePartitons(DataOutputStream outStream, List parts) + public void showTablePartitions(DataOutputStream outStream, List parts) throws HiveException { try { @@ -399,9 +399,9 @@ class TextMetaDataFormatter implements MetaDataFormatter { SessionState ss = SessionState.get(); if (ss != null && ss.getConf() != null && !ss.getConf().getBoolVar(HiveConf.ConfVars.HIVE_DECODE_PARTITION_NAME)) { - outStream.writeBytes(part); + outStream.write(part.getBytes("UTF-8")); } else { - outStream.writeBytes(FileUtils.unescapePathName(part)); + outStream.write(FileUtils.unescapePathName(part).getBytes("UTF-8")); } outStream.write(terminator); } @@ -420,7 +420,7 @@ class TextMetaDataFormatter implements MetaDataFormatter { try { for (String database : databases) { // create a row per database name - outStream.writeBytes(database); + outStream.write(database.getBytes("UTF-8")); outStream.write(terminator); } } catch (IOException e) { @@ -436,26 +436,26 @@ class TextMetaDataFormatter implements MetaDataFormatter { String location, String ownerName, String ownerType, Map params) throws HiveException { try { - outStream.writeBytes(database); + outStream.write(database.getBytes("UTF-8")); outStream.write(separator); if (comment != null) { outStream.write(comment.getBytes("UTF-8")); } outStream.write(separator); if (location != null) { - outStream.writeBytes(location); + outStream.write(location.getBytes("UTF-8")); } outStream.write(separator); if (ownerName != null) { - outStream.writeBytes(ownerName); + outStream.write(ownerName.getBytes("UTF-8")); } outStream.write(separator); if (ownerType != null) { - outStream.writeBytes(ownerType); + outStream.write(ownerType.getBytes("UTF-8")); } outStream.write(separator); if (params != null && !params.isEmpty()) { - outStream.writeBytes(params.toString()); + outStream.write(params.toString().getBytes("UTF-8")); } outStream.write(terminator); } catch (IOException e) { http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/test/queries/clientnegative/table_nonprintable_negative.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientnegative/table_nonprintable_negative.q b/ql/src/test/queries/clientnegative/table_nonprintable_negative.q new file mode 100644 index 0000000..d7b4830 --- /dev/null +++ b/ql/src/test/queries/clientnegative/table_nonprintable_negative.q @@ -0,0 +1,11 @@ +set hive.msck.path.validation=throw; + +dfs ${system:test.dfs.mkdir} hdfs:///tmp/temp_table_external/day=Foo; +dfs -copyFromLocal ../../data/files/in1.txt hdfs:///tmp/temp_table_external/day=Foo; +dfs -ls hdfs:///tmp/temp_table_external/day=Foo; + +create external table table_external (c1 int, c2 int) +partitioned by (day string) +location 'hdfs:///tmp/temp_table_external'; + +msck repair table table_external; http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/test/queries/clientpositive/table_nonprintable.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientpositive/table_nonprintable.q b/ql/src/test/queries/clientpositive/table_nonprintable.q new file mode 100644 index 0000000..5ae228e --- /dev/null +++ b/ql/src/test/queries/clientpositive/table_nonprintable.q @@ -0,0 +1,30 @@ +set hive.msck.path.validation=skip; + +dfs ${system:test.dfs.mkdir} hdfs:///tmp/temp_table_external/day=¢Bar; +dfs -copyFromLocal ../../data/files/in1.txt hdfs:///tmp/temp_table_external/day=¢Bar; +dfs -ls hdfs:///tmp/temp_table_external/day=¢Bar; + +dfs ${system:test.dfs.mkdir} hdfs:///tmp/temp_table_external/day=Foo; +dfs -copyFromLocal ../../data/files/in1.txt hdfs:///tmp/temp_table_external/day=Foo; +dfs -ls hdfs:///tmp/temp_table_external/day=Foo; + +dfs -ls hdfs:///tmp/temp_table_external; + +create external table table_external (c1 int, c2 int) +partitioned by (day string) +location 'hdfs:///tmp/temp_table_external'; + +msck repair table table_external; + +dfs -ls hdfs:///tmp/temp_table_external; + +show partitions table_external; +select * from table_external; + +alter table table_external drop partition (day='¢Bar'); + +show partitions table_external; + +drop table table_external; + +dfs -rmr hdfs:///tmp/temp_table_external; http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/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 new file mode 100644 index 0000000..15af756 --- /dev/null +++ b/ql/src/test/results/clientnegative/table_nonprintable_negative.q.out @@ -0,0 +1,19 @@ +Found 1 items +#### A masked pattern was here #### +PREHOOK: query: create external table table_external (c1 int, c2 int) +partitioned by (day string) +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +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 #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_external +PREHOOK: query: msck repair table table_external +PREHOOK: type: MSCK +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask http://git-wip-us.apache.org/repos/asf/hive/blob/83cc691c/ql/src/test/results/clientpositive/table_nonprintable.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/table_nonprintable.q.out b/ql/src/test/results/clientpositive/table_nonprintable.q.out new file mode 100644 index 0000000..d7c93f2 --- /dev/null +++ b/ql/src/test/results/clientpositive/table_nonprintable.q.out @@ -0,0 +1,72 @@ +Found 1 items +#### A masked pattern was here #### +Found 1 items +#### A masked pattern was here #### +Found 2 items +#### A masked pattern was here #### +PREHOOK: query: create external table table_external (c1 int, c2 int) +partitioned by (day string) +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +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 #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_external +PREHOOK: query: msck repair table table_external +PREHOOK: type: MSCK +POSTHOOK: query: msck repair table table_external +POSTHOOK: type: MSCK +Partitions not in metastore: table_external:day=¢Bar +Repair: Cannot add partition table_external:day=Foo due to invalid characters in the name +Repair: Added partition to metastore table_external:day=¢Bar +Found 2 items +#### A masked pattern was here #### +PREHOOK: query: show partitions table_external +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@table_external +POSTHOOK: query: show partitions table_external +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@table_external +day=¢Bar +PREHOOK: query: select * from table_external +PREHOOK: type: QUERY +PREHOOK: Input: default@table_external +PREHOOK: Input: default@table_external@day=¢Bar +#### A masked pattern was here #### +POSTHOOK: query: select * from table_external +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table_external +POSTHOOK: Input: default@table_external@day=¢Bar +#### A masked pattern was here #### +NULL 35 ¢Bar +48 NULL ¢Bar +100 100 ¢Bar +PREHOOK: query: alter table table_external drop partition (day='¢Bar') +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@table_external +PREHOOK: Output: default@table_external@day=¢Bar +POSTHOOK: query: alter table table_external drop partition (day='¢Bar') +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@table_external +POSTHOOK: Output: default@table_external@day=¢Bar +PREHOOK: query: show partitions table_external +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@table_external +POSTHOOK: query: show partitions table_external +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@table_external +PREHOOK: query: drop table table_external +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@table_external +PREHOOK: Output: default@table_external +POSTHOOK: query: drop table table_external +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@table_external +POSTHOOK: Output: default@table_external +#### A masked pattern was here ####