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 E0A6B11034 for ; Tue, 26 Aug 2014 04:17:41 +0000 (UTC) Received: (qmail 14946 invoked by uid 500); 26 Aug 2014 04:17:41 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 14895 invoked by uid 500); 26 Aug 2014 04:17:41 -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 14884 invoked by uid 99); 26 Aug 2014 04:17:41 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Aug 2014 04:17:41 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Aug 2014 04:17:12 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id C26452388868; Tue, 26 Aug 2014 04:17:08 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1620506 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/exec/ java/org/apache/hadoop/hive/ql/io/merge/ java/org/apache/hadoop/hive/ql/io/orc/ test/queries/clientnegative/ test/queries/clientpositive/ test/results/clientpositive/ Date: Tue, 26 Aug 2014 04:17:08 -0000 To: commits@hive.apache.org From: prasanthj@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140826041708.C26452388868@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: prasanthj Date: Tue Aug 26 04:17:07 2014 New Revision: 1620506 URL: http://svn.apache.org/r1620506 Log: HIVE-7548: Precondition checks should not fail the merge task in case of automatic trigger (Prasanth J reviewed by Gunther Hagleitner) Added: hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat1.q hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat2.q hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out Removed: hive/trunk/ql/src/test/queries/clientnegative/orc_merge1.q hive/trunk/ql/src/test/queries/clientnegative/orc_merge2.q hive/trunk/ql/src/test/queries/clientnegative/orc_merge3.q hive/trunk/ql/src/test/queries/clientnegative/orc_merge4.q hive/trunk/ql/src/test/queries/clientnegative/orc_merge5.q Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeMapper.java hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileMergeMapper.java Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1620506&r1=1620505&r2=1620506&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Tue Aug 26 04:17:07 2014 @@ -1538,6 +1538,24 @@ public final class Utilities { Pattern.compile("^.*?([0-9]+)(_[0-9]{1,6})?(\\..*)?$"); /** + * Some jobs like "INSERT INTO" jobs create copies of files like 0000001_0_copy_2. + * For such files, + * Group 1: 00000001 [taskId] + * Group 3: 0 [task attempId] + * Group 4: _copy_2 [copy suffix] + * Group 6: copy [copy keyword] + * Group 8: 2 [copy file index] + */ + private static final Pattern COPY_FILE_NAME_TO_TASK_ID_REGEX = + Pattern.compile("^.*?"+ // any prefix + "([0-9]+)"+ // taskId + "(_)"+ // separator + "([0-9]{1,6})?"+ // attemptId (limited to 6 digits) + "((_)(\\Bcopy\\B)(_)"+ // copy keyword + "([0-9]{1,6})$)?"+ // copy file index + "(\\..*)?$"); // any suffix/file extension + + /** * This retruns prefix part + taskID for bucket join for partitioned table */ private static final Pattern FILE_NAME_PREFIXED_TASK_ID_REGEX = @@ -1862,21 +1880,42 @@ public final class Utilities { // speculative runs), but the largest should be the correct one since the result // of a successful run should never be smaller than a failed/speculative run. FileStatus toDelete = null; - if (otherFile.getLen() >= one.getLen()) { - toDelete = one; - } else { - toDelete = otherFile; - taskIdToFile.put(taskId, one); - } - long len1 = toDelete.getLen(); - long len2 = taskIdToFile.get(taskId).getLen(); - if (!fs.delete(toDelete.getPath(), true)) { - throw new IOException("Unable to delete duplicate file: " + toDelete.getPath() - + ". Existing file: " + taskIdToFile.get(taskId).getPath()); + + // "LOAD .. INTO" and "INSERT INTO" commands will generate files with + // "_copy_x" suffix. These files are usually read by map tasks and the + // task output gets written to some tmp path. The output file names will + // be of format taskId_attemptId. The usual path for all these tasks is + // srcPath -> taskTmpPath -> tmpPath -> finalPath. + // But, MergeFileTask can move files directly from src path to final path + // without copying it to tmp path. In such cases, different files with + // "_copy_x" suffix will be identified as duplicates (change in value + // of x is wrongly identified as attempt id) and will be deleted. + // To avoid that we will ignore files with "_copy_x" suffix from duplicate + // elimination. + if (!isCopyFile(one.getPath().getName())) { + if (otherFile.getLen() >= one.getLen()) { + toDelete = one; + } else { + toDelete = otherFile; + taskIdToFile.put(taskId, one); + } + long len1 = toDelete.getLen(); + long len2 = taskIdToFile.get(taskId).getLen(); + if (!fs.delete(toDelete.getPath(), true)) { + throw new IOException( + "Unable to delete duplicate file: " + toDelete.getPath() + + ". Existing file: " + + taskIdToFile.get(taskId).getPath()); + } else { + LOG.warn("Duplicate taskid file removed: " + toDelete.getPath() + + " with length " + + len1 + ". Existing file: " + + taskIdToFile.get(taskId).getPath() + " with length " + + len2); + } } else { - LOG.warn("Duplicate taskid file removed: " + toDelete.getPath() + " with length " - + len1 + ". Existing file: " + taskIdToFile.get(taskId).getPath() + " with length " - + len2); + LOG.info(one.getPath() + " file identified as duplicate. This file is" + + " not deleted as it has copySuffix."); } } } @@ -1884,6 +1923,29 @@ public final class Utilities { return taskIdToFile; } + public static boolean isCopyFile(String filename) { + String taskId = filename; + String copyFileSuffix = null; + int dirEnd = filename.lastIndexOf(Path.SEPARATOR); + if (dirEnd != -1) { + taskId = filename.substring(dirEnd + 1); + } + Matcher m = COPY_FILE_NAME_TO_TASK_ID_REGEX.matcher(taskId); + if (!m.matches()) { + LOG.warn("Unable to verify if file name " + filename + " has _copy_ suffix."); + } else { + taskId = m.group(1); + copyFileSuffix = m.group(4); + } + + LOG.debug("Filename: " + filename + " TaskId: " + taskId + " CopySuffix: " + copyFileSuffix); + if (taskId != null && copyFileSuffix != null) { + return true; + } + + return false; + } + public static String getNameMessage(Exception e) { return e.getClass().getName() + "(" + e.getMessage() + ")"; } Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeMapper.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeMapper.java?rev=1620506&r1=1620505&r2=1620506&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeMapper.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeMapper.java Tue Aug 26 04:17:07 2014 @@ -18,8 +18,6 @@ package org.apache.hadoop.hive.ql.io.merge; -import java.io.IOException; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileStatus; @@ -31,6 +29,10 @@ import org.apache.hadoop.hive.ql.metadat import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MapReduceBase; +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + public class MergeMapper extends MapReduceBase { protected JobConf jc; protected Path finalPath; @@ -48,6 +50,7 @@ public class MergeMapper extends MapRedu protected Path tmpPath; protected Path taskTmpPath; protected Path dpPath; + protected Set incompatFileSet; public final static Log LOG = LogFactory.getLog("MergeMapper"); @@ -62,6 +65,7 @@ public class MergeMapper extends MapRedu HiveConf.ConfVars.HIVEMERGECURRENTJOBCONCATENATELISTBUCKETINGDEPTH); Path specPath = MergeOutputFormat.getMergeOutputPath(job); + incompatFileSet = new HashSet(); Path tmpPath = Utilities.toTempPath(specPath); Path taskTmpPath = Utilities.toTaskTempPath(specPath); updatePaths(tmpPath, taskTmpPath); @@ -176,6 +180,23 @@ public class MergeMapper extends MapRedu if (!fs.rename(outPath, finalPath)) { throw new IOException("Unable to rename output to " + finalPath); } + + // move any incompatible files to final path + if (!incompatFileSet.isEmpty()) { + for (Path incompatFile : incompatFileSet) { + String fileName = incompatFile.getName(); + Path destFile = new Path(finalPath.getParent(), fileName); + try { + Utilities.renameOrMoveFiles(fs, incompatFile, destFile); + LOG.info("Moved incompatible file " + incompatFile + " to " + + destFile); + } catch (HiveException e) { + LOG.error("Unable to move " + incompatFile + " to " + destFile); + throw new IOException(e); + } + } + } + } else { if (!autoDelete) { fs.delete(outPath, true); Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileMergeMapper.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileMergeMapper.java?rev=1620506&r1=1620505&r2=1620506&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileMergeMapper.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileMergeMapper.java Tue Aug 26 04:17:07 2014 @@ -18,20 +18,24 @@ package org.apache.hadoop.hive.ql.io.orc; -import java.io.IOException; -import java.util.List; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.merge.MergeMapper; +import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.shims.CombineHiveKey; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Mapper; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + /** * Map task fast merging of ORC files. */ @@ -96,31 +100,9 @@ public class OrcFileMergeMapper extends .inspector(reader.getObjectInspector())); } - // check compatibility with subsequent files - if ((k.getTypes().get(0).getSubtypesCount() != columnCount)) { - throw new IOException("ORCFileMerge failed because the input files are not compatible." - + " Column counts does not match."); - } - - if (!k.compression.equals(compression)) { - throw new IOException("ORCFileMerge failed because the input files are not compatible." - + " Compression codec does not match."); - } - - if (k.compressBufferSize != compressBuffSize) { - throw new IOException("ORCFileMerge failed because the input files are not compatible." - + " Compression buffer size does not match."); - - } - - if (!k.versionList.equals(version)) { - throw new IOException("ORCFileMerge failed because the input files are not compatible." - + " Version does not match."); - } - - if (k.rowIndexStride != rowIndexStride) { - throw new IOException("ORCFileMerge failed because the input files are not compatible." - + " Row index stride does not match."); + if (!checkCompatibility(k, value)) { + incompatFileSet.add(k.getInputPath()); + return; } // next file in the path @@ -153,6 +135,43 @@ public class OrcFileMergeMapper extends } } + private boolean checkCompatibility(OrcFileKeyWrapper k, + OrcFileValueWrapper value) { + // check compatibility with subsequent files + if ((k.getTypes().get(0).getSubtypesCount() != columnCount)) { + LOG.info("Incompatible ORC file merge! Column counts does not match for " + + k.getInputPath()); + return false; + } + + if (!k.compression.equals(compression)) { + LOG.info("Incompatible ORC file merge! Compression codec does not match" + + " for " + k.getInputPath()); + return false; + } + + if (k.compressBufferSize != compressBuffSize) { + LOG.info("Incompatible ORC file merge! Compression buffer size does not" + + " match for " + k.getInputPath()); + return false; + + } + + if (!k.versionList.equals(version)) { + LOG.info("Incompatible ORC file merge! Version does not match for " + + k.getInputPath()); + return false; + } + + if (k.rowIndexStride != rowIndexStride) { + LOG.info("Incompatible ORC file merge! Row index stride does not match" + + " for " + k.getInputPath()); + return false; + } + + return true; + } + @Override public void close() throws IOException { // close writer Added: hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat1.q URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat1.q?rev=1620506&view=auto ============================================================================== --- hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat1.q (added) +++ hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat1.q Tue Aug 26 04:17:07 2014 @@ -0,0 +1,35 @@ +-- SORT_QUERY_RESULTS + +create table orc_merge5 (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc; +create table orc_merge5b (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc; + +load data local inpath '../../data/files/orc_split_elim.orc' into table orc_merge5; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.merge.orcfile.stripe.level=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- 3 mappers +explain insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13; +set hive.exec.orc.write.format=0.12; +insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13; +insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13; +insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13; +set hive.exec.orc.write.format=0.11; +insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13; +insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13; + +-- 5 files total +analyze table orc_merge5b compute statistics noscan; +desc formatted orc_merge5b; +select * from orc_merge5b; + +set hive.merge.orcfile.stripe.level=true; +alter table orc_merge5b concatenate; + +-- 3 file after merging - all 0.12 format files will be merged and 0.11 files will be left behind +analyze table orc_merge5b compute statistics noscan; +desc formatted orc_merge5b; +select * from orc_merge5b; + Added: hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat2.q URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat2.q?rev=1620506&view=auto ============================================================================== --- hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat2.q (added) +++ hive/trunk/ql/src/test/queries/clientpositive/orc_merge_incompat2.q Tue Aug 26 04:17:07 2014 @@ -0,0 +1,51 @@ +-- SORT_QUERY_RESULTS + +-- orc merge file tests for dynamic partition case + +create table orc_merge5 (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc; +create table orc_merge5a (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) partitioned by (st double) stored as orc; + +load data local inpath '../../data/files/orc_split_elim.orc' into table orc_merge5; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=50000; +SET hive.optimize.index.filter=true; +set hive.merge.orcfile.stripe.level=false; +set hive.merge.tezfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.compute.splits.in.am=true; +set tez.am.grouping.min-size=1000; +set tez.am.grouping.max-size=50000; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.optimize.sort.dynamic.partition=false; + +explain insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5; +set hive.exec.orc.default.row.index.stride=1000; +insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid; +insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid; +set hive.exec.orc.default.row.index.stride=2000; +insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid; +insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid; + +analyze table orc_merge5a partition(st=80.0) compute statistics noscan; +analyze table orc_merge5a partition(st=0.8) compute statistics noscan; +desc formatted orc_merge5a partition(st=80.0); +desc formatted orc_merge5a partition(st=0.8); +show partitions orc_merge5a; +select * from orc_merge5a where userid<=13; + +set hive.merge.orcfile.stripe.level=true; +explain alter table orc_merge5a partition(st=80.0) concatenate; +alter table orc_merge5a partition(st=80.0) concatenate; +alter table orc_merge5a partition(st=0.8) concatenate; + +analyze table orc_merge5a partition(st=80.0) compute statistics noscan; +analyze table orc_merge5a partition(st=0.8) compute statistics noscan; +desc formatted orc_merge5a partition(st=80.0); +desc formatted orc_merge5a partition(st=0.8); +show partitions orc_merge5a; +select * from orc_merge5a where userid<=13; + Added: hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out?rev=1620506&view=auto ============================================================================== --- hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out (added) +++ hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat1.q.out Tue Aug 26 04:17:07 2014 @@ -0,0 +1,288 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +create table orc_merge5 (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@orc_merge5 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +create table orc_merge5 (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@orc_merge5 +PREHOOK: query: create table orc_merge5b (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: create table orc_merge5b (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@orc_merge5b +PREHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' into table orc_merge5 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@orc_merge5 +POSTHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' into table orc_merge5 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@orc_merge5 +PREHOOK: query: -- 3 mappers +explain insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +PREHOOK: type: QUERY +POSTHOOK: query: -- 3 mappers +explain insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: orc_merge5 + Statistics: Num rows: 919 Data size: 246402 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (userid <= 13) (type: boolean) + Statistics: Num rows: 306 Data size: 82044 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: userid (type: bigint), string1 (type: string), subtype (type: double), decimal1 (type: decimal(10,0)), ts (type: timestamp) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 306 Data size: 82044 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 306 Data size: 82044 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.orc_merge5b + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.orc_merge5b + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5b +POSTHOOK: Lineage: orc_merge5b.decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5b.string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5b +POSTHOOK: Lineage: orc_merge5b.decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5b.string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5b +POSTHOOK: Lineage: orc_merge5b.decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5b.string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5b +POSTHOOK: Lineage: orc_merge5b.decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5b.string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: insert into table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5 where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5b +POSTHOOK: Lineage: orc_merge5b.decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5b.string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5b.userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: -- 5 files total +analyze table orc_merge5b compute statistics noscan +PREHOOK: type: QUERY +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: -- 5 files total +analyze table orc_merge5b compute statistics noscan +POSTHOOK: type: QUERY +POSTHOOK: Output: default@orc_merge5b +PREHOOK: query: desc formatted orc_merge5b +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@orc_merge5b +POSTHOOK: query: desc formatted orc_merge5b +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@orc_merge5b +# col_name data_type comment + +userid bigint +string1 string +subtype double +decimal1 decimal(10,0) +ts timestamp + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE true + numFiles 5 + numRows 15 + rawDataSize 3825 + totalSize 2739 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from orc_merge5b +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5b +#### A masked pattern was here #### +POSTHOOK: query: select * from orc_merge5b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5b +#### A masked pattern was here #### +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +PREHOOK: query: alter table orc_merge5b concatenate +PREHOOK: type: ALTER_TABLE_MERGE +PREHOOK: Input: default@orc_merge5b +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: alter table orc_merge5b concatenate +POSTHOOK: type: ALTER_TABLE_MERGE +POSTHOOK: Input: default@orc_merge5b +POSTHOOK: Output: default@orc_merge5b +PREHOOK: query: -- 3 file after merging - all 0.12 format files will be merged and 0.11 files will be left behind +analyze table orc_merge5b compute statistics noscan +PREHOOK: type: QUERY +PREHOOK: Output: default@orc_merge5b +POSTHOOK: query: -- 3 file after merging - all 0.12 format files will be merged and 0.11 files will be left behind +analyze table orc_merge5b compute statistics noscan +POSTHOOK: type: QUERY +POSTHOOK: Output: default@orc_merge5b +PREHOOK: query: desc formatted orc_merge5b +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@orc_merge5b +POSTHOOK: query: desc formatted orc_merge5b +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@orc_merge5b +# col_name data_type comment + +userid bigint +string1 string +subtype double +decimal1 decimal(10,0) +ts timestamp + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE true + numFiles 3 + numRows 15 + rawDataSize 3825 + totalSize 2235 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from orc_merge5b +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5b +#### A masked pattern was here #### +POSTHOOK: query: select * from orc_merge5b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5b +#### A masked pattern was here #### +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +13 bar 80.0 2 1969-12-31 16:00:05 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +2 foo 0.8 1 1969-12-31 16:00:00 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 +5 eat 0.8 6 1969-12-31 16:00:20 Added: hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out?rev=1620506&view=auto ============================================================================== --- hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out (added) +++ hive/trunk/ql/src/test/results/clientpositive/orc_merge_incompat2.q.out Tue Aug 26 04:17:07 2014 @@ -0,0 +1,527 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- orc merge file tests for dynamic partition case + +create table orc_merge5 (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@orc_merge5 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- orc merge file tests for dynamic partition case + +create table orc_merge5 (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@orc_merge5 +PREHOOK: query: create table orc_merge5a (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) partitioned by (st double) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@orc_merge5a +POSTHOOK: query: create table orc_merge5a (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) partitioned by (st double) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@orc_merge5a +PREHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' into table orc_merge5 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@orc_merge5 +POSTHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' into table orc_merge5 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@orc_merge5 +PREHOOK: query: explain insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 +PREHOOK: type: QUERY +POSTHOOK: query: explain insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: orc_merge5 + Statistics: Num rows: 919 Data size: 246402 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: userid (type: bigint), string1 (type: string), subtype (type: double), decimal1 (type: decimal(10,0)), ts (type: timestamp), subtype (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 919 Data size: 246402 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 919 Data size: 246402 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.orc_merge5a + + Stage: Stage-0 + Move Operator + tables: + partition: + st + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.orc_merge5a + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5a +POSTHOOK: query: insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: Output: default@orc_merge5a@st=1.8 +POSTHOOK: Output: default@orc_merge5a@st=8.0 +POSTHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5a +POSTHOOK: query: insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: Output: default@orc_merge5a@st=1.8 +POSTHOOK: Output: default@orc_merge5a@st=8.0 +POSTHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5a +POSTHOOK: query: insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: Output: default@orc_merge5a@st=1.8 +POSTHOOK: Output: default@orc_merge5a@st=8.0 +POSTHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5 +PREHOOK: Output: default@orc_merge5a +POSTHOOK: query: insert into table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5 order by userid +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5 +POSTHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: Output: default@orc_merge5a@st=1.8 +POSTHOOK: Output: default@orc_merge5a@st=8.0 +POSTHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=0.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=1.8).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=8.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).decimal1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:decimal1, type:decimal(10,0), comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).string1 SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:string1, type:string, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).subtype SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:subtype, type:double, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).ts SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:ts, type:timestamp, comment:null), ] +POSTHOOK: Lineage: orc_merge5a PARTITION(st=80.0).userid SIMPLE [(orc_merge5)orc_merge5.FieldSchema(name:userid, type:bigint, comment:null), ] +PREHOOK: query: analyze table orc_merge5a partition(st=80.0) compute statistics noscan +PREHOOK: type: QUERY +PREHOOK: Output: default@orc_merge5a +PREHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: query: analyze table orc_merge5a partition(st=80.0) compute statistics noscan +POSTHOOK: type: QUERY +POSTHOOK: Output: default@orc_merge5a +POSTHOOK: Output: default@orc_merge5a@st=80.0 +PREHOOK: query: analyze table orc_merge5a partition(st=0.8) compute statistics noscan +PREHOOK: type: QUERY +PREHOOK: Output: default@orc_merge5a +PREHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: query: analyze table orc_merge5a partition(st=0.8) compute statistics noscan +POSTHOOK: type: QUERY +POSTHOOK: Output: default@orc_merge5a +POSTHOOK: Output: default@orc_merge5a@st=0.8 +PREHOOK: query: desc formatted orc_merge5a partition(st=80.0) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@orc_merge5a +POSTHOOK: query: desc formatted orc_merge5a partition(st=80.0) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@orc_merge5a +# col_name data_type comment + +userid bigint +string1 string +subtype double +decimal1 decimal(10,0) +ts timestamp + +# Partition Information +# col_name data_type comment + +st double + +# Detailed Partition Information +Partition Value: [80.0] +Database: default +Table: orc_merge5a +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 4 + numRows 4 + rawDataSize 1020 + totalSize 1992 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: desc formatted orc_merge5a partition(st=0.8) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@orc_merge5a +POSTHOOK: query: desc formatted orc_merge5a partition(st=0.8) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@orc_merge5a +# col_name data_type comment + +userid bigint +string1 string +subtype double +decimal1 decimal(10,0) +ts timestamp + +# Partition Information +# col_name data_type comment + +st double + +# Detailed Partition Information +Partition Value: [0.8] +Database: default +Table: orc_merge5a +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 4 + numRows 8 + rawDataSize 2040 + totalSize 2104 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions orc_merge5a +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@orc_merge5a +POSTHOOK: query: show partitions orc_merge5a +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@orc_merge5a +st=0.8 +st=1.8 +st=8.0 +st=80.0 +PREHOOK: query: select * from orc_merge5a where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5a +PREHOOK: Input: default@orc_merge5a@st=0.8 +PREHOOK: Input: default@orc_merge5a@st=1.8 +PREHOOK: Input: default@orc_merge5a@st=8.0 +PREHOOK: Input: default@orc_merge5a@st=80.0 +#### A masked pattern was here #### +POSTHOOK: query: select * from orc_merge5a where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5a +POSTHOOK: Input: default@orc_merge5a@st=0.8 +POSTHOOK: Input: default@orc_merge5a@st=1.8 +POSTHOOK: Input: default@orc_merge5a@st=8.0 +POSTHOOK: Input: default@orc_merge5a@st=80.0 +#### A masked pattern was here #### +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +PREHOOK: query: explain alter table orc_merge5a partition(st=80.0) concatenate +PREHOOK: type: ALTER_PARTITION_MERGE +POSTHOOK: query: explain alter table orc_merge5a partition(st=80.0) concatenate +POSTHOOK: type: ALTER_PARTITION_MERGE +STAGE DEPENDENCIES: + Stage-0 is a root stage + Stage-1 depends on stages: Stage-0 + Stage-2 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-0 + + Stage: Stage-1 + Move Operator + tables: + partition: + st 80.0 + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.orc_merge5a + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: alter table orc_merge5a partition(st=80.0) concatenate +PREHOOK: type: ALTER_PARTITION_MERGE +PREHOOK: Input: default@orc_merge5a +PREHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: query: alter table orc_merge5a partition(st=80.0) concatenate +POSTHOOK: type: ALTER_PARTITION_MERGE +POSTHOOK: Input: default@orc_merge5a +POSTHOOK: Output: default@orc_merge5a@st=80.0 +PREHOOK: query: alter table orc_merge5a partition(st=0.8) concatenate +PREHOOK: type: ALTER_PARTITION_MERGE +PREHOOK: Input: default@orc_merge5a +PREHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: query: alter table orc_merge5a partition(st=0.8) concatenate +POSTHOOK: type: ALTER_PARTITION_MERGE +POSTHOOK: Input: default@orc_merge5a +POSTHOOK: Output: default@orc_merge5a@st=0.8 +PREHOOK: query: analyze table orc_merge5a partition(st=80.0) compute statistics noscan +PREHOOK: type: QUERY +PREHOOK: Output: default@orc_merge5a +PREHOOK: Output: default@orc_merge5a@st=80.0 +POSTHOOK: query: analyze table orc_merge5a partition(st=80.0) compute statistics noscan +POSTHOOK: type: QUERY +POSTHOOK: Output: default@orc_merge5a +POSTHOOK: Output: default@orc_merge5a@st=80.0 +PREHOOK: query: analyze table orc_merge5a partition(st=0.8) compute statistics noscan +PREHOOK: type: QUERY +PREHOOK: Output: default@orc_merge5a +PREHOOK: Output: default@orc_merge5a@st=0.8 +POSTHOOK: query: analyze table orc_merge5a partition(st=0.8) compute statistics noscan +POSTHOOK: type: QUERY +POSTHOOK: Output: default@orc_merge5a +POSTHOOK: Output: default@orc_merge5a@st=0.8 +PREHOOK: query: desc formatted orc_merge5a partition(st=80.0) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@orc_merge5a +POSTHOOK: query: desc formatted orc_merge5a partition(st=80.0) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@orc_merge5a +# col_name data_type comment + +userid bigint +string1 string +subtype double +decimal1 decimal(10,0) +ts timestamp + +# Partition Information +# col_name data_type comment + +st double + +# Detailed Partition Information +Partition Value: [80.0] +Database: default +Table: orc_merge5a +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 3 + numRows 4 + rawDataSize 1020 + totalSize 1767 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: desc formatted orc_merge5a partition(st=0.8) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@orc_merge5a +POSTHOOK: query: desc formatted orc_merge5a partition(st=0.8) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@orc_merge5a +# col_name data_type comment + +userid bigint +string1 string +subtype double +decimal1 decimal(10,0) +ts timestamp + +# Partition Information +# col_name data_type comment + +st double + +# Detailed Partition Information +Partition Value: [0.8] +Database: default +Table: orc_merge5a +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 3 + numRows 8 + rawDataSize 2040 + totalSize 1860 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions orc_merge5a +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@orc_merge5a +POSTHOOK: query: show partitions orc_merge5a +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@orc_merge5a +st=0.8 +st=1.8 +st=8.0 +st=80.0 +PREHOOK: query: select * from orc_merge5a where userid<=13 +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_merge5a +PREHOOK: Input: default@orc_merge5a@st=0.8 +PREHOOK: Input: default@orc_merge5a@st=1.8 +PREHOOK: Input: default@orc_merge5a@st=8.0 +PREHOOK: Input: default@orc_merge5a@st=80.0 +#### A masked pattern was here #### +POSTHOOK: query: select * from orc_merge5a where userid<=13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_merge5a +POSTHOOK: Input: default@orc_merge5a@st=0.8 +POSTHOOK: Input: default@orc_merge5a@st=1.8 +POSTHOOK: Input: default@orc_merge5a@st=8.0 +POSTHOOK: Input: default@orc_merge5a@st=80.0 +#### A masked pattern was here #### +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +13 bar 80.0 2 1969-12-31 16:00:05 80.0 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +2 foo 0.8 1 1969-12-31 16:00:00 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8 +5 eat 0.8 6 1969-12-31 16:00:20 0.8