diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index 155abad..d7b566c 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -397,6 +397,7 @@ spark.query.files=alter_merge_orc.q \ sample7.q \ sample8.q \ sample9.q \ + sample10.q \ script_env_var1.q \ script_env_var2.q \ script_pipe.q \ diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index 3ff0782..f618d49 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -155,11 +155,11 @@ public Path getTaskOutPath(String taskId) { /** * Update the final paths according to tmpPath. */ - public Path getFinalPath(String taskId, Path tmpPath, String extension) { + public Path getFinalPath(String fileName, Path tmpPath, String extension) { if (extension != null) { - return new Path(tmpPath, taskId + extension); + return new Path(tmpPath, fileName + extension); } else { - return new Path(tmpPath, taskId); + return new Path(tmpPath, fileName); } } @@ -278,6 +278,7 @@ public Stat getStat() { protected transient JobConf jc; Class outputClass; String taskId; + String fileName; protected boolean filesCreated = false; @@ -457,11 +458,10 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException { Set seenBuckets = new HashSet(); for (int idx = 0; idx < totalFiles; idx++) { if (this.getExecContext() != null && this.getExecContext().getFileId() != null) { - LOG.info("replace taskId from execContext "); - taskId = Utilities.replaceTaskIdFromFilename(taskId, this.getExecContext().getFileId()); + fileName = Utilities.replaceTaskIdFromFilename(taskId, this.getExecContext().getFileId()); - LOG.info("new taskId: FS " + taskId); + LOG.info("get bucket file name with taskId: FS " + fileName); assert !multiFileSpray; assert totalFiles == 1; @@ -475,8 +475,7 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException { int numReducers = totalFiles / numFiles; if (numReducers > 1) { - int currReducer = Integer.valueOf(Utilities.getTaskIdFromFilename(Utilities - .getTaskId(hconf))); + int currReducer = Integer.valueOf(Utilities.getTaskIdFromFilename(taskId)); int reducerIdx = prtner.getPartition(key, null, numReducers); if (currReducer != reducerIdx) { @@ -491,8 +490,13 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException { seenBuckets.add(bucketNum); bucketMap.put(bucketNum, filesIdx); - taskId = Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), bucketNum); + fileName = Utilities.replaceTaskIdFromFilename(taskId, bucketNum); } + + if (fileName == null) { + fileName = Utilities.replaceTaskIdFromFilename(taskId, bucketNum); + } + createBucketForFileIdx(fsp, filesIdx); filesIdx++; } @@ -514,9 +518,9 @@ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) throws HiveException { try { if (isNativeTable) { - fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, null); + fsp.finalPaths[filesIdx] = fsp.getFinalPath(fileName, fsp.tmpPath, null); LOG.info("Final Path: FS " + fsp.finalPaths[filesIdx]); - fsp.outPaths[filesIdx] = fsp.getTaskOutPath(taskId); + fsp.outPaths[filesIdx] = fsp.getTaskOutPath(fileName); LOG.info("Writing to temp file: FS " + fsp.outPaths[filesIdx]); } else { fsp.finalPaths[filesIdx] = fsp.outPaths[filesIdx] = specPath; @@ -527,9 +531,9 @@ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) // we create. String extension = Utilities.getFileExtension(jc, isCompressed, hiveOutputFormat); if (!bDynParts && !this.isSkewedStoredAsSubDirectories) { - fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, parent, extension); + fsp.finalPaths[filesIdx] = fsp.getFinalPath(fileName, parent, extension); } else { - fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, extension); + fsp.finalPaths[filesIdx] = fsp.getFinalPath(fileName, fsp.tmpPath, extension); } LOG.info("New Final Path: FS " + fsp.finalPaths[filesIdx]); @@ -553,7 +557,7 @@ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) // Only set up the updater for insert. For update and delete we don't know unitl we see // the row. ObjectInspector inspector = bDynParts ? subSetOI : outputObjInspector; - int acidBucketNum = Integer.valueOf(Utilities.getTaskIdFromFilename(taskId)); + int acidBucketNum = Integer.valueOf(Utilities.getTaskIdFromFilename(fileName)); fsp.updaters[filesIdx] = HiveFileFormatUtils.getAcidRecordUpdater(jc, conf.getTableInfo(), acidBucketNum, conf, fsp.outPaths[filesIdx], inspector, reporter, -1); } @@ -838,8 +842,8 @@ protected FSPaths getDynOutPaths(List row, String lbDirName) throws Hive if(conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED)) { String buckNum = row.get(row.size() - 1); numericBucketNum = Integer.valueOf(buckNum); - taskId = Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), buckNum); - pathKey = appendToSource(taskId, dpDir); + fileName = Utilities.replaceTaskIdFromFilename(taskId, buckNum); + pathKey = appendToSource(fileName, dpDir); } FSPaths fsp2 = valToPaths.get(pathKey); @@ -1100,7 +1104,7 @@ private void publishStats() throws HiveException { return; } - String taskID = Utilities.getTaskIdFromFilename(Utilities.getTaskId(hconf)); + String taskID = Utilities.getTaskIdFromFilename(taskId); String spSpec = conf.getStaticSpec(); int maxKeyLength = conf.getMaxStatsKeyPrefixLength(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 02f9d99..b4e4d6a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -1042,7 +1042,9 @@ protected synchronized Kryo initialValue() { public static String getTaskId(Configuration hconf) { String taskid = (hconf == null) ? null : hconf.get("mapred.task.id"); if ((taskid == null) || taskid.equals("")) { - return ("" + Math.abs(randGen.nextInt())); + // generate a random int value with fixed 8 digit number. + Integer randomValue = 10000000 + randGen.nextInt(90000000); + return randomValue.toString(); } else { /* * extract the task and attempt id from the hadoop taskid. in version 17 the leading component diff --git ql/src/test/results/clientpositive/spark/sample10.q.out ql/src/test/results/clientpositive/spark/sample10.q.out new file mode 100644 index 0000000..308bab5 --- /dev/null +++ ql/src/test/results/clientpositive/spark/sample10.q.out @@ -0,0 +1,471 @@ +PREHOOK: query: -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +create table srcpartbucket (key string, value string) partitioned by (ds string, hr string) clustered by (key) into 4 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpartbucket +POSTHOOK: query: -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +create table srcpartbucket (key string, value string) partitioned by (ds string, hr string) clustered by (key) into 4 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpartbucket +PREHOOK: query: insert overwrite table srcpartbucket partition(ds, hr) select * from srcpart where ds is not null and key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: default@srcpartbucket +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table srcpartbucket partition(ds, hr) select * from srcpart where ds is not null and key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: default@srcpartbucket@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpartbucket@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@srcpartbucket@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpartbucket@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-09,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-09,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpartbucket PARTITION(ds=2008-04-09,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: explain extended +select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds ORDER BY ds ASC +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds ORDER BY ds ASC +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + srcpartbucket + TOK_TABLEBUCKETSAMPLE + 1 + 4 + TOK_TABLE_OR_COL + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + ds + TOK_SELEXPR + TOK_FUNCTION + count + 1 + TOK_WHERE + TOK_FUNCTION + TOK_ISNOTNULL + TOK_TABLE_OR_COL + ds + TOK_GROUPBY + TOK_TABLE_OR_COL + ds + TOK_ORDERBY + TOK_TABSORTCOLNAMEASC + TOK_TABLE_OR_COL + ds + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 3 <- Reducer 2 (GROUP SORT) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpartbucket + Statistics: Num rows: 12 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: true + predicate: (((hash(key) & 2147483647) % 4) = 0) (type: boolean) + Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: ds (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: 00000000 + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + numFiles 4 + numRows -1 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize -1 + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + totalSize 351 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.srcpartbucket + name: default.srcpartbucket +#### A masked pattern was here #### + Partition + base file name: 00000000 + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + partition values: + ds 2008-04-08 + hr 12 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + numFiles 4 + numRows -1 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize -1 + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + totalSize 351 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.srcpartbucket + name: default.srcpartbucket +#### A masked pattern was here #### + Partition + base file name: 00000000 + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + partition values: + ds 2008-04-09 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + numFiles 4 + numRows -1 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize -1 + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + totalSize 351 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.srcpartbucket + name: default.srcpartbucket +#### A masked pattern was here #### + Partition + base file name: 00000000 + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + partition values: + ds 2008-04-09 + hr 12 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + numFiles 4 + numRows -1 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize -1 + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + totalSize 351 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.srcpartbucket + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpartbucket { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.srcpartbucket + name: default.srcpartbucket + Truncated Path -> Alias: + /srcpartbucket/ds=2008-04-08/hr=11/00000000 [srcpartbucket] + /srcpartbucket/ds=2008-04-08/hr=12/00000000 [srcpartbucket] + /srcpartbucket/ds=2008-04-09/hr=11/00000000 [srcpartbucket] + /srcpartbucket/ds=2008-04-09/hr=12/00000000 [srcpartbucket] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds ORDER BY ds ASC +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpartbucket +PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds ORDER BY ds ASC +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpartbucket +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +2008-04-08 10 +2008-04-09 10 +PREHOOK: query: select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 2 on key) where ds is not null group by ds ORDER BY ds ASC +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpartbucket +PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 2 on key) where ds is not null group by ds ORDER BY ds ASC +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpartbucket +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +2008-04-08 12 +2008-04-09 12 +PREHOOK: query: select * from srcpartbucket where ds is not null ORDER BY key ASC, value ASC, ds ASC, hr ASC +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpartbucket +PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select * from srcpartbucket where ds is not null ORDER BY key ASC, value ASC, ds ASC, hr ASC +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpartbucket +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpartbucket@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 12 +0 val_0 2008-04-08 12 +0 val_0 2008-04-08 12 +0 val_0 2008-04-09 11 +0 val_0 2008-04-09 11 +0 val_0 2008-04-09 11 +0 val_0 2008-04-09 12 +0 val_0 2008-04-09 12 +0 val_0 2008-04-09 12 +2 val_2 2008-04-08 11 +2 val_2 2008-04-08 12 +2 val_2 2008-04-09 11 +2 val_2 2008-04-09 12 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 12 +4 val_4 2008-04-09 11 +4 val_4 2008-04-09 12 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +5 val_5 2008-04-09 11 +5 val_5 2008-04-09 11 +5 val_5 2008-04-09 11 +5 val_5 2008-04-09 12 +5 val_5 2008-04-09 12 +5 val_5 2008-04-09 12 +8 val_8 2008-04-08 11 +8 val_8 2008-04-08 12 +8 val_8 2008-04-09 11 +8 val_8 2008-04-09 12 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 12 +9 val_9 2008-04-09 11 +9 val_9 2008-04-09 12