diff --git data/conf/spark/hive-site.xml data/conf/spark/hive-site.xml deleted file mode 100644 index 016f568..0000000 --- data/conf/spark/hive-site.xml +++ /dev/null @@ -1,233 +0,0 @@ - - - - - - - - - - - - - - - hadoop.tmp.dir - ${test.tmp.dir}/hadoop-tmp - A base for other temporary directories. - - - - hive.exec.scratchdir - ${test.tmp.dir}/scratchdir - Scratch space for Hive jobs - - - - hive.exec.local.scratchdir - ${test.tmp.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - javax.jdo.option.ConnectionURL - jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true - - - - hive.stats.dbconnectionstring - jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true - - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - file://${test.tmp.dir}/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${test.tmp.dir}/log/ - - - - - test.data.files - ${hive.root}/data/files - - - - - hive.jar.path - ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - - test.data.scripts - ${hive.root}/data/scripts - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${test.tmp.dir}/tmp - Location of the structured hive logs - - - - hive.exec.pre.hooks - org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables - Pre Execute Hook for Tests - - - - hive.exec.post.hooks - org.apache.hadoop.hive.ql.hooks.PostExecutePrinter - Post Execute Hook for Tests - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - - Let hive determine whether to run in local mode automatically - Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - false - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - false - Whether Hive ignores the mapjoin hint - - - - io.sort.mb - 10 - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - - hive.stats.dbclass - fs - The default storatge that stores temporary hive statistics. Currently, jdbc, hbase and counter type is supported - - - - hive.execution.engine - spark - Chooses execution engine. Options are: mr (Map reduce, default), tez (hadoop 2 only), spark - - - - spark.master - local-cluster[2,2,1024] - - - - spark.serializer - org.apache.spark.serializer.KryoSerializer - - - - spark.akka.logLifecycleEvents - true - - - - hive.spark.log.dir - ${spark.home}/logs/ - - - - spark.driver.extraClassPath - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - hive.aux.jars.path - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar - - - - hive.users.in.admin.role - hive_admin_user - - - diff --git data/conf/spark/standalone/hive-site.xml data/conf/spark/standalone/hive-site.xml new file mode 100644 index 0000000..016f568 --- /dev/null +++ data/conf/spark/standalone/hive-site.xml @@ -0,0 +1,233 @@ + + + + + + + + + + + + + + + hadoop.tmp.dir + ${test.tmp.dir}/hadoop-tmp + A base for other temporary directories. + + + + hive.exec.scratchdir + ${test.tmp.dir}/scratchdir + Scratch space for Hive jobs + + + + hive.exec.local.scratchdir + ${test.tmp.dir}/localscratchdir/ + Local scratch space for Hive jobs + + + + javax.jdo.option.ConnectionURL + jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true + + + + hive.stats.dbconnectionstring + jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true + + + + + javax.jdo.option.ConnectionDriverName + org.apache.derby.jdbc.EmbeddedDriver + + + + javax.jdo.option.ConnectionUserName + APP + + + + javax.jdo.option.ConnectionPassword + mine + + + + + hive.metastore.warehouse.dir + ${test.warehouse.dir} + + + + + hive.metastore.metadb.dir + file://${test.tmp.dir}/metadb/ + + Required by metastore server or if the uris argument below is not supplied + + + + + test.log.dir + ${test.tmp.dir}/log/ + + + + + test.data.files + ${hive.root}/data/files + + + + + hive.jar.path + ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar + + + + + test.data.scripts + ${hive.root}/data/scripts + + + + + hive.metastore.rawstore.impl + org.apache.hadoop.hive.metastore.ObjectStore + Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database + + + + hive.querylog.location + ${test.tmp.dir}/tmp + Location of the structured hive logs + + + + hive.exec.pre.hooks + org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables + Pre Execute Hook for Tests + + + + hive.exec.post.hooks + org.apache.hadoop.hive.ql.hooks.PostExecutePrinter + Post Execute Hook for Tests + + + + hive.support.concurrency + false + Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. + + + + fs.pfile.impl + org.apache.hadoop.fs.ProxyLocalFileSystem + A proxy for local file system used for cross file system testing + + + + hive.exec.mode.local.auto + false + + Let hive determine whether to run in local mode automatically + Disabling this for tests so that minimr is not affected + + + + + hive.auto.convert.join + false + Whether Hive enable the optimization about converting common join into mapjoin based on the input file size + + + + hive.ignore.mapjoin.hint + false + Whether Hive ignores the mapjoin hint + + + + io.sort.mb + 10 + + + + hive.input.format + org.apache.hadoop.hive.ql.io.CombineHiveInputFormat + The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. + + + + hive.default.rcfile.serde + org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + The default SerDe hive will use for the rcfile format + + + + hive.stats.dbclass + fs + The default storatge that stores temporary hive statistics. Currently, jdbc, hbase and counter type is supported + + + + hive.execution.engine + spark + Chooses execution engine. Options are: mr (Map reduce, default), tez (hadoop 2 only), spark + + + + spark.master + local-cluster[2,2,1024] + + + + spark.serializer + org.apache.spark.serializer.KryoSerializer + + + + spark.akka.logLifecycleEvents + true + + + + hive.spark.log.dir + ${spark.home}/logs/ + + + + spark.driver.extraClassPath + ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar + + + + hive.aux.jars.path + ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar + + + + hive.users.in.admin.role + hive_admin_user + + + diff --git data/conf/spark/yarn-client/hive-site.xml data/conf/spark/yarn-client/hive-site.xml new file mode 100644 index 0000000..9a6f384 --- /dev/null +++ data/conf/spark/yarn-client/hive-site.xml @@ -0,0 +1,248 @@ + + + + + + + + + + + + + + + hadoop.tmp.dir + ${test.tmp.dir}/hadoop-tmp + A base for other temporary directories. + + + + hive.exec.scratchdir + ${test.tmp.dir}/scratchdir + Scratch space for Hive jobs + + + + hive.exec.local.scratchdir + ${test.tmp.dir}/localscratchdir/ + Local scratch space for Hive jobs + + + + javax.jdo.option.ConnectionURL + jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true + + + + hive.stats.dbconnectionstring + jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true + + + + + javax.jdo.option.ConnectionDriverName + org.apache.derby.jdbc.EmbeddedDriver + + + + javax.jdo.option.ConnectionUserName + APP + + + + javax.jdo.option.ConnectionPassword + mine + + + + + hive.metastore.warehouse.dir + ${test.warehouse.dir} + + + + + hive.metastore.metadb.dir + file://${test.tmp.dir}/metadb/ + + Required by metastore server or if the uris argument below is not supplied + + + + + test.log.dir + ${test.tmp.dir}/log/ + + + + + test.data.files + ${hive.root}/data/files + + + + + hive.jar.path + ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar + + + + + test.data.scripts + ${hive.root}/data/scripts + + + + + hive.metastore.rawstore.impl + org.apache.hadoop.hive.metastore.ObjectStore + Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database + + + + hive.querylog.location + ${test.tmp.dir}/tmp + Location of the structured hive logs + + + + hive.exec.pre.hooks + org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables + Pre Execute Hook for Tests + + + + hive.exec.post.hooks + org.apache.hadoop.hive.ql.hooks.PostExecutePrinter + Post Execute Hook for Tests + + + + hive.support.concurrency + false + Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. + + + + fs.pfile.impl + org.apache.hadoop.fs.ProxyLocalFileSystem + A proxy for local file system used for cross file system testing + + + + hive.exec.mode.local.auto + false + + Let hive determine whether to run in local mode automatically + Disabling this for tests so that minimr is not affected + + + + + hive.auto.convert.join + false + Whether Hive enable the optimization about converting common join into mapjoin based on the input file size + + + + hive.ignore.mapjoin.hint + false + Whether Hive ignores the mapjoin hint + + + + io.sort.mb + 10 + + + + hive.input.format + org.apache.hadoop.hive.ql.io.CombineHiveInputFormat + The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. + + + + hive.default.rcfile.serde + org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + The default SerDe hive will use for the rcfile format + + + + hive.stats.dbclass + fs + The default storatge that stores temporary hive statistics. Currently, jdbc, hbase and counter type is supported + + + + hive.execution.engine + spark + Chooses execution engine. Options are: mr (Map reduce, default), tez (hadoop 2 only), spark + + + + spark.master + yarn-client + + + + spark.serializer + org.apache.spark.serializer.KryoSerializer + + + + spark.akka.logLifecycleEvents + true + + + + hive.spark.log.dir + ${spark.home}/logs/ + + + + spark.driver.extraClassPath + ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar + + + + spark.executor.instances + 1 + + + + spark.executor.cores + 2 + + + + spark.testing + true + + + + hive.aux.jars.path + ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/com/google/guava/guava/${guava.version}/guava-${guava.version}.jar:${maven.local.repository}/org/apache/hadoop/hadoop-common/${hadoop-23.version}/hadoop-common-${hadoop-23.version}.jar:${maven.local.repository}/org/apache/hadoop/hadoop-yarn-api/${hadoop-23.version}/hadoop-yarn-api-${hadoop-23.version}.jar + + + + hive.users.in.admin.role + hive_admin_user + + + diff --git itests/pom.xml itests/pom.xml index e1e88f6..e3fe654 100644 --- itests/pom.xml +++ itests/pom.xml @@ -87,7 +87,7 @@ curl -Sso $DOWNLOAD_DIR/$tarName $url fi tar -zxf $DOWNLOAD_DIR/$tarName -C $BASE_DIR - mv $BASE_DIR/${finalName}* $BASE_DIR/$finalName + mv $BASE_DIR/spark-${spark.version}-bin-hadoop2-without-hive $BASE_DIR/$finalName } mkdir -p $DOWNLOAD_DIR download "http://d3jw87u4immizc.cloudfront.net/spark-tarball/spark-${spark.version}-bin-hadoop2-without-hive.tgz" "spark" diff --git itests/qtest-spark/pom.xml itests/qtest-spark/pom.xml index d12fad5..62b2b4a 100644 --- itests/qtest-spark/pom.xml +++ itests/qtest-spark/pom.xml @@ -37,7 +37,6 @@ false - spark false ${hadoop-23.version} -mkdir -p @@ -333,22 +332,39 @@ + + diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index f583aaf..7139f5f 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -713,6 +713,7 @@ spark.query.files=add_part_multiple.q, \ join_thrift.q, \ join_vc.q, \ join_view.q, \ + lateral_view_explode2.q, \ leftsemijoin.q, \ leftsemijoin_mr.q, \ limit_partition_metadataonly.q, \ @@ -996,5 +997,52 @@ spark.query.files=add_part_multiple.q, \ vectorized_shufflejoin.q, \ vectorized_string_funcs.q, \ vectorized_timestamp_funcs.q, \ - windowing.q, \ - lateral_view_explode2.q + windowing.q + +miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\ + bucket4.q,\ + bucket5.q,\ + bucket6.q,\ + bucketizedhiveinputformat.q,\ + bucketmapjoin6.q,\ + bucketmapjoin7.q,\ + constprog_partitioner.q,\ + disable_merge_for_bucketing.q,\ + empty_dir_in_table.q,\ + external_table_with_space_in_location_path.q,\ + file_with_header_footer.q,\ + groupby2.q,\ + import_exported_table.q,\ + index_bitmap3.q,\ + index_bitmap_auto.q,\ + infer_bucket_sort_bucketed_table.q,\ + infer_bucket_sort_dyn_part.q,\ + infer_bucket_sort_map_operators.q,\ + infer_bucket_sort_merge.q,\ + infer_bucket_sort_num_buckets.q,\ + infer_bucket_sort_reducers_power_two.q,\ + input16_cc.q,\ + join1.q,\ + leftsemijoin_mr.q,\ + list_bucket_dml_10.q,\ + load_fs2.q,\ + load_hdfs_file_with_space_in_the_name.q,\ + optrstat_groupby.q,\ + parallel_orderby.q,\ + ql_rewrite_gbtoidx.q,\ + ql_rewrite_gbtoidx_cbo_1.q,\ + ql_rewrite_gbtoidx_cbo_2.q,\ + quotedid_smb.q,\ + reduce_deduplicate.q,\ + remote_script.q,\ + root_dir_external_table.q,\ + schemeAuthority.q,\ + schemeAuthority2.q,\ + scriptfile1.q,\ + scriptfile1_win.q,\ + smb_mapjoin_8.q,\ + stats_counter.q,\ + stats_counter_partitioned.q,\ + temp_table_external.q,\ + truncate_column_buckets.q,\ + uber_reduce.q diff --git itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java index 095b9bd..80ce667 100644 --- itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java +++ itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java @@ -302,6 +302,7 @@ public void initConf() throws Exception { tez, spark, encrypted, + miniSparkOnYarn, none; public static MiniClusterType valueForString(String type) { @@ -313,6 +314,8 @@ public static MiniClusterType valueForString(String type) { return spark; } else if (type.equals("encrypted")) { return encrypted; + } else if (type.equals("miniSparkOnYarn")) { + return miniSparkOnYarn; } else { return none; } @@ -380,6 +383,8 @@ public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString()); if (clusterType == MiniClusterType.tez) { mr = shims.getMiniTezCluster(conf, 4, uriString, 1); + } else if (clusterType == MiniClusterType.miniSparkOnYarn) { + mr = shims.getMiniSparkCluster(conf, 4, uriString, 1); } else { mr = shims.getMiniMrCluster(conf, 4, uriString, 1); } @@ -875,7 +880,8 @@ public String cliInit(String tname, boolean recreate) throws Exception { ss.setIsSilent(true); SessionState oldSs = SessionState.get(); - if (oldSs != null && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.spark)) { + if (oldSs != null && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.spark + || clusterType == MiniClusterType.miniSparkOnYarn)) { sparkSession = oldSs.getSparkSession(); ss.setSparkSession(sparkSession); oldSs.setSparkSession(null); @@ -935,7 +941,8 @@ private CliSessionState startSessionState() ss.err = System.out; SessionState oldSs = SessionState.get(); - if (oldSs != null && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.spark)) { + if (oldSs != null && (clusterType == MiniClusterType.tez || clusterType == MiniClusterType.spark + || clusterType == MiniClusterType.miniSparkOnYarn)) { sparkSession = oldSs.getSparkSession(); ss.setSparkSession(sparkSession); oldSs.setSparkSession(null); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 41a2ab7..0f3c93d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.Serializable; import java.net.MalformedURLException; +import java.net.URL; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -76,8 +77,8 @@ private transient SparkConf sparkConf; private transient HiveConf hiveConf; - private transient List localJars = new ArrayList(); - private transient List localFiles = new ArrayList(); + private transient List localJars = new ArrayList(); + private transient List localFiles = new ArrayList(); private final transient long sparkClientTimtout; @@ -161,26 +162,28 @@ private void refreshLocalResources(SparkWork sparkWork, HiveConf conf) { private void addResources(String addedFiles) { for (String addedFile : CSV_SPLITTER.split(Strings.nullToEmpty(addedFiles))) { - if (!localFiles.contains(addedFile)) { - localFiles.add(addedFile); - try { - remoteClient.addFile(SparkUtilities.getURL(addedFile)); - } catch (MalformedURLException e) { - LOG.warn("Failed to add file:" + addedFile); + try { + URL fileUrl = SparkUtilities.getURL(addedFile); + if (fileUrl != null && !localFiles.contains(fileUrl)) { + localFiles.add(fileUrl); + remoteClient.addFile(fileUrl); } + } catch (MalformedURLException e) { + LOG.warn("Failed to add file:" + addedFile); } } } private void addJars(String addedJars) { for (String addedJar : CSV_SPLITTER.split(Strings.nullToEmpty(addedJars))) { - if (!localJars.contains(addedJar)) { - localJars.add(addedJar); - try { - remoteClient.addJar(SparkUtilities.getURL(addedJar)); - } catch (MalformedURLException e) { - LOG.warn("Failed to add jar:" + addedJar); + try { + URL jarUrl = SparkUtilities.getURL(addedJar); + if (jarUrl != null && !localJars.contains(jarUrl)) { + localJars.add(jarUrl); + remoteClient.addJar(jarUrl); } + } catch (MalformedURLException e) { + LOG.warn("Failed to add jar:" + addedJar); } } } diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/auto_sortmerge_join_16.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/auto_sortmerge_join_16.q.out new file mode 100644 index 0000000..d4ecb19 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/auto_sortmerge_join_16.q.out @@ -0,0 +1,256 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE stage_bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (file_tag STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@stage_bucket_big +POSTHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE stage_bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (file_tag STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@stage_bucket_big +PREHOOK: query: CREATE TABLE bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (day STRING, pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (day STRING, pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: CREATE TABLE stage_bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (file_tag STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@stage_bucket_small +POSTHOOK: query: CREATE TABLE stage_bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (file_tag STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@stage_bucket_small +PREHOOK: query: CREATE TABLE bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: CREATE TABLE bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@stage_bucket_small +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@stage_bucket_small +POSTHOOK: Output: default@stage_bucket_small@file_tag=1 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@stage_bucket_small +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@stage_bucket_small +POSTHOOK: Output: default@stage_bucket_small@file_tag=2 +PREHOOK: query: insert overwrite table bucket_small partition(pri) +select +key, +value, +file_tag as pri +from +stage_bucket_small +where file_tag between 1 and 2 +PREHOOK: type: QUERY +PREHOOK: Input: default@stage_bucket_small +PREHOOK: Input: default@stage_bucket_small@file_tag=1 +PREHOOK: Input: default@stage_bucket_small@file_tag=2 +PREHOOK: Output: default@bucket_small +POSTHOOK: query: insert overwrite table bucket_small partition(pri) +select +key, +value, +file_tag as pri +from +stage_bucket_small +where file_tag between 1 and 2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@stage_bucket_small +POSTHOOK: Input: default@stage_bucket_small@file_tag=1 +POSTHOOK: Input: default@stage_bucket_small@file_tag=2 +POSTHOOK: Output: default@bucket_small@pri=1 +POSTHOOK: Output: default@bucket_small@pri=2 +POSTHOOK: Lineage: bucket_small PARTITION(pri=1).key SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:key, type:bigint, comment:null), ] +POSTHOOK: Lineage: bucket_small PARTITION(pri=1).value SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:value, type:string, comment:null), ] +POSTHOOK: Lineage: bucket_small PARTITION(pri=2).key SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:key, type:bigint, comment:null), ] +POSTHOOK: Lineage: bucket_small PARTITION(pri=2).value SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@stage_bucket_big +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@stage_bucket_big +POSTHOOK: Output: default@stage_bucket_big@file_tag=1 +PREHOOK: query: insert overwrite table bucket_big partition(day,pri) +select +key, +value, +'day1' as day, +1 as pri +from +stage_bucket_big +where +file_tag='1' +PREHOOK: type: QUERY +PREHOOK: Input: default@stage_bucket_big +PREHOOK: Input: default@stage_bucket_big@file_tag=1 +PREHOOK: Output: default@bucket_big +POSTHOOK: query: insert overwrite table bucket_big partition(day,pri) +select +key, +value, +'day1' as day, +1 as pri +from +stage_bucket_big +where +file_tag='1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@stage_bucket_big +POSTHOOK: Input: default@stage_bucket_big@file_tag=1 +POSTHOOK: Output: default@bucket_big@day=day1/pri=1 +POSTHOOK: Lineage: bucket_big PARTITION(day=day1,pri=1).key SIMPLE [(stage_bucket_big)stage_bucket_big.FieldSchema(name:key, type:bigint, comment:null), ] +POSTHOOK: Lineage: bucket_big PARTITION(day=day1,pri=1).value SIMPLE [(stage_bucket_big)stage_bucket_big.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select +a.key , +a.value , +b.value , +'day1' as day, +1 as pri +from +( +select +key, +value +from bucket_big where day='day1' +) a +left outer join +( +select +key, +value +from bucket_small +where pri between 1 and 2 +) b +on +(a.key = b.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@day=day1/pri=1 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@pri=1 +PREHOOK: Input: default@bucket_small@pri=2 +#### A masked pattern was here #### +POSTHOOK: query: select +a.key , +a.value , +b.value , +'day1' as day, +1 as pri +from +( +select +key, +value +from bucket_big where day='day1' +) a +left outer join +( +select +key, +value +from bucket_small +where pri between 1 and 2 +) b +on +(a.key = b.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@day=day1/pri=1 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@pri=1 +POSTHOOK: Input: default@bucket_small@pri=2 +#### A masked pattern was here #### +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +374 val_374 val_374 day1 1 +374 val_374 val_374 day1 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/bucket4.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/bucket4.q.out new file mode 100644 index 0000000..7dd49ac --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/bucket4.q.out @@ -0,0 +1,471 @@ +PREHOOK: query: CREATE TABLE bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket4_1 +POSTHOOK: query: CREATE TABLE bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket4_1 +PREHOOK: query: explain extended +insert overwrite table bucket4_1 +select * from src +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +insert overwrite table bucket4_1 +select * from src +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + bucket4_1 + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + + +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 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToInteger(_col0) (type: int) + sort order: + + Map-reduce partition columns: UDFToInteger(_col0) (type: int) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col0 (type: string), _col1 (type: string) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + /src [src] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Extract + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.bucket4_1 + serialization.ddl struct bucket4_1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket4_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true + + Stage: Stage-0 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.bucket4_1 + serialization.ddl struct bucket4_1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket4_1 + + Stage: Stage-2 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table bucket4_1 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@bucket4_1 +POSTHOOK: query: insert overwrite table bucket4_1 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@bucket4_1 +POSTHOOK: Lineage: bucket4_1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: bucket4_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: explain +select * from bucket4_1 tablesample (bucket 1 out of 2) s +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from bucket4_1 tablesample (bucket 1 out of 2) s +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: s + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + ListSink + +PREHOOK: query: select * from bucket4_1 tablesample (bucket 1 out of 2) s +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket4_1 +#### A masked pattern was here #### +POSTHOOK: query: select * from bucket4_1 tablesample (bucket 1 out of 2) s +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket4_1 +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 +8 val_8 +10 val_10 +12 val_12 +12 val_12 +18 val_18 +18 val_18 +20 val_20 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +28 val_28 +30 val_30 +34 val_34 +42 val_42 +42 val_42 +44 val_44 +54 val_54 +58 val_58 +58 val_58 +64 val_64 +66 val_66 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +78 val_78 +80 val_80 +82 val_82 +84 val_84 +84 val_84 +86 val_86 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +96 val_96 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +104 val_104 +104 val_104 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +120 val_120 +120 val_120 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +134 val_134 +134 val_134 +136 val_136 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +146 val_146 +146 val_146 +150 val_150 +152 val_152 +152 val_152 +156 val_156 +158 val_158 +160 val_160 +162 val_162 +164 val_164 +164 val_164 +166 val_166 +168 val_168 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +176 val_176 +176 val_176 +178 val_178 +180 val_180 +186 val_186 +190 val_190 +192 val_192 +194 val_194 +196 val_196 +200 val_200 +200 val_200 +202 val_202 +208 val_208 +208 val_208 +208 val_208 +214 val_214 +216 val_216 +216 val_216 +218 val_218 +222 val_222 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +238 val_238 +238 val_238 +242 val_242 +242 val_242 +244 val_244 +248 val_248 +252 val_252 +256 val_256 +256 val_256 +258 val_258 +260 val_260 +262 val_262 +266 val_266 +272 val_272 +272 val_272 +274 val_274 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +282 val_282 +282 val_282 +284 val_284 +286 val_286 +288 val_288 +288 val_288 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +306 val_306 +308 val_308 +310 val_310 +316 val_316 +316 val_316 +316 val_316 +318 val_318 +318 val_318 +318 val_318 +322 val_322 +322 val_322 +332 val_332 +336 val_336 +338 val_338 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +366 val_366 +368 val_368 +374 val_374 +378 val_378 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +392 val_392 +394 val_394 +396 val_396 +396 val_396 +396 val_396 +400 val_400 +402 val_402 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +414 val_414 +414 val_414 +418 val_418 +424 val_424 +424 val_424 +430 val_430 +430 val_430 +430 val_430 +432 val_432 +436 val_436 +438 val_438 +438 val_438 +438 val_438 +444 val_444 +446 val_446 +448 val_448 +452 val_452 +454 val_454 +454 val_454 +454 val_454 +458 val_458 +458 val_458 +460 val_460 +462 val_462 +462 val_462 +466 val_466 +466 val_466 +466 val_466 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +470 val_470 +472 val_472 +478 val_478 +478 val_478 +480 val_480 +480 val_480 +480 val_480 +482 val_482 +484 val_484 +490 val_490 +492 val_492 +492 val_492 +494 val_494 +496 val_496 +498 val_498 +498 val_498 +498 val_498 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/bucket5.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/bucket5.q.out new file mode 100644 index 0000000..53a3759 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/bucket5.q.out @@ -0,0 +1,455 @@ +PREHOOK: query: -- Tests that when a multi insert inserts into a bucketed table and a table which is not bucketed +-- the bucketed table is not merged and the table which is not bucketed is + +CREATE TABLE bucketed_table(key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucketed_table +POSTHOOK: query: -- Tests that when a multi insert inserts into a bucketed table and a table which is not bucketed +-- the bucketed table is not merged and the table which is not bucketed is + +CREATE TABLE bucketed_table(key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucketed_table +PREHOOK: query: CREATE TABLE unbucketed_table(key INT, value STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@unbucketed_table +POSTHOOK: query: CREATE TABLE unbucketed_table(key INT, value STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@unbucketed_table +PREHOOK: query: EXPLAIN EXTENDED +FROM src +INSERT OVERWRITE TABLE bucketed_table SELECT key, value +INSERT OVERWRITE TABLE unbucketed_table SELECT key, value cluster by key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN EXTENDED +FROM src +INSERT OVERWRITE TABLE bucketed_table SELECT key, value +INSERT OVERWRITE TABLE unbucketed_table SELECT key, value cluster by key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + bucketed_table + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + value + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + unbucketed_table + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + value + TOK_CLUSTERBY + TOK_TABLE_OR_COL + key + + +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-1 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 2 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToInteger(_col0) (type: int) + sort order: + + Map-reduce partition columns: UDFToInteger(_col0) (type: int) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col0 (type: string), _col1 (type: string) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + /src [src] + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: string) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + /src [src] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Extract + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 500 Data size: 5312 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: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.bucketed_table + serialization.ddl struct bucketed_table { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucketed_table + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Select Operator + expressions: UDFToInteger(KEY.reducesinkkey0) (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 2 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 500 Data size: 5312 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: + bucket_count -1 + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.unbucketed_table + serialization.ddl struct unbucketed_table { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.unbucketed_table + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + + Stage: Stage-0 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.bucketed_table + serialization.ddl struct bucketed_table { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucketed_table + + Stage: Stage-3 + Stats-Aggr Operator +#### A masked pattern was here #### + + Stage: Stage-1 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.unbucketed_table + serialization.ddl struct unbucketed_table { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.unbucketed_table + + Stage: Stage-4 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: FROM src +INSERT OVERWRITE TABLE bucketed_table SELECT key, value +INSERT OVERWRITE TABLE unbucketed_table SELECT key, value cluster by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@bucketed_table +PREHOOK: Output: default@unbucketed_table +POSTHOOK: query: FROM src +INSERT OVERWRITE TABLE bucketed_table SELECT key, value +INSERT OVERWRITE TABLE unbucketed_table SELECT key, value cluster by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@bucketed_table +POSTHOOK: Output: default@unbucketed_table +POSTHOOK: Lineage: bucketed_table.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: bucketed_table.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: unbucketed_table.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: unbucketed_table.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESC FORMATTED bucketed_table +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@bucketed_table +POSTHOOK: query: DESC FORMATTED bucketed_table +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@bucketed_table +# col_name data_type comment + +key int +value string + +# 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 + SORTBUCKETCOLSPREFIX TRUE + numFiles 2 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: 2 +Bucket Columns: [key] +Sort Columns: [Order(col:key, order:1)] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 1 OUT OF 2) s LIMIT 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@bucketed_table +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 1 OUT OF 2) s LIMIT 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucketed_table +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 +8 val_8 +10 val_10 +12 val_12 +12 val_12 +18 val_18 +PREHOOK: query: SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 2 OUT OF 2) s LIMIT 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@bucketed_table +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 2 OUT OF 2) s LIMIT 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucketed_table +#### A masked pattern was here #### +5 val_5 +5 val_5 +5 val_5 +9 val_9 +11 val_11 +15 val_15 +15 val_15 +17 val_17 +19 val_19 +27 val_27 +PREHOOK: query: -- Should be 2 (not merged) +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM bucketed_table +PREHOOK: type: QUERY +PREHOOK: Input: default@bucketed_table +#### A masked pattern was here #### +POSTHOOK: query: -- Should be 2 (not merged) +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM bucketed_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucketed_table +#### A masked pattern was here #### +2 +PREHOOK: query: -- Should be 1 (merged) +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM unbucketed_table +PREHOOK: type: QUERY +PREHOOK: Input: default@unbucketed_table +#### A masked pattern was here #### +POSTHOOK: query: -- Should be 1 (merged) +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM unbucketed_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@unbucketed_table +#### A masked pattern was here #### +2 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/bucket6.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/bucket6.q.out new file mode 100644 index 0000000..ea51fb4 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/bucket6.q.out @@ -0,0 +1,193 @@ +PREHOOK: query: CREATE TABLE src_bucket(key STRING, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_bucket +POSTHOOK: query: CREATE TABLE src_bucket(key STRING, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_bucket +PREHOOK: query: explain +insert into table src_bucket select key,value from srcpart +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert into table src_bucket select key,value from srcpart +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 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Extract + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src_bucket + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src_bucket + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert into table src_bucket select key,value from srcpart +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@src_bucket +POSTHOOK: query: insert into table src_bucket select key,value from srcpart +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@src_bucket +POSTHOOK: Lineage: src_bucket.key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src_bucket.value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: select * from src_bucket limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@src_bucket +#### A masked pattern was here #### +POSTHOOK: query: select * from src_bucket limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_bucket +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +105 val_105 +105 val_105 +105 val_105 +105 val_105 +11 val_11 +11 val_11 +11 val_11 +11 val_11 +114 val_114 +114 val_114 +114 val_114 +114 val_114 +116 val_116 +116 val_116 +116 val_116 +116 val_116 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +136 val_136 +136 val_136 +136 val_136 +136 val_136 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +143 val_143 +143 val_143 +143 val_143 +145 val_145 +145 val_145 +145 val_145 +145 val_145 +149 val_149 +149 val_149 +149 val_149 +149 val_149 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/bucketizedhiveinputformat.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/bucketizedhiveinputformat.q.out new file mode 100644 index 0000000..98c5802 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/bucketizedhiveinputformat.q.out @@ -0,0 +1,111 @@ +PREHOOK: query: CREATE TABLE T1(name STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T1 +POSTHOOK: query: CREATE TABLE T1(name STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T2 +POSTHOOK: query: CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: INSERT OVERWRITE TABLE T2 SELECT * FROM ( +SELECT tmp1.name as name FROM ( + SELECT name, 'MMM' AS n FROM T1) tmp1 + JOIN (SELECT 'MMM' AS n FROM T1) tmp2 + JOIN (SELECT 'MMM' AS n FROM T1) tmp3 + ON tmp1.n = tmp2.n AND tmp1.n = tmp3.n) ttt LIMIT 5000000 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2 +POSTHOOK: query: INSERT OVERWRITE TABLE T2 SELECT * FROM ( +SELECT tmp1.name as name FROM ( + SELECT name, 'MMM' AS n FROM T1) tmp1 + JOIN (SELECT 'MMM' AS n FROM T1) tmp2 + JOIN (SELECT 'MMM' AS n FROM T1) tmp3 + ON tmp1.n = tmp2.n AND tmp1.n = tmp3.n) ttt LIMIT 5000000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.name SIMPLE [(t1)t1.FieldSchema(name:name, type:string, comment:null), ] +PREHOOK: query: CREATE TABLE T3(name STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T3 +POSTHOOK: query: CREATE TABLE T3(name STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T3 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t3 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t3 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T3 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t3 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T3 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t3 +PREHOOK: query: -- 2 split by max.split.size +SELECT COUNT(1) FROM T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: -- 2 split by max.split.size +SELECT COUNT(1) FROM T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +5000000 +PREHOOK: query: -- 1 split for two file +SELECT COUNT(1) FROM T3 +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: -- 1 split for two file +SELECT COUNT(1) FROM T3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- 1 split +SELECT COUNT(1) FROM T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: -- 1 split +SELECT COUNT(1) FROM T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +5000000 +PREHOOK: query: -- 2 split for two file +SELECT COUNT(1) FROM T3 +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: -- 2 split for two file +SELECT COUNT(1) FROM T3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +1000 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/bucketmapjoin6.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/bucketmapjoin6.q.out new file mode 100644 index 0000000..198404b --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/bucketmapjoin6.q.out @@ -0,0 +1,146 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +create table tmp1 (a string, b string) clustered by (a) sorted by (a) into 10 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tmp1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +create table tmp1 (a string, b string) clustered by (a) sorted by (a) into 10 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tmp1 +PREHOOK: query: create table tmp2 (a string, b string) clustered by (a) sorted by (a) into 10 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tmp2 +POSTHOOK: query: create table tmp2 (a string, b string) clustered by (a) sorted by (a) into 10 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tmp2 +PREHOOK: query: insert overwrite table tmp1 select * from src where key < 50 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tmp1 +POSTHOOK: query: insert overwrite table tmp1 select * from src where key < 50 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tmp1 +POSTHOOK: Lineage: tmp1.a SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp1.b SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert overwrite table tmp2 select * from src where key < 50 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tmp2 +POSTHOOK: query: insert overwrite table tmp2 select * from src where key < 50 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tmp2 +POSTHOOK: Lineage: tmp2.a SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp2.b SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: create table tmp3 (a string, b string, c string) clustered by (a) sorted by (a) into 10 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tmp3 +POSTHOOK: query: create table tmp3 (a string, b string, c string) clustered by (a) sorted by (a) into 10 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tmp3 +PREHOOK: query: insert overwrite table tmp3 + select /*+ MAPJOIN(l) */ i.a, i.b, l.b + from tmp1 i join tmp2 l ON i.a = l.a +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp1 +PREHOOK: Input: default@tmp2 +PREHOOK: Output: default@tmp3 +POSTHOOK: query: insert overwrite table tmp3 + select /*+ MAPJOIN(l) */ i.a, i.b, l.b + from tmp1 i join tmp2 l ON i.a = l.a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp1 +POSTHOOK: Input: default@tmp2 +POSTHOOK: Output: default@tmp3 +POSTHOOK: Lineage: tmp3.a SIMPLE [(tmp1)i.FieldSchema(name:a, type:string, comment:null), ] +POSTHOOK: Lineage: tmp3.b SIMPLE [(tmp1)i.FieldSchema(name:b, type:string, comment:null), ] +POSTHOOK: Lineage: tmp3.c SIMPLE [(tmp2)l.FieldSchema(name:b, type:string, comment:null), ] +PREHOOK: query: select * from tmp3 +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp3 +#### A masked pattern was here #### +POSTHOOK: query: select * from tmp3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp3 +#### A masked pattern was here #### +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +10 val_10 val_10 +11 val_11 val_11 +12 val_12 val_12 +12 val_12 val_12 +12 val_12 val_12 +12 val_12 val_12 +15 val_15 val_15 +15 val_15 val_15 +15 val_15 val_15 +15 val_15 val_15 +17 val_17 val_17 +18 val_18 val_18 +18 val_18 val_18 +18 val_18 val_18 +18 val_18 val_18 +19 val_19 val_19 +2 val_2 val_2 +20 val_20 val_20 +24 val_24 val_24 +24 val_24 val_24 +24 val_24 val_24 +24 val_24 val_24 +26 val_26 val_26 +26 val_26 val_26 +26 val_26 val_26 +26 val_26 val_26 +27 val_27 val_27 +28 val_28 val_28 +30 val_30 val_30 +33 val_33 val_33 +34 val_34 val_34 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +37 val_37 val_37 +37 val_37 val_37 +37 val_37 val_37 +37 val_37 val_37 +4 val_4 val_4 +41 val_41 val_41 +42 val_42 val_42 +42 val_42 val_42 +42 val_42 val_42 +42 val_42 val_42 +43 val_43 val_43 +44 val_44 val_44 +47 val_47 val_47 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +8 val_8 val_8 +9 val_9 val_9 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/bucketmapjoin7.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/bucketmapjoin7.q.out new file mode 100644 index 0000000..1cdcbd5 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/bucketmapjoin7.q.out @@ -0,0 +1,384 @@ +PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcbucket_mapjoin_part_1 +POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcbucket_mapjoin_part_1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcbucket_mapjoin_part_1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcbucket_mapjoin_part_1 +POSTHOOK: Output: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcbucket_mapjoin_part_2 +POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcbucket_mapjoin_part_2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcbucket_mapjoin_part_2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcbucket_mapjoin_part_2 +POSTHOOK: Output: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +PREHOOK: query: -- Tests that bucket map join works with a table with more than one level of partitioning + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' +ORDER BY a.key, b.value LIMIT 1 +PREHOOK: type: QUERY +POSTHOOK: query: -- Tests that bucket map join works with a table with more than one level of partitioning + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' +ORDER BY a.key, b.value LIMIT 1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + srcbucket_mapjoin_part_1 + a + TOK_TABREF + TOK_TABNAME + srcbucket_mapjoin_part_2 + b + AND + AND + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + = + . + TOK_TABLE_OR_COL + a + ds + '2008-04-08' + = + . + TOK_TABLE_OR_COL + b + ds + '2008-04-08' + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_HINTLIST + TOK_HINT + TOK_MAPJOIN + TOK_HINTARGLIST + b + TOK_SELEXPR + . + TOK_TABLE_OR_COL + a + key + TOK_SELEXPR + . + TOK_TABLE_OR_COL + b + value + TOK_ORDERBY + TOK_TABSORTCOLNAMEASC + . + TOK_TABLE_OR_COL + a + key + TOK_TABSORTCOLNAMEASC + . + TOK_TABLE_OR_COL + b + value + TOK_LIMIT + 1 + + +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 26 Data size: 2750 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1375 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + Position of Big Table: 0 + Local Work: + Map Reduce Local Work + Bucket Mapjoin Context: + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: hr=0 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 0 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + numFiles 2 + numRows 0 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_2 + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_2 + name: default.srcbucket_mapjoin_part_2 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_2/ds=2008-04-08/hr=0 [b] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 687 Data size: 2750 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Statistics: Num rows: 344 Data size: 1377 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col8 + input vertices: + 1 Map 3 + Position of Big Table: 0 + Statistics: Num rows: 378 Data size: 1514 Basic stats: COMPLETE Column stats: NONE + BucketMapJoin: true + Select Operator + expressions: _col0 (type: int), _col8 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 378 Data size: 1514 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 378 Data size: 1514 Basic stats: COMPLETE Column stats: NONE + tag: -1 + auto parallelism: false + Local Work: + Map Reduce Local Work + Bucket Mapjoin Context: + Alias Bucket File Name Mapping: +#### A masked pattern was here #### + Alias Bucket Output File Name Mapping: +#### A masked pattern was here #### + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: hr=0 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 0 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_1 + numFiles 2 + numRows 0 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 0 + serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.srcbucket_mapjoin_part_1 + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcbucket_mapjoin_part_1 + name: default.srcbucket_mapjoin_part_1 + Truncated Path -> Alias: + /srcbucket_mapjoin_part_1/ds=2008-04-08/hr=0 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 378 Data size: 1514 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 1 Data size: 4 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 int:string + 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 /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' +ORDER BY a.key, b.value LIMIT 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcbucket_mapjoin_part_1 +PREHOOK: Input: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +PREHOOK: Input: default@srcbucket_mapjoin_part_2 +PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +#### A masked pattern was here #### +POSTHOOK: query: SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' +ORDER BY a.key, b.value LIMIT 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcbucket_mapjoin_part_1 +POSTHOOK: Input: default@srcbucket_mapjoin_part_1@ds=2008-04-08/hr=0 +POSTHOOK: Input: default@srcbucket_mapjoin_part_2 +POSTHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08/hr=0 +#### A masked pattern was here #### +0 val_0 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/constprog_partitioner.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/constprog_partitioner.q.out new file mode 100644 index 0000000..602de6e --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/constprog_partitioner.q.out @@ -0,0 +1,191 @@ +PREHOOK: query: EXPLAIN +SELECT src1.key, src1.key + 1, src2.value + FROM src src1 join src src2 ON src1.key = src2.key AND src1.key = 100 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT src1.key, src1.key + 1, src2.value + FROM src src1 join src src2 ON src1.key = src2.key AND src1.key = 100 +POSTHOOK: type: QUERY +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 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 100) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '100' (type: string) + sort order: + + Map-reduce partition columns: '100' (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: src2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 100) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '100' (type: string) + sort order: + + Map-reduce partition columns: '100' (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col6 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '100' (type: string), 101.0 (type: double), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT src1.key, src1.key + 1, src2.value + FROM src src1 join src src2 ON src1.key = src2.key AND src1.key = 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT src1.key, src1.key + 1, src2.value + FROM src src1 join src src2 ON src1.key = src2.key AND src1.key = 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +100 101.0 val_100 +100 101.0 val_100 +100 101.0 val_100 +100 101.0 val_100 +PREHOOK: query: EXPLAIN +SELECT l_partkey, l_suppkey +FROM lineitem li +WHERE li.l_linenumber = 1 AND + li.l_orderkey IN (SELECT l_orderkey FROM lineitem WHERE l_shipmode = 'AIR' AND l_linenumber = li.l_linenumber) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT l_partkey, l_suppkey +FROM lineitem li +WHERE li.l_linenumber = 1 AND + li.l_orderkey IN (SELECT l_orderkey FROM lineitem WHERE l_shipmode = 'AIR' AND l_linenumber = li.l_linenumber) +POSTHOOK: type: QUERY +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 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: li + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((l_orderkey is not null and l_linenumber is not null) and (l_linenumber = 1)) (type: boolean) + Statistics: Num rows: 12 Data size: 1439 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: l_orderkey (type: int), l_linenumber (type: int) + sort order: ++ + Map-reduce partition columns: l_orderkey (type: int), l_linenumber (type: int) + Statistics: Num rows: 12 Data size: 1439 Basic stats: COMPLETE Column stats: NONE + value expressions: l_partkey (type: int), l_suppkey (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: lineitem + Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((((l_shipmode = 'AIR') and l_orderkey is not null) and l_linenumber is not null) and (l_linenumber = 1)) (type: boolean) + Statistics: Num rows: 6 Data size: 719 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 719 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int), _col1 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 719 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 6 Data size: 719 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 l_orderkey (type: int), l_linenumber (type: int) + 1 _col0 (type: int), _col1 (type: int) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 13 Data size: 1582 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1582 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1582 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT l_partkey, l_suppkey +FROM lineitem li +WHERE li.l_linenumber = 1 AND + li.l_orderkey IN (SELECT l_orderkey FROM lineitem WHERE l_shipmode = 'AIR' AND l_linenumber = li.l_linenumber) +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +#### A masked pattern was here #### +POSTHOOK: query: SELECT l_partkey, l_suppkey +FROM lineitem li +WHERE li.l_linenumber = 1 AND + li.l_orderkey IN (SELECT l_orderkey FROM lineitem WHERE l_shipmode = 'AIR' AND l_linenumber = li.l_linenumber) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +#### A masked pattern was here #### +108570 8571 +4297 1798 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/disable_merge_for_bucketing.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/disable_merge_for_bucketing.q.out new file mode 100644 index 0000000..3864c44 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/disable_merge_for_bucketing.q.out @@ -0,0 +1,496 @@ +PREHOOK: query: CREATE TABLE bucket2_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket2_1 +POSTHOOK: query: CREATE TABLE bucket2_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket2_1 +PREHOOK: query: explain extended +insert overwrite table bucket2_1 +select * from src +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +insert overwrite table bucket2_1 +select * from src +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + bucket2_1 + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + + +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 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: UDFToInteger(_col0) (type: int) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col0 (type: string), _col1 (type: string) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + /src [src] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Extract + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true + + Stage: Stage-0 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string +#### A masked pattern was here #### + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + + Stage: Stage-2 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table bucket2_1 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@bucket2_1 +POSTHOOK: query: insert overwrite table bucket2_1 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@bucket2_1 +POSTHOOK: Lineage: bucket2_1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: bucket2_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: explain +select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key +POSTHOOK: type: QUERY +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 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket2_1 +#### A masked pattern was here #### +POSTHOOK: query: select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket2_1 +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 +8 val_8 +10 val_10 +12 val_12 +12 val_12 +18 val_18 +18 val_18 +20 val_20 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +28 val_28 +30 val_30 +34 val_34 +42 val_42 +42 val_42 +44 val_44 +54 val_54 +58 val_58 +58 val_58 +64 val_64 +66 val_66 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +78 val_78 +80 val_80 +82 val_82 +84 val_84 +84 val_84 +86 val_86 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +96 val_96 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +104 val_104 +104 val_104 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +120 val_120 +120 val_120 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +134 val_134 +134 val_134 +136 val_136 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +146 val_146 +146 val_146 +150 val_150 +152 val_152 +152 val_152 +156 val_156 +158 val_158 +160 val_160 +162 val_162 +164 val_164 +164 val_164 +166 val_166 +168 val_168 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +176 val_176 +176 val_176 +178 val_178 +180 val_180 +186 val_186 +190 val_190 +192 val_192 +194 val_194 +196 val_196 +200 val_200 +200 val_200 +202 val_202 +208 val_208 +208 val_208 +208 val_208 +214 val_214 +216 val_216 +216 val_216 +218 val_218 +222 val_222 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +238 val_238 +238 val_238 +242 val_242 +242 val_242 +244 val_244 +248 val_248 +252 val_252 +256 val_256 +256 val_256 +258 val_258 +260 val_260 +262 val_262 +266 val_266 +272 val_272 +272 val_272 +274 val_274 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +282 val_282 +282 val_282 +284 val_284 +286 val_286 +288 val_288 +288 val_288 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +306 val_306 +308 val_308 +310 val_310 +316 val_316 +316 val_316 +316 val_316 +318 val_318 +318 val_318 +318 val_318 +322 val_322 +322 val_322 +332 val_332 +336 val_336 +338 val_338 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +366 val_366 +368 val_368 +374 val_374 +378 val_378 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +392 val_392 +394 val_394 +396 val_396 +396 val_396 +396 val_396 +400 val_400 +402 val_402 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +414 val_414 +414 val_414 +418 val_418 +424 val_424 +424 val_424 +430 val_430 +430 val_430 +430 val_430 +432 val_432 +436 val_436 +438 val_438 +438 val_438 +438 val_438 +444 val_444 +446 val_446 +448 val_448 +452 val_452 +454 val_454 +454 val_454 +454 val_454 +458 val_458 +458 val_458 +460 val_460 +462 val_462 +462 val_462 +466 val_466 +466 val_466 +466 val_466 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +470 val_470 +472 val_472 +478 val_478 +478 val_478 +480 val_480 +480 val_480 +480 val_480 +482 val_482 +484 val_484 +490 val_490 +492 val_492 +492 val_492 +494 val_494 +496 val_496 +498 val_498 +498 val_498 +498 val_498 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/empty_dir_in_table.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/empty_dir_in_table.q.out new file mode 100644 index 0000000..e804de3 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/empty_dir_in_table.q.out @@ -0,0 +1,46 @@ +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@roottable +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@roottable +PREHOOK: query: select count(*) from roottable +PREHOOK: type: QUERY +PREHOOK: Input: default@roottable +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from roottable +POSTHOOK: type: QUERY +POSTHOOK: Input: default@roottable +#### A masked pattern was here #### +0 +PREHOOK: query: insert into table roottable select key from src where (key < 20) order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@roottable +POSTHOOK: query: insert into table roottable select key from src where (key < 20) order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@roottable +POSTHOOK: Lineage: roottable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: select count(*) from roottable +PREHOOK: type: QUERY +PREHOOK: Input: default@roottable +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from roottable +POSTHOOK: type: QUERY +POSTHOOK: Input: default@roottable +#### A masked pattern was here #### +20 +PREHOOK: query: select count(*) from roottable +PREHOOK: type: QUERY +PREHOOK: Input: default@roottable +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from roottable +POSTHOOK: type: QUERY +POSTHOOK: Input: default@roottable +#### A masked pattern was here #### +20 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/external_table_with_space_in_location_path.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/external_table_with_space_in_location_path.q.out new file mode 100644 index 0000000..831e339 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/external_table_with_space_in_location_path.q.out @@ -0,0 +1,88 @@ +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@spacetest +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@spacetest +PREHOOK: query: SELECT * FROM spacetest +PREHOOK: type: QUERY +PREHOOK: Input: default@spacetest +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM spacetest +POSTHOOK: type: QUERY +POSTHOOK: Input: default@spacetest +#### A masked pattern was here #### +12 jason +13 steven +15 joe +PREHOOK: query: SELECT count(*) FROM spacetest +PREHOOK: type: QUERY +PREHOOK: Input: default@spacetest +#### A masked pattern was here #### +POSTHOOK: query: SELECT count(*) FROM spacetest +POSTHOOK: type: QUERY +POSTHOOK: Input: default@spacetest +#### A masked pattern was here #### +3 +PREHOOK: query: DROP TABLE spacetest +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@spacetest +PREHOOK: Output: default@spacetest +POSTHOOK: query: DROP TABLE spacetest +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@spacetest +POSTHOOK: Output: default@spacetest +PREHOOK: query: CREATE EXTERNAL TABLE spacetestpartition (id int, message string) PARTITIONED BY (day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@spacetestpartition +POSTHOOK: query: CREATE EXTERNAL TABLE spacetestpartition (id int, message string) PARTITIONED BY (day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@spacetestpartition +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@spacetestpartition +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@spacetestpartition +POSTHOOK: Output: default@spacetestpartition@day=10 +PREHOOK: query: SELECT * FROM spacetestpartition +PREHOOK: type: QUERY +PREHOOK: Input: default@spacetestpartition +PREHOOK: Input: default@spacetestpartition@day=10 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM spacetestpartition +POSTHOOK: type: QUERY +POSTHOOK: Input: default@spacetestpartition +POSTHOOK: Input: default@spacetestpartition@day=10 +#### A masked pattern was here #### +12 jason 10 +13 steven 10 +15 joe 10 +PREHOOK: query: SELECT count(*) FROM spacetestpartition +PREHOOK: type: QUERY +PREHOOK: Input: default@spacetestpartition +PREHOOK: Input: default@spacetestpartition@day=10 +#### A masked pattern was here #### +POSTHOOK: query: SELECT count(*) FROM spacetestpartition +POSTHOOK: type: QUERY +POSTHOOK: Input: default@spacetestpartition +POSTHOOK: Input: default@spacetestpartition@day=10 +#### A masked pattern was here #### +3 +PREHOOK: query: DROP TABLE spacetestpartition +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@spacetestpartition +PREHOOK: Output: default@spacetestpartition +POSTHOOK: query: DROP TABLE spacetestpartition +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@spacetestpartition +POSTHOOK: Output: default@spacetestpartition +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/file_with_header_footer.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/file_with_header_footer.q.out new file mode 100644 index 0000000..ca3dadb --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/file_with_header_footer.q.out @@ -0,0 +1,184 @@ +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@header_footer_table_1 +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@header_footer_table_1 +PREHOOK: query: SELECT * FROM header_footer_table_1 +PREHOOK: type: QUERY +PREHOOK: Input: default@header_footer_table_1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM header_footer_table_1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@header_footer_table_1 +#### A masked pattern was here #### +steven hive 1 +dave oozie 2 +xifa phd 3 +chuan hadoop 4 +shanyu senior 5 +steven2 hive 11 +dave2 oozie 12 +xifa2 phd 13 +chuan2 hadoop 14 +shanyu2 senior 15 +david3 oozie 22 +PREHOOK: query: SELECT * FROM header_footer_table_1 WHERE id < 50 +PREHOOK: type: QUERY +PREHOOK: Input: default@header_footer_table_1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM header_footer_table_1 WHERE id < 50 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@header_footer_table_1 +#### A masked pattern was here #### +steven hive 1 +dave oozie 2 +xifa phd 3 +chuan hadoop 4 +shanyu senior 5 +steven2 hive 11 +dave2 oozie 12 +xifa2 phd 13 +chuan2 hadoop 14 +shanyu2 senior 15 +david3 oozie 22 +PREHOOK: query: CREATE EXTERNAL TABLE header_footer_table_2 (name string, message string, id int) PARTITIONED BY (year int, month int, day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2") +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@header_footer_table_2 +POSTHOOK: query: CREATE EXTERNAL TABLE header_footer_table_2 (name string, message string, id int) PARTITIONED BY (year int, month int, day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2") +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@header_footer_table_2 +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@header_footer_table_2 +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@header_footer_table_2 +POSTHOOK: Output: default@header_footer_table_2@year=2012/month=1/day=1 +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@header_footer_table_2 +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@header_footer_table_2 +POSTHOOK: Output: default@header_footer_table_2@year=2012/month=1/day=2 +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@header_footer_table_2 +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@header_footer_table_2 +POSTHOOK: Output: default@header_footer_table_2@year=2012/month=1/day=3 +PREHOOK: query: SELECT * FROM header_footer_table_2 +PREHOOK: type: QUERY +PREHOOK: Input: default@header_footer_table_2 +PREHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=1 +PREHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=2 +PREHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM header_footer_table_2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@header_footer_table_2 +POSTHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=1 +POSTHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=2 +POSTHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=3 +#### A masked pattern was here #### +steven hive 1 2012 1 1 +dave oozie 2 2012 1 1 +xifa phd 3 2012 1 1 +chuan hadoop 4 2012 1 1 +shanyu senior 5 2012 1 1 +steven2 hive 11 2012 1 2 +dave2 oozie 12 2012 1 2 +xifa2 phd 13 2012 1 2 +chuan2 hadoop 14 2012 1 2 +shanyu2 senior 15 2012 1 2 +david3 oozie 22 2012 1 3 +PREHOOK: query: SELECT * FROM header_footer_table_2 WHERE id < 50 +PREHOOK: type: QUERY +PREHOOK: Input: default@header_footer_table_2 +PREHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=1 +PREHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=2 +PREHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM header_footer_table_2 WHERE id < 50 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@header_footer_table_2 +POSTHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=1 +POSTHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=2 +POSTHOOK: Input: default@header_footer_table_2@year=2012/month=1/day=3 +#### A masked pattern was here #### +steven hive 1 2012 1 1 +dave oozie 2 2012 1 1 +xifa phd 3 2012 1 1 +chuan hadoop 4 2012 1 1 +shanyu senior 5 2012 1 1 +steven2 hive 11 2012 1 2 +dave2 oozie 12 2012 1 2 +xifa2 phd 13 2012 1 2 +chuan2 hadoop 14 2012 1 2 +shanyu2 senior 15 2012 1 2 +david3 oozie 22 2012 1 3 +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@emptytable +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@emptytable +PREHOOK: query: SELECT * FROM emptytable +PREHOOK: type: QUERY +PREHOOK: Input: default@emptytable +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM emptytable +POSTHOOK: type: QUERY +POSTHOOK: Input: default@emptytable +#### A masked pattern was here #### +PREHOOK: query: SELECT * FROM emptytable WHERE id < 50 +PREHOOK: type: QUERY +PREHOOK: Input: default@emptytable +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM emptytable WHERE id < 50 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@emptytable +#### A masked pattern was here #### +PREHOOK: query: DROP TABLE header_footer_table_1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@header_footer_table_1 +PREHOOK: Output: default@header_footer_table_1 +POSTHOOK: query: DROP TABLE header_footer_table_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@header_footer_table_1 +POSTHOOK: Output: default@header_footer_table_1 +PREHOOK: query: DROP TABLE header_footer_table_2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@header_footer_table_2 +PREHOOK: Output: default@header_footer_table_2 +POSTHOOK: query: DROP TABLE header_footer_table_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@header_footer_table_2 +POSTHOOK: Output: default@header_footer_table_2 +PREHOOK: query: DROP TABLE emptytable +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@emptytable +PREHOOK: Output: default@emptytable +POSTHOOK: query: DROP TABLE emptytable +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@emptytable +POSTHOOK: Output: default@emptytable +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/groupby1.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/groupby1.q.out new file mode 100644 index 0000000..f661a62 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/groupby1.q.out @@ -0,0 +1,421 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest_g1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest_g1 +PREHOOK: query: EXPLAIN +FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key +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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: rand() (type: double) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: substr(value, 5) (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: partial1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: double) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col1 (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_g1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_g1 + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dest_g1 +POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dest_g1 +POSTHOOK: Lineage: dest_g1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest_g1.value EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: SELECT dest_g1.* FROM dest_g1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest_g1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT dest_g1.* FROM dest_g1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest_g1 +#### A masked pattern was here #### +0 0.0 +10 10.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +11 11.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +12 24.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +15 30.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +17 17.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +18 36.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +19 19.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +2 2.0 +20 20.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +24 48.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +26 52.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +27 27.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +28 28.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +30 30.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +33 33.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +34 34.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +35 105.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +37 74.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +4 4.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +41 41.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +42 84.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +43 43.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +44 44.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +47 47.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 +5 15.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +8 8.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +9 9.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/groupby2.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/groupby2.q.out new file mode 100644 index 0000000..f6be571 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/groupby2.q.out @@ -0,0 +1,111 @@ +PREHOOK: query: CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest_g2 +POSTHOOK: query: CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest_g2 +PREHOOK: query: EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) +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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: substr(key, 1, 1) (type: string), substr(value, 5) (type: string) + sort order: ++ + Map-reduce partition columns: substr(key, 1, 1) (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0), sum(KEY._col1:0._col0) + keys: KEY._col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int), concat(_col0, _col2) (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_g2 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_g2 + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: FROM src +INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dest_g2 +POSTHOOK: query: FROM src +INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dest_g2 +POSTHOOK: Lineage: dest_g2.c1 EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest_g2.c2 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest_g2.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: -- SORT_QUERY_RESULTS + +SELECT dest_g2.* FROM dest_g2 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest_g2 +#### A masked pattern was here #### +POSTHOOK: query: -- SORT_QUERY_RESULTS + +SELECT dest_g2.* FROM dest_g2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest_g2 +#### A masked pattern was here #### +0 1 00.0 +1 71 116414.0 +2 69 225571.0 +3 62 332004.0 +4 74 452763.0 +5 6 5397.0 +6 5 6398.0 +7 6 7735.0 +8 8 8762.0 +9 7 91047.0 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/import_exported_table.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/import_exported_table.q.out new file mode 100644 index 0000000..d2f098d --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/import_exported_table.q.out @@ -0,0 +1,25 @@ +#### A masked pattern was here #### +PREHOOK: type: IMPORT +#### A masked pattern was here #### +POSTHOOK: type: IMPORT +#### A masked pattern was here #### +POSTHOOK: Output: default@j1_41 +PREHOOK: query: DESCRIBE j1_41 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@j1_41 +POSTHOOK: query: DESCRIBE j1_41 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@j1_41 +a string +b int +PREHOOK: query: SELECT * from j1_41 +PREHOOK: type: QUERY +PREHOOK: Input: default@j1_41 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * from j1_41 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@j1_41 +#### A masked pattern was here #### +johndee 1 +burks 2 +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/index_bitmap3.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/index_bitmap3.q.out new file mode 100644 index 0000000..c4c25e1 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/index_bitmap3.q.out @@ -0,0 +1,268 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +POSTHOOK: query: -- SORT_QUERY_RESULTS + +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + +PREHOOK: query: EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +POSTHOOK: query: EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + +PREHOOK: query: CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@src +POSTHOOK: query: CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src1_index__ +PREHOOK: query: CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@src +POSTHOOK: query: CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src2_index__ +PREHOOK: query: ALTER INDEX src1_index ON src REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@src +PREHOOK: Output: default@default__src_src1_index__ +POSTHOOK: query: ALTER INDEX src1_index ON src REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src1_index__ +POSTHOOK: Lineage: default__src_src1_index__._bitmaps EXPRESSION [(src)src.FieldSchema(name:ROW__OFFSET__INSIDE__BLOCK, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src1_index__._bucketname SIMPLE [(src)src.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__src_src1_index__._offset SIMPLE [(src)src.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src1_index__.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: ALTER INDEX src2_index ON src REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@src +PREHOOK: Output: default@default__src_src2_index__ +POSTHOOK: query: ALTER INDEX src2_index ON src REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src2_index__ +POSTHOOK: Lineage: default__src_src2_index__._bitmaps EXPRESSION [(src)src.FieldSchema(name:ROW__OFFSET__INSIDE__BLOCK, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src2_index__._bucketname SIMPLE [(src)src.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__src_src2_index__._offset SIMPLE [(src)src.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src2_index__.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: SELECT * FROM default__src_src1_index__ +PREHOOK: type: QUERY +PREHOOK: Input: default@default__src_src1_index__ +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM default__src_src1_index__ +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__src_src1_index__ +#### A masked pattern was here #### +PREHOOK: query: SELECT * FROM default__src_src2_index__ +PREHOOK: type: QUERY +PREHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM default__src_src2_index__ +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +PREHOOK: query: EXPLAIN +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +POSTHOOK: type: QUERY +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default__src_src1_index__ + Statistics: Num rows: 500 Data size: 46311 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((key = 0) and _bucketname is not null) and _offset is not null) (type: boolean) + Statistics: Num rows: 63 Data size: 5835 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _bucketname (type: string), _offset (type: bigint), _bitmaps (type: array) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 63 Data size: 5835 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint) + Statistics: Num rows: 63 Data size: 5835 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: array) + Map 4 + Map Operator Tree: + TableScan + alias: default__src_src2_index__ + Statistics: Num rows: 500 Data size: 48311 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((value = 'val_0') and _bucketname is not null) and _offset is not null) (type: boolean) + Statistics: Num rows: 63 Data size: 6087 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _bucketname (type: string), _offset (type: bigint), _bitmaps (type: array) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 63 Data size: 6087 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint) + Statistics: Num rows: 63 Data size: 6087 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: array) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: bigint) + 1 _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col5 + Statistics: Num rows: 69 Data size: 6418 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(_col2,_col5))) (type: boolean) + Statistics: Num rows: 35 Data size: 3255 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 35 Data size: 3255 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: collect_set(_col1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 35 Data size: 3255 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: 35 Data size: 3255 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: array) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: collect_set(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 17 Data size: 1581 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: array) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 17 Data size: 1581 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 17 Data size: 1581 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +#### A masked pattern was here #### +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +PREHOOK: type: QUERY +PREHOOK: Input: default@default__src_src1_index__ +PREHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__src_src1_index__ +POSTHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +PREHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +PREHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +PREHOOK: query: DROP INDEX src1_index ON src +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@src +POSTHOOK: query: DROP INDEX src1_index ON src +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@src +PREHOOK: query: DROP INDEX src2_index ON src +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@src +POSTHOOK: query: DROP INDEX src2_index ON src +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/index_bitmap_auto.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/index_bitmap_auto.q.out new file mode 100644 index 0000000..60c5683 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/index_bitmap_auto.q.out @@ -0,0 +1,287 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- try the query without indexing, with manual indexing, and with automatic indexing +-- without indexing +SELECT key, value FROM src WHERE key=0 AND value = "val_0" +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- try the query without indexing, with manual indexing, and with automatic indexing +-- without indexing +SELECT key, value FROM src WHERE key=0 AND value = "val_0" +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +PREHOOK: query: -- create indices +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +POSTHOOK: query: -- create indices +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + +PREHOOK: query: EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +POSTHOOK: query: EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + +PREHOOK: query: CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@src +POSTHOOK: query: CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src1_index__ +PREHOOK: query: CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@src +POSTHOOK: query: CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src2_index__ +PREHOOK: query: ALTER INDEX src1_index ON src REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@src +PREHOOK: Output: default@default__src_src1_index__ +POSTHOOK: query: ALTER INDEX src1_index ON src REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src1_index__ +POSTHOOK: Lineage: default__src_src1_index__._bitmaps EXPRESSION [(src)src.FieldSchema(name:ROW__OFFSET__INSIDE__BLOCK, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src1_index__._bucketname SIMPLE [(src)src.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__src_src1_index__._offset SIMPLE [(src)src.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src1_index__.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: ALTER INDEX src2_index ON src REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@src +PREHOOK: Output: default@default__src_src2_index__ +POSTHOOK: query: ALTER INDEX src2_index ON src REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src2_index__ +POSTHOOK: Lineage: default__src_src2_index__._bitmaps EXPRESSION [(src)src.FieldSchema(name:ROW__OFFSET__INSIDE__BLOCK, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src2_index__._bucketname SIMPLE [(src)src.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__src_src2_index__._offset SIMPLE [(src)src.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src2_index__.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: SELECT * FROM default__src_src1_index__ +PREHOOK: type: QUERY +PREHOOK: Input: default@default__src_src1_index__ +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM default__src_src1_index__ +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__src_src1_index__ +#### A masked pattern was here #### +PREHOOK: query: SELECT * FROM default__src_src2_index__ +PREHOOK: type: QUERY +PREHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM default__src_src2_index__ +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +PREHOOK: query: -- manual indexing +EXPLAIN +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +PREHOOK: type: QUERY +POSTHOOK: query: -- manual indexing +EXPLAIN +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +POSTHOOK: type: QUERY +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default__src_src1_index__ + Statistics: Num rows: 500 Data size: 46311 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((key = 0) and _bucketname is not null) and _offset is not null) (type: boolean) + Statistics: Num rows: 63 Data size: 5835 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _bucketname (type: string), _offset (type: bigint), _bitmaps (type: array) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 63 Data size: 5835 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint) + Statistics: Num rows: 63 Data size: 5835 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: array) + Map 4 + Map Operator Tree: + TableScan + alias: default__src_src2_index__ + Statistics: Num rows: 500 Data size: 48311 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((value = 'val_0') and _bucketname is not null) and _offset is not null) (type: boolean) + Statistics: Num rows: 63 Data size: 6087 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _bucketname (type: string), _offset (type: bigint), _bitmaps (type: array) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 63 Data size: 6087 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint) + Statistics: Num rows: 63 Data size: 6087 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: array) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: bigint) + 1 _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col5 + Statistics: Num rows: 69 Data size: 6418 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (not EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(_col2,_col5))) (type: boolean) + Statistics: Num rows: 35 Data size: 3255 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 35 Data size: 3255 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: collect_set(_col1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 35 Data size: 3255 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: 35 Data size: 3255 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: array) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: collect_set(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 17 Data size: 1581 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: array) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 17 Data size: 1581 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 17 Data size: 1581 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +#### A masked pattern was here #### +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +PREHOOK: type: QUERY +PREHOOK: Input: default@default__src_src1_index__ +PREHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` +FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__src_src1_index__ +POSTHOOK: Input: default@default__src_src2_index__ +#### A masked pattern was here #### +PREHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +PREHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT key, value FROM src WHERE key=0 AND value = "val_0" +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +PREHOOK: query: DROP INDEX src1_index ON src +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@src +POSTHOOK: query: DROP INDEX src1_index ON src +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@src +PREHOOK: query: DROP INDEX src2_index ON src +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@src +POSTHOOK: query: DROP INDEX src2_index ON src +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_bucketed_table.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_bucketed_table.q.out new file mode 100644 index 0000000..2852ae9 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_bucketed_table.q.out @@ -0,0 +1,123 @@ +PREHOOK: query: -- Test writing to a bucketed table, the output should be bucketed by the bucketing key into the +-- a number of files equal to the number of buckets +CREATE TABLE test_table_bucketed (key STRING, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (value) SORTED BY (value) INTO 3 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table_bucketed +POSTHOOK: query: -- Test writing to a bucketed table, the output should be bucketed by the bucketing key into the +-- a number of files equal to the number of buckets +CREATE TABLE test_table_bucketed (key STRING, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (value) SORTED BY (value) INTO 3 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table_bucketed +PREHOOK: query: -- Despite the fact that normally inferring would say this table is bucketed and sorted on key, +-- this should be bucketed and sorted by value into 3 buckets +INSERT OVERWRITE TABLE test_table_bucketed PARTITION (part = '1') +SELECT key, count(1) FROM src GROUP BY KEY +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table_bucketed@part=1 +POSTHOOK: query: -- Despite the fact that normally inferring would say this table is bucketed and sorted on key, +-- this should be bucketed and sorted by value into 3 buckets +INSERT OVERWRITE TABLE test_table_bucketed PARTITION (part = '1') +SELECT key, count(1) FROM src GROUP BY KEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table_bucketed@part=1 +POSTHOOK: Lineage: test_table_bucketed PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table_bucketed PARTITION(part=1).value EXPRESSION [(src)src.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table_bucketed PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table_bucketed +POSTHOOK: query: DESCRIBE FORMATTED test_table_bucketed PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table_bucketed +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table_bucketed +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 3 + numRows 309 + rawDataSize 1482 + totalSize 1791 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: 3 +Bucket Columns: [value] +Sort Columns: [Order(col:value, order:1)] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- If the count(*) from sampling the buckets matches the count(*) from each file, the table is +-- bucketed +SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 1 OUT OF 3) WHERE part = '1' +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table_bucketed +PREHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +POSTHOOK: query: -- If the count(*) from sampling the buckets matches the count(*) from each file, the table is +-- bucketed +SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 1 OUT OF 3) WHERE part = '1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table_bucketed +POSTHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +31 +PREHOOK: query: SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 2 OUT OF 3) WHERE part = '1' +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table_bucketed +PREHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 2 OUT OF 3) WHERE part = '1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table_bucketed +POSTHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +179 +PREHOOK: query: SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 3 OUT OF 3) WHERE part = '1' +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table_bucketed +PREHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 3 OUT OF 3) WHERE part = '1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table_bucketed +POSTHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +99 +PREHOOK: query: SELECT cnt FROM (SELECT INPUT__FILE__NAME, COUNT(*) cnt FROM test_table_bucketed WHERE part = '1' +GROUP BY INPUT__FILE__NAME ORDER BY INPUT__FILE__NAME ASC LIMIT 3) a +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table_bucketed +PREHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT cnt FROM (SELECT INPUT__FILE__NAME, COUNT(*) cnt FROM test_table_bucketed WHERE part = '1' +GROUP BY INPUT__FILE__NAME ORDER BY INPUT__FILE__NAME ASC LIMIT 3) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table_bucketed +POSTHOOK: Input: default@test_table_bucketed@part=1 +#### A masked pattern was here #### +31 +179 +99 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_dyn_part.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_dyn_part.q.out new file mode 100644 index 0000000..272503d --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_dyn_part.q.out @@ -0,0 +1,647 @@ +PREHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where dynamic partitioning is used. + +CREATE TABLE test_table LIKE srcpart +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table +POSTHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where dynamic partitioning is used. + +CREATE TABLE test_table LIKE srcpart +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table +PREHOOK: query: ALTER TABLE test_table SET FILEFORMAT RCFILE +PREHOOK: type: ALTERTABLE_FILEFORMAT +PREHOOK: Input: default@test_table +PREHOOK: Output: default@test_table +POSTHOOK: query: ALTER TABLE test_table SET FILEFORMAT RCFILE +POSTHOOK: type: ALTERTABLE_FILEFORMAT +POSTHOOK: Input: default@test_table +POSTHOOK: Output: default@test_table +PREHOOK: query: -- Simple case, this should not be bucketed or sorted + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, value, ds, hr FROM srcpart +WHERE ds = '2008-04-08' +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: Output: default@test_table +POSTHOOK: query: -- Simple case, this should not be bucketed or sorted + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, value, ds, hr FROM srcpart +WHERE ds = '2008-04-08' +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: Output: default@test_table@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@test_table@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 500 + rawDataSize 4812 + totalSize 5293 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 12] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 500 + rawDataSize 4812 + totalSize 5293 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- This should not be bucketed or sorted since the partition keys are in the set of bucketed +-- and sorted columns for the output + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, COUNT(*), ds, hr FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key, ds, hr +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: Output: default@test_table +POSTHOOK: query: -- This should not be bucketed or sorted since the partition keys are in the set of bucketed +-- and sorted columns for the output + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, COUNT(*), ds, hr FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key, ds, hr +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: Output: default@test_table@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@test_table@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart)srcpart.null, ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).value EXPRESSION [(srcpart)srcpart.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 309 + rawDataSize 1173 + totalSize 1479 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 12] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 309 + rawDataSize 1173 + totalSize 1479 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Both partitions should be bucketed and sorted by key + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, value, '2008-04-08', IF (key % 2 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a +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: Output: default@test_table +POSTHOOK: query: -- Both partitions should be bucketed and sorted by key + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, value, '2008-04-08', IF (key % 2 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a +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: Output: default@test_table@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@test_table@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart)srcpart.null, ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).value EXPRESSION [(srcpart)srcpart.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 155 + rawDataSize 586 + totalSize 835 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 12] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 154 + rawDataSize 591 + totalSize 821 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: CREATE TABLE srcpart_merge_dp LIKE srcpart +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_merge_dp +POSTHOOK: query: CREATE TABLE srcpart_merge_dp LIKE srcpart +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_merge_dp +PREHOOK: query: CREATE TABLE srcpart_merge_dp_rc LIKE srcpart +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_merge_dp_rc +POSTHOOK: query: CREATE TABLE srcpart_merge_dp_rc LIKE srcpart +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_merge_dp_rc +PREHOOK: query: ALTER TABLE srcpart_merge_dp_rc SET FILEFORMAT RCFILE +PREHOOK: type: ALTERTABLE_FILEFORMAT +PREHOOK: Input: default@srcpart_merge_dp_rc +PREHOOK: Output: default@srcpart_merge_dp_rc +POSTHOOK: query: ALTER TABLE srcpart_merge_dp_rc SET FILEFORMAT RCFILE +POSTHOOK: type: ALTERTABLE_FILEFORMAT +POSTHOOK: Input: default@srcpart_merge_dp_rc +POSTHOOK: Output: default@srcpart_merge_dp_rc +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcpart_merge_dp +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcpart_merge_dp +POSTHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@srcpart_merge_dp +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@srcpart_merge_dp +POSTHOOK: Output: default@srcpart_merge_dp@ds=2008-04-08/hr=12 +PREHOOK: query: INSERT OVERWRITE TABLE srcpart_merge_dp_rc PARTITION (ds = '2008-04-08', hr) +SELECT key, value, hr FROM srcpart_merge_dp WHERE ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart_merge_dp +PREHOOK: Input: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart_merge_dp@ds=2008-04-08/hr=12 +PREHOOK: Output: default@srcpart_merge_dp_rc@ds=2008-04-08 +POSTHOOK: query: INSERT OVERWRITE TABLE srcpart_merge_dp_rc PARTITION (ds = '2008-04-08', hr) +SELECT key, value, hr FROM srcpart_merge_dp WHERE ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart_merge_dp +POSTHOOK: Input: default@srcpart_merge_dp@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart_merge_dp@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@srcpart_merge_dp_rc@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_merge_dp_rc@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: srcpart_merge_dp_rc PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart_merge_dp)srcpart_merge_dp.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_merge_dp_rc PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart_merge_dp)srcpart_merge_dp.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_merge_dp_rc PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart_merge_dp)srcpart_merge_dp.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_merge_dp_rc PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart_merge_dp)srcpart_merge_dp.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- Tests dynamic partitions where bucketing/sorting can be inferred, but some partitions are +-- merged and some are moved. Currently neither should be bucketed or sorted, in the future, +-- (ds='2008-04-08', hr='12') may be bucketed and sorted, (ds='2008-04-08', hr='11') should +-- definitely not be. + +EXPLAIN +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key, value, IF (key % 100 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a +PREHOOK: type: QUERY +POSTHOOK: query: -- Tests dynamic partitions where bucketing/sorting can be inferred, but some partitions are +-- merged and some are moved. Currently neither should be bucketed or sorted, in the future, +-- (ds='2008-04-08', hr='12') may be bucketed and sorted, (ds='2008-04-08', hr='11') should +-- definitely not be. + +EXPLAIN +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key, value, IF (key % 100 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a +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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 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: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToString(_col1) (type: string), if(((_col0 % 100) = 0), '11', '12') (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.test_table + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2008-04-08 + hr + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.test_table + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key, value, IF (key % 100 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a +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: Output: default@test_table@ds=2008-04-08 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key, value, IF (key % 100 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a +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: Output: default@test_table@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@test_table@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=11).value EXPRESSION [(srcpart)srcpart.null, ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=12).value EXPRESSION [(srcpart)srcpart.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 4 + rawDataSize 14 + totalSize 171 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string default +value string default + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 12] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 305 + rawDataSize 1163 + totalSize 1484 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_map_operators.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_map_operators.q.out new file mode 100644 index 0000000..171e96b --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_map_operators.q.out @@ -0,0 +1,592 @@ +PREHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, in particular, this tests +-- that operators in the mapper have no effect + +CREATE TABLE test_table1 (key STRING, value STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table1 +POSTHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, in particular, this tests +-- that operators in the mapper have no effect + +CREATE TABLE test_table1 (key STRING, value STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table1 +PREHOOK: query: CREATE TABLE test_table2 (key STRING, value STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table2 +POSTHOOK: query: CREATE TABLE test_table2 (key STRING, value STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table2 +PREHOOK: query: INSERT OVERWRITE TABLE test_table1 SELECT key, value FROM src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table1 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table1 SELECT key, value FROM src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table1 +POSTHOOK: Lineage: test_table1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table2 SELECT key, value FROM src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table2 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table2 SELECT key, value FROM src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table2 +POSTHOOK: Lineage: test_table2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: CREATE TABLE test_table_out (key STRING, value STRING) PARTITIONED BY (part STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table_out +POSTHOOK: query: CREATE TABLE test_table_out (key STRING, value STRING) PARTITIONED BY (part STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table_out +PREHOOK: query: -- Test map group by doesn't affect inference, should not be bucketed or sorted +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, count(*) FROM test_table1 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: -- Test map group by doesn't affect inference, should not be bucketed or sorted +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, count(*) FROM test_table1 GROUP BY key +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 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: test_table1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-0 + Move Operator + tables: + partition: + part 1 + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, count(*) FROM test_table1 GROUP BY key +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table1 +PREHOOK: Output: default@test_table_out@part=1 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, count(*) FROM test_table1 GROUP BY key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table1 +POSTHOOK: Output: default@test_table_out@part=1 +POSTHOOK: Lineage: test_table_out PARTITION(part=1).key SIMPLE [(test_table1)test_table1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: test_table_out PARTITION(part=1).value EXPRESSION [(test_table1)test_table1.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table_out +POSTHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table_out +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table_out +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 309 + rawDataSize 1482 + totalSize 1791 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test map group by doesn't affect inference, should be bucketed and sorted by value +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT a.key, a.value FROM ( + SELECT key, count(*) AS value FROM test_table1 GROUP BY key +) a JOIN ( + SELECT key, value FROM src +) b +ON (a.value = b.value) +PREHOOK: type: QUERY +POSTHOOK: query: -- Test map group by doesn't affect inference, should be bucketed and sorted by value +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT a.key, a.value FROM ( + SELECT key, count(*) AS value FROM test_table1 GROUP BY key +) a JOIN ( + SELECT key, value FROM src +) b +ON (a.value = b.value) +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 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: test_table1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: key (type: string) + mode: final + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(_col1) is not null (type: boolean) + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(_col1) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col1) (type: double) + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(value) is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(_col1) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col1) (type: double) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col1) (type: double) + 1 UDFToDouble(_col1) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-0 + Move Operator + tables: + partition: + part 1 + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT a.key, a.value FROM ( + SELECT key, cast(count(*) AS STRING) AS value FROM test_table1 GROUP BY key +) a JOIN ( + SELECT key, value FROM src +) b +ON (a.value = b.value) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@test_table1 +PREHOOK: Output: default@test_table_out@part=1 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT a.key, a.value FROM ( + SELECT key, cast(count(*) AS STRING) AS value FROM test_table1 GROUP BY key +) a JOIN ( + SELECT key, value FROM src +) b +ON (a.value = b.value) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@test_table1 +POSTHOOK: Output: default@test_table_out@part=1 +POSTHOOK: Lineage: test_table_out PARTITION(part=1).key SIMPLE [(test_table1)test_table1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: test_table_out PARTITION(part=1).value EXPRESSION [(test_table1)test_table1.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table_out +POSTHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table_out +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table_out +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 0 + rawDataSize 0 + totalSize 0 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test SMB join doesn't affect inference, should not be bucketed or sorted +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Test SMB join doesn't affect inference, should not be bucketed or sorted +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key +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 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0, _col6 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-0 + Move Operator + tables: + partition: + part 1 + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table1 +PREHOOK: Input: default@test_table2 +PREHOOK: Output: default@test_table_out@part=1 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table1 +POSTHOOK: Input: default@test_table2 +POSTHOOK: Output: default@test_table_out@part=1 +POSTHOOK: Lineage: test_table_out PARTITION(part=1).key SIMPLE [(test_table1)a.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: test_table_out PARTITION(part=1).value SIMPLE [(test_table2)b.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table_out +POSTHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table_out +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table_out +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 1028 + rawDataSize 10968 + totalSize 11996 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test SMB join doesn't affect inference, should be bucketed and sorted by key +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ b.value, count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key +GROUP BY b.value +PREHOOK: type: QUERY +POSTHOOK: query: -- Test SMB join doesn't affect inference, should be bucketed and sorted by key +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ b.value, count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key +GROUP BY b.value +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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col6 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: _col6 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 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: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-0 + Move Operator + tables: + partition: + part 1 + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table_out + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ b.value, count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key +GROUP BY b.value +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table1 +PREHOOK: Input: default@test_table2 +PREHOOK: Output: default@test_table_out@part=1 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ b.value, count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key +GROUP BY b.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table1 +POSTHOOK: Input: default@test_table2 +POSTHOOK: Output: default@test_table_out@part=1 +POSTHOOK: Lineage: test_table_out PARTITION(part=1).key SIMPLE [(test_table2)b.FieldSchema(name:value, type:string, comment:null), ] +POSTHOOK: Lineage: test_table_out PARTITION(part=1).value EXPRESSION [(test_table1)a.null, (test_table2)b.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table_out +POSTHOOK: query: DESCRIBE FORMATTED test_table_out PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table_out +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table_out +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 309 + rawDataSize 2728 + totalSize 3037 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_merge.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_merge.q.out new file mode 100644 index 0000000..0f352d7 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_merge.q.out @@ -0,0 +1,126 @@ +PREHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where where merging may or may not be used. + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table +POSTHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where where merging may or may not be used. + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table +PREHOOK: query: -- Tests a reduce task followed by a merge. The output should be neither bucketed nor sorted. +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Tests a reduce task followed by a merge. The output should be neither bucketed nor sorted. +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 1028 + rawDataSize 10968 + totalSize 11996 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Tests a reduce task followed by a move. The output should be bucketed and sorted. +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Tests a reduce task followed by a move. The output should be bucketed and sorted. +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 1028 + rawDataSize 10968 + totalSize 11996 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_num_buckets.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_num_buckets.q.out new file mode 100644 index 0000000..c57530e --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_num_buckets.q.out @@ -0,0 +1,218 @@ +PREHOOK: query: CREATE TABLE test_table (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table +POSTHOOK: query: CREATE TABLE test_table (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table +PREHOOK: query: -- Tests dynamic partitions where bucketing/sorting can be inferred, but not all reducers write +-- all partitions. The subquery produces rows as follows +-- key = 0: +-- 0, , 0 +-- key = 1: +-- 0, , 1 +-- key = 2: +-- 1, , 0 +-- This means that by distributing by the first column into two reducers, and using the third +-- columns as a dynamic partition, the dynamic partition for 0 will get written in both reducers +-- and the partition for 1 will get written in one reducer. So hr=0 should be bucketed by key +-- and hr=1 should not. + +EXPLAIN +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key2, value, cast(hr as int) FROM +(SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 2) as hr +FROM srcpart +WHERE ds = '2008-04-08') a +DISTRIBUTE BY key2 +PREHOOK: type: QUERY +POSTHOOK: query: -- Tests dynamic partitions where bucketing/sorting can be inferred, but not all reducers write +-- all partitions. The subquery produces rows as follows +-- key = 0: +-- 0, , 0 +-- key = 1: +-- 0, , 1 +-- key = 2: +-- 1, , 0 +-- This means that by distributing by the first column into two reducers, and using the third +-- columns as a dynamic partition, the dynamic partition for 0 will get written in both reducers +-- and the partition for 1 will get written in one reducer. So hr=0 should be bucketed by key +-- and hr=1 should not. + +EXPLAIN +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key2, value, cast(hr as int) FROM +(SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 2) as hr +FROM srcpart +WHERE ds = '2008-04-08') a +DISTRIBUTE BY key2 +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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: if(((key % 3) < 2), 0, 1) (type: int), value (type: string), UDFToInteger((key % 2)) (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2008-04-08 + hr + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key2, value, cast(hr as int) FROM +(SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 3 % 2) as hr +FROM srcpart +WHERE ds = '2008-04-08') a +DISTRIBUTE BY key2 +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: Output: default@test_table@ds=2008-04-08 +POSTHOOK: query: INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key2, value, cast(hr as int) FROM +(SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 3 % 2) as hr +FROM srcpart +WHERE ds = '2008-04-08') a +DISTRIBUTE BY key2 +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: Output: default@test_table@ds=2008-04-08/hr=0 +POSTHOOK: Output: default@test_table@ds=2008-04-08/hr=1 +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=0).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=0).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=1).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(ds=2008-04-08,hr=1).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='0') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='0') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key int +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 0] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 2 + numRows 670 + rawDataSize 5888 + totalSize 6558 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key int +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 330 + rawDataSize 2924 + totalSize 3254 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_reducers_power_two.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_reducers_power_two.q.out new file mode 100644 index 0000000..2a25dfe --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/infer_bucket_sort_reducers_power_two.q.out @@ -0,0 +1,348 @@ +PREHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, it also verifies that the +-- number of reducers chosen will be a power of two + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table +POSTHOOK: query: -- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, it also verifies that the +-- number of reducers chosen will be a power of two + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table +PREHOOK: query: -- Test group by, should be bucketed and sorted by group by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Test group by, should be bucketed and sorted by group by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 5 + numRows 309 + rawDataSize 1482 + totalSize 1791 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test join, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Test join, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)a.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 5 + numRows 1028 + rawDataSize 10968 + totalSize 11996 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test join with two keys, should be bucketed and sorted by join keys +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key AND a.value = b.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Test join with two keys, should be bucketed and sorted by join keys +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key AND a.value = b.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)a.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 3 + numRows 1028 + rawDataSize 10968 + totalSize 11996 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test join on three tables on same key, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.key = c.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Test join on three tables on same key, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.key = c.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)a.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 7 + numRows 2654 + rawDataSize 28466 + totalSize 31120 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test join on three tables on different keys, should be bucketed and sorted by latter key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.value = c.value) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Test join on three tables on different keys, should be bucketed and sorted by latter key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.value = c.value) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)a.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 5 + numRows 2654 + rawDataSize 28466 + totalSize 31120 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- Test group by in subquery with another group by outside, should be bucketed and sorted by the +-- key of the outer group by +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT count(1), value FROM (SELECT key, count(1) as value FROM src group by key) a group by value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table@part=1 +POSTHOOK: query: -- Test group by in subquery with another group by outside, should be bucketed and sorted by the +-- key of the outer group by +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT count(1), value FROM (SELECT key, count(1) as value FROM src group by key) a group by value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table@part=1 +POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ] +POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ] +PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test_table +POSTHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '1') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test_table +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +part string + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: test_table +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 3 + numRows 5 + rawDataSize 19 + totalSize 24 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/input16_cc.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/input16_cc.q.out new file mode 100644 index 0000000..9ea3472 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/input16_cc.q.out @@ -0,0 +1,534 @@ +PREHOOK: query: -- TestSerDe is a user defined serde where the default delimiter is Ctrl-B +-- the user is overwriting it with ctrlC + +DROP TABLE INPUT16_CC +PREHOOK: type: DROPTABLE +POSTHOOK: query: -- TestSerDe is a user defined serde where the default delimiter is Ctrl-B +-- the user is overwriting it with ctrlC + +DROP TABLE INPUT16_CC +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE INPUT16_CC(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties ('testserde.default.serialization.format'='\003', 'dummy.prop.not.used'='dummyy.val') STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@INPUT16_CC +POSTHOOK: query: CREATE TABLE INPUT16_CC(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties ('testserde.default.serialization.format'='\003', 'dummy.prop.not.used'='dummyy.val') STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@INPUT16_CC +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@input16_cc +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@input16_cc +PREHOOK: query: SELECT INPUT16_CC.VALUE, INPUT16_CC.KEY FROM INPUT16_CC +PREHOOK: type: QUERY +PREHOOK: Input: default@input16_cc +#### A masked pattern was here #### +POSTHOOK: query: SELECT INPUT16_CC.VALUE, INPUT16_CC.KEY FROM INPUT16_CC +POSTHOOK: type: QUERY +POSTHOOK: Input: default@input16_cc +#### A masked pattern was here #### +val_238 238 +val_86 86 +val_311 311 +val_27 27 +val_165 165 +val_409 409 +val_255 255 +val_278 278 +val_98 98 +val_484 484 +val_265 265 +val_193 193 +val_401 401 +val_150 150 +val_273 273 +val_224 224 +val_369 369 +val_66 66 +val_128 128 +val_213 213 +val_146 146 +val_406 406 +val_429 429 +val_374 374 +val_152 152 +val_469 469 +val_145 145 +val_495 495 +val_37 37 +val_327 327 +val_281 281 +val_277 277 +val_209 209 +val_15 15 +val_82 82 +val_403 403 +val_166 166 +val_417 417 +val_430 430 +val_252 252 +val_292 292 +val_219 219 +val_287 287 +val_153 153 +val_193 193 +val_338 338 +val_446 446 +val_459 459 +val_394 394 +val_237 237 +val_482 482 +val_174 174 +val_413 413 +val_494 494 +val_207 207 +val_199 199 +val_466 466 +val_208 208 +val_174 174 +val_399 399 +val_396 396 +val_247 247 +val_417 417 +val_489 489 +val_162 162 +val_377 377 +val_397 397 +val_309 309 +val_365 365 +val_266 266 +val_439 439 +val_342 342 +val_367 367 +val_325 325 +val_167 167 +val_195 195 +val_475 475 +val_17 17 +val_113 113 +val_155 155 +val_203 203 +val_339 339 +val_0 0 +val_455 455 +val_128 128 +val_311 311 +val_316 316 +val_57 57 +val_302 302 +val_205 205 +val_149 149 +val_438 438 +val_345 345 +val_129 129 +val_170 170 +val_20 20 +val_489 489 +val_157 157 +val_378 378 +val_221 221 +val_92 92 +val_111 111 +val_47 47 +val_72 72 +val_4 4 +val_280 280 +val_35 35 +val_427 427 +val_277 277 +val_208 208 +val_356 356 +val_399 399 +val_169 169 +val_382 382 +val_498 498 +val_125 125 +val_386 386 +val_437 437 +val_469 469 +val_192 192 +val_286 286 +val_187 187 +val_176 176 +val_54 54 +val_459 459 +val_51 51 +val_138 138 +val_103 103 +val_239 239 +val_213 213 +val_216 216 +val_430 430 +val_278 278 +val_176 176 +val_289 289 +val_221 221 +val_65 65 +val_318 318 +val_332 332 +val_311 311 +val_275 275 +val_137 137 +val_241 241 +val_83 83 +val_333 333 +val_180 180 +val_284 284 +val_12 12 +val_230 230 +val_181 181 +val_67 67 +val_260 260 +val_404 404 +val_384 384 +val_489 489 +val_353 353 +val_373 373 +val_272 272 +val_138 138 +val_217 217 +val_84 84 +val_348 348 +val_466 466 +val_58 58 +val_8 8 +val_411 411 +val_230 230 +val_208 208 +val_348 348 +val_24 24 +val_463 463 +val_431 431 +val_179 179 +val_172 172 +val_42 42 +val_129 129 +val_158 158 +val_119 119 +val_496 496 +val_0 0 +val_322 322 +val_197 197 +val_468 468 +val_393 393 +val_454 454 +val_100 100 +val_298 298 +val_199 199 +val_191 191 +val_418 418 +val_96 96 +val_26 26 +val_165 165 +val_327 327 +val_230 230 +val_205 205 +val_120 120 +val_131 131 +val_51 51 +val_404 404 +val_43 43 +val_436 436 +val_156 156 +val_469 469 +val_468 468 +val_308 308 +val_95 95 +val_196 196 +val_288 288 +val_481 481 +val_457 457 +val_98 98 +val_282 282 +val_197 197 +val_187 187 +val_318 318 +val_318 318 +val_409 409 +val_470 470 +val_137 137 +val_369 369 +val_316 316 +val_169 169 +val_413 413 +val_85 85 +val_77 77 +val_0 0 +val_490 490 +val_87 87 +val_364 364 +val_179 179 +val_118 118 +val_134 134 +val_395 395 +val_282 282 +val_138 138 +val_238 238 +val_419 419 +val_15 15 +val_118 118 +val_72 72 +val_90 90 +val_307 307 +val_19 19 +val_435 435 +val_10 10 +val_277 277 +val_273 273 +val_306 306 +val_224 224 +val_309 309 +val_389 389 +val_327 327 +val_242 242 +val_369 369 +val_392 392 +val_272 272 +val_331 331 +val_401 401 +val_242 242 +val_452 452 +val_177 177 +val_226 226 +val_5 5 +val_497 497 +val_402 402 +val_396 396 +val_317 317 +val_395 395 +val_58 58 +val_35 35 +val_336 336 +val_95 95 +val_11 11 +val_168 168 +val_34 34 +val_229 229 +val_233 233 +val_143 143 +val_472 472 +val_322 322 +val_498 498 +val_160 160 +val_195 195 +val_42 42 +val_321 321 +val_430 430 +val_119 119 +val_489 489 +val_458 458 +val_78 78 +val_76 76 +val_41 41 +val_223 223 +val_492 492 +val_149 149 +val_449 449 +val_218 218 +val_228 228 +val_138 138 +val_453 453 +val_30 30 +val_209 209 +val_64 64 +val_468 468 +val_76 76 +val_74 74 +val_342 342 +val_69 69 +val_230 230 +val_33 33 +val_368 368 +val_103 103 +val_296 296 +val_113 113 +val_216 216 +val_367 367 +val_344 344 +val_167 167 +val_274 274 +val_219 219 +val_239 239 +val_485 485 +val_116 116 +val_223 223 +val_256 256 +val_263 263 +val_70 70 +val_487 487 +val_480 480 +val_401 401 +val_288 288 +val_191 191 +val_5 5 +val_244 244 +val_438 438 +val_128 128 +val_467 467 +val_432 432 +val_202 202 +val_316 316 +val_229 229 +val_469 469 +val_463 463 +val_280 280 +val_2 2 +val_35 35 +val_283 283 +val_331 331 +val_235 235 +val_80 80 +val_44 44 +val_193 193 +val_321 321 +val_335 335 +val_104 104 +val_466 466 +val_366 366 +val_175 175 +val_403 403 +val_483 483 +val_53 53 +val_105 105 +val_257 257 +val_406 406 +val_409 409 +val_190 190 +val_406 406 +val_401 401 +val_114 114 +val_258 258 +val_90 90 +val_203 203 +val_262 262 +val_348 348 +val_424 424 +val_12 12 +val_396 396 +val_201 201 +val_217 217 +val_164 164 +val_431 431 +val_454 454 +val_478 478 +val_298 298 +val_125 125 +val_431 431 +val_164 164 +val_424 424 +val_187 187 +val_382 382 +val_5 5 +val_70 70 +val_397 397 +val_480 480 +val_291 291 +val_24 24 +val_351 351 +val_255 255 +val_104 104 +val_70 70 +val_163 163 +val_438 438 +val_119 119 +val_414 414 +val_200 200 +val_491 491 +val_237 237 +val_439 439 +val_360 360 +val_248 248 +val_479 479 +val_305 305 +val_417 417 +val_199 199 +val_444 444 +val_120 120 +val_429 429 +val_169 169 +val_443 443 +val_323 323 +val_325 325 +val_277 277 +val_230 230 +val_478 478 +val_178 178 +val_468 468 +val_310 310 +val_317 317 +val_333 333 +val_493 493 +val_460 460 +val_207 207 +val_249 249 +val_265 265 +val_480 480 +val_83 83 +val_136 136 +val_353 353 +val_172 172 +val_214 214 +val_462 462 +val_233 233 +val_406 406 +val_133 133 +val_175 175 +val_189 189 +val_454 454 +val_375 375 +val_401 401 +val_421 421 +val_407 407 +val_384 384 +val_256 256 +val_26 26 +val_134 134 +val_67 67 +val_384 384 +val_379 379 +val_18 18 +val_462 462 +val_492 492 +val_100 100 +val_298 298 +val_9 9 +val_341 341 +val_498 498 +val_146 146 +val_458 458 +val_362 362 +val_186 186 +val_285 285 +val_348 348 +val_167 167 +val_18 18 +val_273 273 +val_183 183 +val_281 281 +val_344 344 +val_97 97 +val_469 469 +val_315 315 +val_84 84 +val_28 28 +val_37 37 +val_448 448 +val_152 152 +val_348 348 +val_307 307 +val_194 194 +val_414 414 +val_477 477 +val_222 222 +val_126 126 +val_90 90 +val_169 169 +val_403 403 +val_400 400 +val_200 200 +val_97 97 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/join1.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/join1.q.out new file mode 100644 index 0000000..145bbe4 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/join1.q.out @@ -0,0 +1,1143 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest_j1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest_j1 +PREHOOK: query: EXPLAIN +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value +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 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: src2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: value (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0, _col6 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col0) (type: int), _col6 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_j1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_j1 + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dest_j1 +POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dest_j1 +POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest_j1.value SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: SELECT dest_j1.* FROM dest_j1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest_j1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT dest_j1.* FROM dest_j1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest_j1 +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +10 val_10 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +105 val_105 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +150 val_150 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +20 val_20 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +241 val_241 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +266 val_266 +27 val_27 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +28 val_28 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +30 val_30 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +33 val_33 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +34 val_34 +341 val_341 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +351 val_351 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +4 val_4 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +41 val_41 +411 val_411 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +421 val_421 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +43 val_43 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +44 val_44 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +47 val_47 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +77 val_77 +78 val_78 +8 val_8 +80 val_80 +82 val_82 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +9 val_9 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +98 val_98 +98 val_98 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/leftsemijoin_mr.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/leftsemijoin_mr.q.out new file mode 100644 index 0000000..fe63057 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/leftsemijoin_mr.q.out @@ -0,0 +1,98 @@ +PREHOOK: query: CREATE TABLE T1(key INT) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T1 +POSTHOOK: query: CREATE TABLE T1(key INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: CREATE TABLE T2(key INT) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T2 +POSTHOOK: query: CREATE TABLE T2(key INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t2 +PREHOOK: query: -- Run this query using TestMinimrCliDriver + +SELECT * FROM T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: -- Run this query using TestMinimrCliDriver + +SELECT * FROM T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 +1 +PREHOOK: query: SELECT * FROM T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +PREHOOK: query: SELECT T1.key FROM T1 LEFT SEMI JOIN (SELECT key FROM T2 SORT BY key) tmp ON (T1.key=tmp.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT T1.key FROM T1 LEFT SEMI JOIN (SELECT key FROM T2 SORT BY key) tmp ON (T1.key=tmp.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +1 +1 +PREHOOK: query: SELECT T1.key FROM T1 LEFT SEMI JOIN (SELECT key FROM T2 SORT BY key) tmp ON (T1.key=tmp.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT T1.key FROM T1 LEFT SEMI JOIN (SELECT key FROM T2 SORT BY key) tmp ON (T1.key=tmp.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +1 +1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/list_bucket_dml_10.q.java1.7.out ql/src/test/results/clientpositive/miniSparkOnYarn/list_bucket_dml_10.q.java1.7.out new file mode 100644 index 0000000..d765eaf --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/list_bucket_dml_10.q.java1.7.out @@ -0,0 +1,282 @@ +PREHOOK: query: -- run this test case in minimr to ensure it works in cluster +-- JAVA_VERSION_SPECIFIC_OUTPUT + +-- list bucketing DML: static partition. multiple skewed columns. +-- ds=2008-04-08/hr=11/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- 5263 000000_0 +-- 5263 000001_0 +-- ds=2008-04-08/hr=11/key=103/value=val_103: +-- 99 000000_0 +-- 99 000001_0 +-- ds=2008-04-08/hr=11/key=484/value=val_484: +-- 87 000000_0 +-- 87 000001_0 + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key) on ('484','51','103') + stored as DIRECTORIES + STORED AS RCFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@list_bucketing_static_part +POSTHOOK: query: -- run this test case in minimr to ensure it works in cluster +-- JAVA_VERSION_SPECIFIC_OUTPUT + +-- list bucketing DML: static partition. multiple skewed columns. +-- ds=2008-04-08/hr=11/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- 5263 000000_0 +-- 5263 000001_0 +-- ds=2008-04-08/hr=11/key=103/value=val_103: +-- 99 000000_0 +-- 99 000001_0 +-- ds=2008-04-08/hr=11/key=484/value=val_484: +-- 87 000000_0 +-- 87 000001_0 + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key) on ('484','51','103') + stored as DIRECTORIES + STORED AS RCFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@list_bucketing_static_part +PREHOOK: query: -- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src +PREHOOK: type: QUERY +POSTHOOK: query: -- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + list_bucketing_static_part + TOK_PARTSPEC + TOK_PARTVAL + ds + '2008-04-08' + TOK_PARTVAL + hr + '11' + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + value + + +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 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Static Partition Specification: ds=2008-04-08/hr=11/ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.list_bucketing_static_part + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct list_bucketing_static_part { 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.list_bucketing_static_part + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + /src [src] + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2008-04-08 + hr 11 + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count -1 + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.list_bucketing_static_part + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct list_bucketing_static_part { 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.list_bucketing_static_part + + Stage: Stage-2 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +POSTHOOK: query: insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +POSTHOOK: Lineage: list_bucketing_static_part PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: list_bucketing_static_part PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- check DML result +show partitions list_bucketing_static_part +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@list_bucketing_static_part +POSTHOOK: query: -- check DML result +show partitions list_bucketing_static_part +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@list_bucketing_static_part +ds=2008-04-08/hr=11 +PREHOOK: query: desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@list_bucketing_static_part +POSTHOOK: query: desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@list_bucketing_static_part +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: list_bucketing_static_part +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 4 + numRows 500 + rawDataSize 4812 + totalSize 5520 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Stored As SubDirectories: Yes +Skewed Columns: [key] +Skewed Values: [[484], [51], [103]] +#### A masked pattern was here #### +Skewed Value to Truncated Path: {[484]=/list_bucketing_static_part/ds=2008-04-08/hr=11/key=484, [103]=/list_bucketing_static_part/ds=2008-04-08/hr=11/key=103, [51]=/list_bucketing_static_part/ds=2008-04-08/hr=11/key=51} +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/load_fs2.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/load_fs2.q.out new file mode 100644 index 0000000..1846542 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/load_fs2.q.out @@ -0,0 +1,121 @@ +PREHOOK: query: -- HIVE-3300 [jira] LOAD DATA INPATH fails if a hdfs file with same name is added to table +-- 'loader' table is used only for uploading kv1.txt to HDFS (!hdfs -put is not working on minMRDriver) + +create table result (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@result +POSTHOOK: query: -- HIVE-3300 [jira] LOAD DATA INPATH fails if a hdfs file with same name is added to table +-- 'loader' table is used only for uploading kv1.txt to HDFS (!hdfs -put is not working on minMRDriver) + +create table result (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@result +PREHOOK: query: create table loader (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@loader +POSTHOOK: query: create table loader (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@loader +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table loader +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@loader +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table loader +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@loader +PREHOOK: query: load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@result +POSTHOOK: query: load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@result +PREHOOK: query: show table extended like result +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like result +POSTHOOK: type: SHOW_TABLESTATUS +tableName:result +#### A masked pattern was here #### +inputformat:org.apache.hadoop.mapred.TextInputFormat +outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +columns:struct columns { string key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:1 +totalFileSize:5812 +maxFileSize:5812 +minFileSize:5812 +#### A masked pattern was here #### + +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table loader +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@loader +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table loader +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@loader +PREHOOK: query: load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@result +POSTHOOK: query: load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@result +PREHOOK: query: show table extended like result +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like result +POSTHOOK: type: SHOW_TABLESTATUS +tableName:result +#### A masked pattern was here #### +inputformat:org.apache.hadoop.mapred.TextInputFormat +outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +columns:struct columns { string key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:2 +totalFileSize:11624 +maxFileSize:5812 +minFileSize:5812 +#### A masked pattern was here #### + +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table loader +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@loader +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table loader +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@loader +PREHOOK: query: load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@result +POSTHOOK: query: load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@result +PREHOOK: query: show table extended like result +PREHOOK: type: SHOW_TABLESTATUS +POSTHOOK: query: show table extended like result +POSTHOOK: type: SHOW_TABLESTATUS +tableName:result +#### A masked pattern was here #### +inputformat:org.apache.hadoop.mapred.TextInputFormat +outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +columns:struct columns { string key, string value} +partitioned:false +partitionColumns: +totalNumberFiles:3 +totalFileSize:17436 +maxFileSize:5812 +minFileSize:5812 +#### A masked pattern was here #### + diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/load_hdfs_file_with_space_in_the_name.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/load_hdfs_file_with_space_in_the_name.q.out new file mode 100644 index 0000000..d934722 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/load_hdfs_file_with_space_in_the_name.q.out @@ -0,0 +1,25 @@ +PREHOOK: query: CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@load_file_with_space_in_the_name +POSTHOOK: query: CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@load_file_with_space_in_the_name +#### A masked pattern was here #### +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@load_file_with_space_in_the_name +#### A masked pattern was here #### +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@load_file_with_space_in_the_name +#### A masked pattern was here #### +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@load_file_with_space_in_the_name +#### A masked pattern was here #### +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@load_file_with_space_in_the_name +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/parallel_orderby.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/parallel_orderby.q.out new file mode 100644 index 0000000..0194dbb --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/parallel_orderby.q.out @@ -0,0 +1,305 @@ +PREHOOK: query: create table src5 (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src5 +POSTHOOK: query: create table src5 (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src5 +PREHOOK: query: load data local inpath '../../data/files/kv5.txt' into table src5 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@src5 +POSTHOOK: query: load data local inpath '../../data/files/kv5.txt' into table src5 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@src5 +PREHOOK: query: load data local inpath '../../data/files/kv5.txt' into table src5 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@src5 +POSTHOOK: query: load data local inpath '../../data/files/kv5.txt' into table src5 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@src5 +PREHOOK: query: explain +create table total_ordered as select * from src5 order by key, value +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: explain +create table total_ordered as select * from src5 order by key, value +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-0 + Stage-2 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src5 + Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 560 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.total_ordered + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-3 + Create Table Operator: + Create Table + columns: key string, value string + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.total_ordered + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: create table total_ordered as select * from src5 order by key, value +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src5 +PREHOOK: Output: database:default +PREHOOK: Output: default@total_ordered +POSTHOOK: query: create table total_ordered as select * from src5 order by key, value +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src5 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@total_ordered +PREHOOK: query: desc formatted total_ordered +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@total_ordered +POSTHOOK: query: desc formatted total_ordered +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@total_ordered +# col_name data_type comment + +key string +value string + +# 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 1 + numRows 48 + rawDataSize 512 + totalSize 560 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from total_ordered +PREHOOK: type: QUERY +PREHOOK: Input: default@total_ordered +#### A masked pattern was here #### +POSTHOOK: query: select * from total_ordered +POSTHOOK: type: QUERY +POSTHOOK: Input: default@total_ordered +#### A masked pattern was here #### +128 val_128 +128 val_128 +150 val_150 +150 val_150 +165 val_165 +165 val_165 +193 val_193 +193 val_193 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_214 +213 val_214 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_239 +238 val_239 +238 val_240 +238 val_240 +255 val_255 +255 val_255 +265 val_265 +265 val_265 +27 val_27 +27 val_27 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +66 val_66 +66 val_66 +86 val_86 +86 val_86 +98 val_98 +98 val_98 +PREHOOK: query: -- rolling back to single task in case that the number of sample is not enough + +drop table total_ordered +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@total_ordered +PREHOOK: Output: default@total_ordered +POSTHOOK: query: -- rolling back to single task in case that the number of sample is not enough + +drop table total_ordered +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@total_ordered +POSTHOOK: Output: default@total_ordered +PREHOOK: query: create table total_ordered as select * from src5 order by key, value +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src5 +PREHOOK: Output: database:default +PREHOOK: Output: default@total_ordered +POSTHOOK: query: create table total_ordered as select * from src5 order by key, value +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src5 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@total_ordered +PREHOOK: query: desc formatted total_ordered +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@total_ordered +POSTHOOK: query: desc formatted total_ordered +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@total_ordered +# col_name data_type comment + +key string +value string + +# 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 1 + numRows 48 + rawDataSize 512 + totalSize 560 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from total_ordered +PREHOOK: type: QUERY +PREHOOK: Input: default@total_ordered +#### A masked pattern was here #### +POSTHOOK: query: select * from total_ordered +POSTHOOK: type: QUERY +POSTHOOK: Input: default@total_ordered +#### A masked pattern was here #### +128 val_128 +128 val_128 +150 val_150 +150 val_150 +165 val_165 +165 val_165 +193 val_193 +193 val_193 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_214 +213 val_214 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_239 +238 val_239 +238 val_240 +238 val_240 +255 val_255 +255 val_255 +265 val_265 +265 val_265 +27 val_27 +27 val_27 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +66 val_66 +66 val_66 +86 val_86 +86 val_86 +98 val_98 +98 val_98 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx.q.out new file mode 100644 index 0000000..fd75329 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx.q.out @@ -0,0 +1,2730 @@ +PREHOOK: query: DROP TABLE IF EXISTS lineitem_ix +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE IF EXISTS lineitem_ix +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE lineitem_ix (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@lineitem_ix +POSTHOOK: query: CREATE TABLE lineitem_ix (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@lineitem_ix +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem_ix +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@lineitem_ix +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem_ix +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@lineitem_ix +PREHOOK: query: CREATE INDEX lineitem_ix_lshipdate_idx ON TABLE lineitem_ix(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@lineitem_ix +POSTHOOK: query: CREATE INDEX lineitem_ix_lshipdate_idx ON TABLE lineitem_ix(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +PREHOOK: query: ALTER INDEX lineitem_ix_lshipdate_idx ON lineitem_ix REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@lineitem_ix +PREHOOK: Output: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: query: ALTER INDEX lineitem_ix_lshipdate_idx ON lineitem_ix REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__._bucketname SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__._count_of_l_shipdate EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:l_shipdate, type:string, comment:null), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__._offsets EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__.l_shipdate SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:l_shipdate, type:string, comment:null), ] +PREHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +PREHOOK: type: QUERY +POSTHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string) + outputColumnNames: l_shipdate + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_shipdate) + keys: l_shipdate (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 120 Data size: 12099 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: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992-04-27 1 +1992-07-02 1 +1992-07-10 1 +1992-07-21 1 +1993-04-01 1 +1993-04-13 1 +1993-05-14 1 +1993-10-29 2 +1993-11-09 2 +1993-12-04 1 +1993-12-09 2 +1993-12-14 1 +1994-01-12 1 +1994-01-16 1 +1994-01-26 2 +1994-02-02 1 +1994-02-13 1 +1994-02-19 1 +1994-02-21 1 +1994-03-03 1 +1994-03-17 1 +1994-06-03 1 +1994-06-06 1 +1994-07-02 1 +1994-07-19 1 +1994-07-31 1 +1994-08-08 1 +1994-08-17 1 +1994-08-24 1 +1994-09-30 1 +1994-10-03 1 +1994-10-16 1 +1994-10-31 1 +1994-12-01 1 +1994-12-24 1 +1994-12-30 1 +1995-04-20 1 +1995-07-06 1 +1995-07-17 1 +1995-07-21 1 +1995-08-04 1 +1995-08-07 1 +1995-08-14 1 +1995-08-28 1 +1995-10-23 1 +1995-11-08 1 +1995-11-26 1 +1996-01-10 1 +1996-01-15 1 +1996-01-16 1 +1996-01-19 1 +1996-01-22 1 +1996-01-29 1 +1996-01-30 1 +1996-02-01 2 +1996-02-03 1 +1996-02-10 1 +1996-02-11 1 +1996-02-21 1 +1996-03-13 1 +1996-03-21 1 +1996-03-30 1 +1996-04-12 1 +1996-04-21 1 +1996-05-07 1 +1996-09-26 1 +1996-09-29 1 +1996-10-02 1 +1996-10-17 1 +1996-11-04 1 +1996-11-14 1 +1996-12-08 1 +1997-01-25 1 +1997-01-27 1 +1997-01-28 1 +1997-02-20 1 +1997-03-18 1 +1997-04-17 1 +1997-04-19 1 +1998-01-29 1 +1998-02-23 1 +1998-03-05 1 +1998-04-10 1 +1998-04-12 1 +1998-05-23 1 +1998-06-19 1 +1998-06-24 1 +1998-06-26 1 +1998-06-27 1 +1998-07-04 1 +1998-08-11 1 +1998-08-13 1 +1998-10-09 1 +1998-10-23 1 +1998-10-30 1 +PREHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +PREHOOK: type: QUERY +POSTHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: l_shipdate (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 95 Data size: 8960 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: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992-04-27 1 +1992-07-02 1 +1992-07-10 1 +1992-07-21 1 +1993-04-01 1 +1993-04-13 1 +1993-05-14 1 +1993-10-29 2 +1993-11-09 2 +1993-12-04 1 +1993-12-09 2 +1993-12-14 1 +1994-01-12 1 +1994-01-16 1 +1994-01-26 2 +1994-02-02 1 +1994-02-13 1 +1994-02-19 1 +1994-02-21 1 +1994-03-03 1 +1994-03-17 1 +1994-06-03 1 +1994-06-06 1 +1994-07-02 1 +1994-07-19 1 +1994-07-31 1 +1994-08-08 1 +1994-08-17 1 +1994-08-24 1 +1994-09-30 1 +1994-10-03 1 +1994-10-16 1 +1994-10-31 1 +1994-12-01 1 +1994-12-24 1 +1994-12-30 1 +1995-04-20 1 +1995-07-06 1 +1995-07-17 1 +1995-07-21 1 +1995-08-04 1 +1995-08-07 1 +1995-08-14 1 +1995-08-28 1 +1995-10-23 1 +1995-11-08 1 +1995-11-26 1 +1996-01-10 1 +1996-01-15 1 +1996-01-16 1 +1996-01-19 1 +1996-01-22 1 +1996-01-29 1 +1996-01-30 1 +1996-02-01 2 +1996-02-03 1 +1996-02-10 1 +1996-02-11 1 +1996-02-21 1 +1996-03-13 1 +1996-03-21 1 +1996-03-30 1 +1996-04-12 1 +1996-04-21 1 +1996-05-07 1 +1996-09-26 1 +1996-09-29 1 +1996-10-02 1 +1996-10-17 1 +1996-11-04 1 +1996-11-14 1 +1996-12-08 1 +1997-01-25 1 +1997-01-27 1 +1997-01-28 1 +1997-02-20 1 +1997-03-18 1 +1997-04-17 1 +1997-04-19 1 +1998-01-29 1 +1998-02-23 1 +1998-03-05 1 +1998-04-10 1 +1998-04-12 1 +1998-05-23 1 +1998-06-19 1 +1998-06-24 1 +1998-06-26 1 +1998-06-27 1 +1998-07-04 1 +1998-08-11 1 +1998-08-13 1 +1998-10-09 1 +1998-10-23 1 +1998-10-30 1 +PREHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +POSTHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string) + outputColumnNames: l_shipdate + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992 4 1 +1992 7 3 +1993 4 2 +1993 5 1 +1993 10 2 +1993 11 2 +1993 12 4 +1994 1 4 +1994 2 4 +1994 3 2 +1994 6 2 +1994 7 3 +1994 8 3 +1994 9 1 +1994 10 3 +1994 12 3 +1995 4 1 +1995 7 3 +1995 8 4 +1995 10 1 +1995 11 2 +1996 1 7 +1996 2 6 +1996 3 3 +1996 4 2 +1996 5 1 +1996 9 2 +1996 10 2 +1996 11 2 +1996 12 1 +1997 1 3 +1997 2 1 +1997 3 1 +1997 4 2 +1998 1 1 +1998 2 1 +1998 3 1 +1998 4 2 +1998 5 1 +1998 6 4 +1998 7 1 +1998 8 2 +1998 10 3 +PREHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +POSTHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992 4 1 +1992 7 3 +1993 4 2 +1993 5 1 +1993 10 2 +1993 11 2 +1993 12 4 +1994 1 4 +1994 2 4 +1994 3 2 +1994 6 2 +1994 7 3 +1994 8 3 +1994 9 1 +1994 10 3 +1994 12 3 +1995 4 1 +1995 7 3 +1995 8 4 +1995 10 1 +1995 11 2 +1996 1 7 +1996 2 6 +1996 3 3 +1996 4 2 +1996 5 1 +1996 9 2 +1996 10 2 +1996 11 2 +1996 12 1 +1997 1 3 +1997 2 1 +1997 3 1 +1997 4 2 +1998 1 1 +1998 2 1 +1998 3 1 +1998 4 2 +1998 5 1 +1998 6 4 +1998 7 1 +1998 8 2 +1998 10 3 +PREHOOK: query: explain select lastyear.month, + thisyear.month, + (thisyear.monthly_shipments - lastyear.monthly_shipments) / +lastyear.monthly_shipments as monthly_shipments_delta + from (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1997 + group by year(l_shipdate), month(l_shipdate) + ) lastyear join + (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1998 + group by year(l_shipdate), month(l_shipdate) + ) thisyear + on lastyear.month = thisyear.month +PREHOOK: type: QUERY +POSTHOOK: query: explain select lastyear.month, + thisyear.month, + (thisyear.monthly_shipments - lastyear.monthly_shipments) / +lastyear.monthly_shipments as monthly_shipments_delta + from (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1997 + group by year(l_shipdate), month(l_shipdate) + ) lastyear join + (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1998 + group by year(l_shipdate), month(l_shipdate) + ) thisyear + on lastyear.month = thisyear.month +POSTHOOK: type: QUERY +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, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lastyear:default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (year(l_shipdate) = 1997) (type: boolean) + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: thisyear:default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (year(l_shipdate) = 1998) (type: boolean) + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col1, _col2, _col4, _col5 + Statistics: Num rows: 13 Data size: 1244 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col4 (type: int), ((_col5 - _col2) / _col2) (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1244 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1244 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select l_shipdate, cnt +from (select l_shipdate, count(l_shipdate) as cnt from lineitem_ix group by l_shipdate +union all +select l_shipdate, l_orderkey as cnt +from lineitem_ix) dummy +PREHOOK: type: QUERY +POSTHOOK: query: explain select l_shipdate, cnt +from (select l_shipdate, count(l_shipdate) as cnt from lineitem_ix group by l_shipdate +union all +select l_shipdate, l_orderkey as cnt +from lineitem_ix) dummy +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Select Operator + expressions: l_shipdate (type: string), UDFToLong(l_orderkey) (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Map 2 + Map Operator Tree: + TableScan + alias: null-subquery1:dummy-subquery1:default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: l_shipdate (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 95 Data size: 8960 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: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: CREATE TABLE tbl(key int, value int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl +POSTHOOK: query: CREATE TABLE tbl(key int, value int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl +PREHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@tbl +POSTHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +PREHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tbl +PREHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._bucketname SIMPLE [(tbl)tbl.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._count_of_key EXPRESSION [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._offsets EXPRESSION [(tbl)tbl.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__.key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: EXPLAIN select key, count(key) from tbl where key = 1 group by key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, count(key) from tbl where key = 1 group by key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (key = 1) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 1 (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key, count(key) from tbl group by key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, count(key) from tbl group by key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tbl_tbl_key_idx__ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(1) from tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(1) from tbl +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(key) from tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(key) from tbl +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tbl_tbl_key_idx__ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl GROUP BY value, key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl GROUP BY value, key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: int), key (type: int) + outputColumnNames: value, key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: value (type: int), key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl WHERE key = 3 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl WHERE key = 3 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (key = 3) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 3 (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl WHERE value = 2 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl WHERE value = 2 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 2) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl GROUP BY key, substr(key,2,3) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl GROUP BY key, substr(key,2,3) +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), substr(key, 2, 3) (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key, value FROM tbl GROUP BY value, key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, value FROM tbl GROUP BY value, key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: int), key (type: int) + outputColumnNames: value, key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: value (type: int), key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key, value FROM tbl WHERE value = 1 GROUP BY key, value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, value FROM tbl WHERE value = 1 GROUP BY key, value +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 1) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), 1 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 2) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), 2 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 AND key = 3 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 AND key = 3 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((value = 2) and (key = 3)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 3 (type: int), 2 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = key) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl WHERE value = key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl WHERE value = key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = key) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), substr(value, 2, 3) (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), substr(value, 2, 3) (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select * FROM (select DISTINCT key, value FROM tbl) v1 WHERE v1.value = 2 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select * FROM (select DISTINCT key, value FROM tbl) v1 WHERE v1.value = 2 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 2) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), 2 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: DROP TABLE tbl +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tbl +PREHOOK: Output: default@tbl +POSTHOOK: query: DROP TABLE tbl +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@tbl +PREHOOK: query: CREATE TABLE tblpart (key int, value string) PARTITIONED BY (ds string, hr int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tblpart +POSTHOOK: query: CREATE TABLE tblpart (key int, value string) PARTITIONED BY (ds string, hr int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tblpart +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Output: default@tblpart@ds=2008-04-08/hr=11 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@tblpart@ds=2008-04-08/hr=11 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@tblpart@ds=2008-04-08/hr=12 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@tblpart@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Output: default@tblpart@ds=2008-04-09/hr=11 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@tblpart@ds=2008-04-09/hr=11 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: default@tblpart@ds=2008-04-09/hr=12 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: default@tblpart@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: CREATE INDEX tbl_part_index ON TABLE tblpart(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@tblpart +POSTHOOK: query: CREATE INDEX tbl_part_index ON TABLE tblpart(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@tblpart +POSTHOOK: Output: default@default__tblpart_tbl_part_index__ +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=11) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-08/hr=11 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=11 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=11) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=11 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tblpart + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=12) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=12 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=12) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=11) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-09/hr=11 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=11 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=11) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=11 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=12) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-09/hr=12 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=12 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=12) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tblpart_tbl_part_index__ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: DROP INDEX tbl_part_index on tblpart +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@tblpart +POSTHOOK: query: DROP INDEX tbl_part_index on tblpart +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@tblpart +PREHOOK: query: DROP TABLE tblpart +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tblpart +PREHOOK: Output: default@tblpart +POSTHOOK: query: DROP TABLE tblpart +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tblpart +POSTHOOK: Output: default@tblpart +PREHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl +POSTHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@tbl +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@tbl +PREHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@tbl +POSTHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +PREHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tbl +PREHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._bucketname SIMPLE [(tbl)tbl.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._count_of_key EXPRESSION [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._offsets EXPRESSION [(tbl)tbl.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__.key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: explain select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl +#### A masked pattern was here #### +POSTHOOK: query: select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl +#### A masked pattern was here #### +1 1 +2 3 +3 2 +4 2 +6 1 +7 1 +PREHOOK: query: explain select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tbl_tbl_key_idx__ + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@default__tbl_tbl_key_idx__ +PREHOOK: Input: default@tbl +#### A masked pattern was here #### +POSTHOOK: query: select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__tbl_tbl_key_idx__ +POSTHOOK: Input: default@tbl +#### A masked pattern was here #### +1 1 +2 3 +3 2 +4 2 +6 1 +7 1 +PREHOOK: query: DROP TABLE tbl +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tbl +PREHOOK: Output: default@tbl +POSTHOOK: query: DROP TABLE tbl +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@tbl diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx_cbo_1.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx_cbo_1.q.out new file mode 100644 index 0000000..f3d9507 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx_cbo_1.q.out @@ -0,0 +1,2730 @@ +PREHOOK: query: DROP TABLE IF EXISTS lineitem_ix +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE IF EXISTS lineitem_ix +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE lineitem_ix (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@lineitem_ix +POSTHOOK: query: CREATE TABLE lineitem_ix (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@lineitem_ix +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem_ix +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@lineitem_ix +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem_ix +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@lineitem_ix +PREHOOK: query: CREATE INDEX lineitem_ix_lshipdate_idx ON TABLE lineitem_ix(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@lineitem_ix +POSTHOOK: query: CREATE INDEX lineitem_ix_lshipdate_idx ON TABLE lineitem_ix(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +PREHOOK: query: ALTER INDEX lineitem_ix_lshipdate_idx ON lineitem_ix REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@lineitem_ix +PREHOOK: Output: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: query: ALTER INDEX lineitem_ix_lshipdate_idx ON lineitem_ix REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__._bucketname SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__._count_of_l_shipdate EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:l_shipdate, type:string, comment:null), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__._offsets EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_lshipdate_idx__.l_shipdate SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:l_shipdate, type:string, comment:null), ] +PREHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +PREHOOK: type: QUERY +POSTHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string) + outputColumnNames: l_shipdate + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_shipdate) + keys: l_shipdate (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 120 Data size: 12099 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: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992-04-27 1 +1992-07-02 1 +1992-07-10 1 +1992-07-21 1 +1993-04-01 1 +1993-04-13 1 +1993-05-14 1 +1993-10-29 2 +1993-11-09 2 +1993-12-04 1 +1993-12-09 2 +1993-12-14 1 +1994-01-12 1 +1994-01-16 1 +1994-01-26 2 +1994-02-02 1 +1994-02-13 1 +1994-02-19 1 +1994-02-21 1 +1994-03-03 1 +1994-03-17 1 +1994-06-03 1 +1994-06-06 1 +1994-07-02 1 +1994-07-19 1 +1994-07-31 1 +1994-08-08 1 +1994-08-17 1 +1994-08-24 1 +1994-09-30 1 +1994-10-03 1 +1994-10-16 1 +1994-10-31 1 +1994-12-01 1 +1994-12-24 1 +1994-12-30 1 +1995-04-20 1 +1995-07-06 1 +1995-07-17 1 +1995-07-21 1 +1995-08-04 1 +1995-08-07 1 +1995-08-14 1 +1995-08-28 1 +1995-10-23 1 +1995-11-08 1 +1995-11-26 1 +1996-01-10 1 +1996-01-15 1 +1996-01-16 1 +1996-01-19 1 +1996-01-22 1 +1996-01-29 1 +1996-01-30 1 +1996-02-01 2 +1996-02-03 1 +1996-02-10 1 +1996-02-11 1 +1996-02-21 1 +1996-03-13 1 +1996-03-21 1 +1996-03-30 1 +1996-04-12 1 +1996-04-21 1 +1996-05-07 1 +1996-09-26 1 +1996-09-29 1 +1996-10-02 1 +1996-10-17 1 +1996-11-04 1 +1996-11-14 1 +1996-12-08 1 +1997-01-25 1 +1997-01-27 1 +1997-01-28 1 +1997-02-20 1 +1997-03-18 1 +1997-04-17 1 +1997-04-19 1 +1998-01-29 1 +1998-02-23 1 +1998-03-05 1 +1998-04-10 1 +1998-04-12 1 +1998-05-23 1 +1998-06-19 1 +1998-06-24 1 +1998-06-26 1 +1998-06-27 1 +1998-07-04 1 +1998-08-11 1 +1998-08-13 1 +1998-10-09 1 +1998-10-23 1 +1998-10-30 1 +PREHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +PREHOOK: type: QUERY +POSTHOOK: query: explain select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: l_shipdate (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 95 Data size: 8960 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: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select l_shipdate, count(l_shipdate) +from lineitem_ix +group by l_shipdate +order by l_shipdate +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992-04-27 1 +1992-07-02 1 +1992-07-10 1 +1992-07-21 1 +1993-04-01 1 +1993-04-13 1 +1993-05-14 1 +1993-10-29 2 +1993-11-09 2 +1993-12-04 1 +1993-12-09 2 +1993-12-14 1 +1994-01-12 1 +1994-01-16 1 +1994-01-26 2 +1994-02-02 1 +1994-02-13 1 +1994-02-19 1 +1994-02-21 1 +1994-03-03 1 +1994-03-17 1 +1994-06-03 1 +1994-06-06 1 +1994-07-02 1 +1994-07-19 1 +1994-07-31 1 +1994-08-08 1 +1994-08-17 1 +1994-08-24 1 +1994-09-30 1 +1994-10-03 1 +1994-10-16 1 +1994-10-31 1 +1994-12-01 1 +1994-12-24 1 +1994-12-30 1 +1995-04-20 1 +1995-07-06 1 +1995-07-17 1 +1995-07-21 1 +1995-08-04 1 +1995-08-07 1 +1995-08-14 1 +1995-08-28 1 +1995-10-23 1 +1995-11-08 1 +1995-11-26 1 +1996-01-10 1 +1996-01-15 1 +1996-01-16 1 +1996-01-19 1 +1996-01-22 1 +1996-01-29 1 +1996-01-30 1 +1996-02-01 2 +1996-02-03 1 +1996-02-10 1 +1996-02-11 1 +1996-02-21 1 +1996-03-13 1 +1996-03-21 1 +1996-03-30 1 +1996-04-12 1 +1996-04-21 1 +1996-05-07 1 +1996-09-26 1 +1996-09-29 1 +1996-10-02 1 +1996-10-17 1 +1996-11-04 1 +1996-11-14 1 +1996-12-08 1 +1997-01-25 1 +1997-01-27 1 +1997-01-28 1 +1997-02-20 1 +1997-03-18 1 +1997-04-17 1 +1997-04-19 1 +1998-01-29 1 +1998-02-23 1 +1998-03-05 1 +1998-04-10 1 +1998-04-12 1 +1998-05-23 1 +1998-06-19 1 +1998-06-24 1 +1998-06-26 1 +1998-06-27 1 +1998-07-04 1 +1998-08-11 1 +1998-08-13 1 +1998-10-09 1 +1998-10-23 1 +1998-10-30 1 +PREHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +POSTHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string) + outputColumnNames: l_shipdate + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 120 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 60 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992 4 1 +1992 7 3 +1993 4 2 +1993 5 1 +1993 10 2 +1993 11 2 +1993 12 4 +1994 1 4 +1994 2 4 +1994 3 2 +1994 6 2 +1994 7 3 +1994 8 3 +1994 9 1 +1994 10 3 +1994 12 3 +1995 4 1 +1995 7 3 +1995 8 4 +1995 10 1 +1995 11 2 +1996 1 7 +1996 2 6 +1996 3 3 +1996 4 2 +1996 5 1 +1996 9 2 +1996 10 2 +1996 11 2 +1996 12 1 +1997 1 3 +1997 2 1 +1997 3 1 +1997 4 2 +1998 1 1 +1998 2 1 +1998 3 1 +1998 4 2 +1998 5 1 +1998 6 4 +1998 7 1 +1998 8 2 +1998 10 3 +PREHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +POSTHOOK: query: explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem_ix +group by year(l_shipdate), month(l_shipdate) +order by year, month +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_lshipdate_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1992 4 1 +1992 7 3 +1993 4 2 +1993 5 1 +1993 10 2 +1993 11 2 +1993 12 4 +1994 1 4 +1994 2 4 +1994 3 2 +1994 6 2 +1994 7 3 +1994 8 3 +1994 9 1 +1994 10 3 +1994 12 3 +1995 4 1 +1995 7 3 +1995 8 4 +1995 10 1 +1995 11 2 +1996 1 7 +1996 2 6 +1996 3 3 +1996 4 2 +1996 5 1 +1996 9 2 +1996 10 2 +1996 11 2 +1996 12 1 +1997 1 3 +1997 2 1 +1997 3 1 +1997 4 2 +1998 1 1 +1998 2 1 +1998 3 1 +1998 4 2 +1998 5 1 +1998 6 4 +1998 7 1 +1998 8 2 +1998 10 3 +PREHOOK: query: explain select lastyear.month, + thisyear.month, + (thisyear.monthly_shipments - lastyear.monthly_shipments) / +lastyear.monthly_shipments as monthly_shipments_delta + from (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1997 + group by year(l_shipdate), month(l_shipdate) + ) lastyear join + (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1998 + group by year(l_shipdate), month(l_shipdate) + ) thisyear + on lastyear.month = thisyear.month +PREHOOK: type: QUERY +POSTHOOK: query: explain select lastyear.month, + thisyear.month, + (thisyear.monthly_shipments - lastyear.monthly_shipments) / +lastyear.monthly_shipments as monthly_shipments_delta + from (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1997 + group by year(l_shipdate), month(l_shipdate) + ) lastyear join + (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem_ix + where year(l_shipdate) = 1998 + group by year(l_shipdate), month(l_shipdate) + ) thisyear + on lastyear.month = thisyear.month +POSTHOOK: type: QUERY +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, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lastyear:default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (year(l_shipdate) = 1997) (type: boolean) + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: thisyear:default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (year(l_shipdate) = 1998) (type: boolean) + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: year(l_shipdate) (type: int), month(l_shipdate) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 47 Data size: 4432 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 24 Data size: 2263 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col1, _col2, _col4, _col5 + Statistics: Num rows: 13 Data size: 1244 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col4 (type: int), ((_col5 - _col2) / _col2) (type: double) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1244 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1244 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 12 Data size: 1131 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select l_shipdate, cnt +from (select l_shipdate, count(l_shipdate) as cnt from lineitem_ix group by l_shipdate +union all +select l_shipdate, l_orderkey as cnt +from lineitem_ix) dummy +PREHOOK: type: QUERY +POSTHOOK: query: explain select l_shipdate, cnt +from (select l_shipdate, count(l_shipdate) as cnt from lineitem_ix group by l_shipdate +union all +select l_shipdate, l_orderkey as cnt +from lineitem_ix) dummy +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Select Operator + expressions: l_shipdate (type: string), UDFToLong(l_orderkey) (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Map 2 + Map Operator Tree: + TableScan + alias: null-subquery1:dummy-subquery1:default.default__lineitem_ix_lineitem_ix_lshipdate_idx__ + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint) + outputColumnNames: l_shipdate, _count_of_l_shipdate + Statistics: Num rows: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_shipdate) + keys: l_shipdate (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 95 Data size: 8960 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: 95 Data size: 8960 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: CREATE TABLE tbl(key int, value int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl +POSTHOOK: query: CREATE TABLE tbl(key int, value int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl +PREHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@tbl +POSTHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +PREHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tbl +PREHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._bucketname SIMPLE [(tbl)tbl.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._count_of_key EXPRESSION [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._offsets EXPRESSION [(tbl)tbl.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__.key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: EXPLAIN select key, count(key) from tbl where key = 1 group by key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, count(key) from tbl where key = 1 group by key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (key = 1) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 1 (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key, count(key) from tbl group by key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, count(key) from tbl group by key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tbl_tbl_key_idx__ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(1) from tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(1) from tbl +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(key) from tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(key) from tbl +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tbl_tbl_key_idx__ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl GROUP BY value, key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl GROUP BY value, key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: int), key (type: int) + outputColumnNames: value, key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: value (type: int), key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl WHERE key = 3 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl WHERE key = 3 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (key = 3) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 3 (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl WHERE value = 2 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl WHERE value = 2 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 2) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key FROM tbl GROUP BY key, substr(key,2,3) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key FROM tbl GROUP BY key, substr(key,2,3) +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), substr(key, 2, 3) (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key, value FROM tbl GROUP BY value, key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, value FROM tbl GROUP BY value, key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: value (type: int), key (type: int) + outputColumnNames: value, key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: value (type: int), key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: int), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select key, value FROM tbl WHERE value = 1 GROUP BY key, value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select key, value FROM tbl WHERE value = 1 GROUP BY key, value +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 1) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), 1 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 2) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), 2 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 AND key = 3 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 AND key = 3 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: ((value = 2) and (key = 3)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 3 (type: int), 2 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, value FROM tbl WHERE value = key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = key) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl WHERE value = key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl WHERE value = key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = key) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), substr(value, 2, 3) (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), substr(value, 2, 3) (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select * FROM (select DISTINCT key, value FROM tbl) v1 WHERE v1.value = 2 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select * FROM (select DISTINCT key, value FROM tbl) v1 WHERE v1.value = 2 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (value = 2) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: int), 2 (type: int) + outputColumnNames: key, value + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: key (type: int), value (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: int), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: DROP TABLE tbl +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tbl +PREHOOK: Output: default@tbl +POSTHOOK: query: DROP TABLE tbl +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@tbl +PREHOOK: query: CREATE TABLE tblpart (key int, value string) PARTITIONED BY (ds string, hr int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tblpart +POSTHOOK: query: CREATE TABLE tblpart (key int, value string) PARTITIONED BY (ds string, hr int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tblpart +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Output: default@tblpart@ds=2008-04-08/hr=11 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@tblpart@ds=2008-04-08/hr=11 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@tblpart@ds=2008-04-08/hr=12 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@tblpart@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Output: default@tblpart@ds=2008-04-09/hr=11 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@tblpart@ds=2008-04-09/hr=11 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: default@tblpart@ds=2008-04-09/hr=12 +POSTHOOK: query: INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: default@tblpart@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tblpart PARTITION(ds=2008-04-09,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: CREATE INDEX tbl_part_index ON TABLE tblpart(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@tblpart +POSTHOOK: query: CREATE INDEX tbl_part_index ON TABLE tblpart(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@tblpart +POSTHOOK: Output: default@default__tblpart_tbl_part_index__ +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=11) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-08/hr=11 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=11 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=11) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=11 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tblpart + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=12) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=12 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=12) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-08,hr=12).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=11) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-09/hr=11 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=11 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=11) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=11 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=11).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=12) REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tblpart +PREHOOK: Input: default@tblpart@ds=2008-04-09/hr=12 +PREHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=12 +POSTHOOK: query: ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=12) REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tblpart +POSTHOOK: Input: default@tblpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: default@default__tblpart_tbl_part_index__@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12)._bucketname SIMPLE [(tblpart)tblpart.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12)._count_of_key EXPRESSION [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12)._offsets EXPRESSION [(tblpart)tblpart.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tblpart_tbl_part_index__ PARTITION(ds=2008-04-09,hr=12).key SIMPLE [(tblpart)tblpart.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tblpart_tbl_part_index__ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: DROP INDEX tbl_part_index on tblpart +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@tblpart +POSTHOOK: query: DROP INDEX tbl_part_index on tblpart +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@tblpart +PREHOOK: query: DROP TABLE tblpart +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tblpart +PREHOOK: Output: default@tblpart +POSTHOOK: query: DROP TABLE tblpart +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tblpart +POSTHOOK: Output: default@tblpart +PREHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl +POSTHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@tbl +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@tbl +PREHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@tbl +POSTHOOK: query: CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +PREHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@tbl +PREHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: query: ALTER INDEX tbl_key_idx ON tbl REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@default__tbl_tbl_key_idx__ +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._bucketname SIMPLE [(tbl)tbl.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._count_of_key EXPRESSION [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__._offsets EXPRESSION [(tbl)tbl.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__tbl_tbl_key_idx__.key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: explain select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int) + outputColumnNames: key + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 17 Data size: 70 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 32 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl +#### A masked pattern was here #### +POSTHOOK: query: select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl +#### A masked pattern was here #### +1 1 +2 3 +3 2 +4 2 +6 1 +7 1 +PREHOOK: query: explain select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__tbl_tbl_key_idx__ + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 215 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, count(key) from tbl group by key order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@default__tbl_tbl_key_idx__ +PREHOOK: Input: default@tbl +#### A masked pattern was here #### +POSTHOOK: query: select key, count(key) from tbl group by key order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__tbl_tbl_key_idx__ +POSTHOOK: Input: default@tbl +#### A masked pattern was here #### +1 1 +2 3 +3 2 +4 2 +6 1 +7 1 +PREHOOK: query: DROP TABLE tbl +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tbl +PREHOOK: Output: default@tbl +POSTHOOK: query: DROP TABLE tbl +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@tbl diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx_cbo_2.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx_cbo_2.q.out new file mode 100644 index 0000000..957a8e6 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/ql_rewrite_gbtoidx_cbo_2.q.out @@ -0,0 +1,4043 @@ +PREHOOK: query: DROP TABLE IF EXISTS lineitem_ix +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE IF EXISTS lineitem_ix +POSTHOOK: type: DROPTABLE +PREHOOK: query: DROP INDEX IF EXISTS lineitem_ix_L_ORDERKEY_idx on lineitem_ix +PREHOOK: type: DROPINDEX +POSTHOOK: query: DROP INDEX IF EXISTS lineitem_ix_L_ORDERKEY_idx on lineitem_ix +POSTHOOK: type: DROPINDEX +PREHOOK: query: DROP INDEX IF EXISTS lineitem_ix_L_PARTKEY_idx on lineitem_ix +PREHOOK: type: DROPINDEX +POSTHOOK: query: DROP INDEX IF EXISTS lineitem_ix_L_PARTKEY_idx on lineitem_ix +POSTHOOK: type: DROPINDEX +PREHOOK: query: CREATE TABLE lineitem_ix (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@lineitem_ix +POSTHOOK: query: CREATE TABLE lineitem_ix (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@lineitem_ix +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem_ix +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@lineitem_ix +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem_ix +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@lineitem_ix +PREHOOK: query: CREATE INDEX lineitem_ix_L_ORDERKEY_idx ON TABLE lineitem_ix(L_ORDERKEY) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(L_ORDERKEY)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@lineitem_ix +POSTHOOK: query: CREATE INDEX lineitem_ix_L_ORDERKEY_idx ON TABLE lineitem_ix(L_ORDERKEY) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(L_ORDERKEY)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: query: ALTER INDEX lineitem_ix_L_ORDERKEY_idx ON lineitem_ix REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@lineitem_ix +PREHOOK: Output: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: query: ALTER INDEX lineitem_ix_L_ORDERKEY_idx ON lineitem_ix REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_orderkey_idx__._bucketname SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_orderkey_idx__._count_of_l_orderkey EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:l_orderkey, type:int, comment:null), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_orderkey_idx__._offsets EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_orderkey_idx__.l_orderkey SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:l_orderkey, type:int, comment:null), ] +PREHOOK: query: CREATE INDEX lineitem_ix_L_PARTKEY_idx ON TABLE lineitem_ix(L_PARTKEY) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(L_PARTKEY)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@lineitem_ix +POSTHOOK: query: CREATE INDEX lineitem_ix_L_PARTKEY_idx ON TABLE lineitem_ix(L_PARTKEY) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(L_PARTKEY)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +PREHOOK: query: ALTER INDEX lineitem_ix_L_PARTKEY_idx ON lineitem_ix REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@lineitem_ix +PREHOOK: Output: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +POSTHOOK: query: ALTER INDEX lineitem_ix_L_PARTKEY_idx ON lineitem_ix REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Output: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_partkey_idx__._bucketname SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_partkey_idx__._count_of_l_partkey EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:l_partkey, type:int, comment:null), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_partkey_idx__._offsets EXPRESSION [(lineitem_ix)lineitem_ix.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__lineitem_ix_lineitem_ix_l_partkey_idx__.l_partkey SIMPLE [(lineitem_ix)lineitem_ix.FieldSchema(name:l_partkey, type:int, comment:null), ] +PREHOOK: query: explain +select count(1) +from lineitem_ix +PREHOOK: type: QUERY +POSTHOOK: query: explain +select count(1) +from lineitem_ix +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 0 Data size: 12099 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 0 Data size: 12099 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(1) +from lineitem_ix +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select count(1) +from lineitem_ix +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +100 +PREHOOK: query: explain +select count(L_ORDERKEY) +from lineitem_ix +PREHOOK: type: QUERY +POSTHOOK: query: explain +select count(L_ORDERKEY) +from lineitem_ix +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(L_ORDERKEY) +from lineitem_ix +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select count(L_ORDERKEY) +from lineitem_ix +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +100 +PREHOOK: query: explain select L_ORDERKEY+L_PARTKEY as keysum, +count(L_ORDERKEY), count(L_PARTKEY) +from lineitem_ix +group by L_ORDERKEY, L_PARTKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain select L_ORDERKEY+L_PARTKEY as keysum, +count(L_ORDERKEY), count(L_PARTKEY) +from lineitem_ix +group by L_ORDERKEY, L_PARTKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), l_partkey (type: int) + outputColumnNames: l_orderkey, l_partkey + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_orderkey), count(l_partkey) + keys: l_orderkey (type: int), l_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint), _col3 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (_col0 + _col1) (type: int), _col2 (type: bigint), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY+L_PARTKEY as keysum, +count(L_ORDERKEY), count(L_PARTKEY) +from lineitem_ix +group by L_ORDERKEY, L_PARTKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY+L_PARTKEY as keysum, +count(L_ORDERKEY), count(L_PARTKEY) +from lineitem_ix +group by L_ORDERKEY, L_PARTKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +119952 1 1 +79258 1 1 +115278 1 1 +20260 1 1 +88396 1 1 +73880 1 1 +137336 1 1 +37536 1 1 +175248 1 1 +94796 1 1 +12940 1 1 +15636 1 1 +163080 1 1 +82736 1 1 +60552 1 1 +67870 1 1 +7136 1 1 +44804 1 1 +123932 1 1 +115184 1 1 +85210 1 1 +128452 1 1 +145250 1 1 +139642 1 1 +123172 1 1 +54558 1 1 +45804 1 1 +40314 1 1 +196226 1 1 +62002 1 1 +137502 1 1 +195706 1 1 +135486 1 1 +169578 1 1 +96716 1 1 +77796 1 1 +4300 1 1 +62146 1 1 +183098 1 1 +35048 1 1 +82826 1 1 +106172 1 1 +119574 1 1 +24028 1 1 +64198 1 1 +89448 1 1 +103326 1 1 +40680 1 1 +20629 1 1 +2133 1 1 +104249 1 1 +37201 1 1 +197953 1 1 +55725 1 1 +67311 1 1 +102629 1 1 +86015 1 1 +21703 1 1 +175877 1 1 +59759 1 1 +182059 1 1 +37571 1 1 +61369 1 1 +44193 1 1 +2359 1 1 +34503 1 1 +11647 1 1 +186621 1 1 +119803 1 1 +30797 1 1 +120931 1 1 +94407 1 1 +88039 1 1 +126819 1 1 +178373 1 1 +22667 1 1 +63701 1 1 +173555 1 1 +87581 1 1 +2775 1 1 +49665 1 1 +18573 1 1 +65987 1 1 +151901 1 1 +155191 1 1 +19039 1 1 +157245 1 1 +85843 1 1 +179879 1 1 +1453 1 1 +161975 1 1 +108575 1 1 +173667 1 1 +109841 1 1 +33951 1 1 +94787 1 1 +92139 1 1 +139315 1 1 +29383 1 1 +485 1 1 +PREHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY) +from lineitem_ix +where L_ORDERKEY = 7 +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY) +from lineitem_ix +where L_ORDERKEY = 7 +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey = 7) (type: boolean) + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 7 (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(L_ORDERKEY) +from lineitem_ix +where L_ORDERKEY = 7 +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(L_ORDERKEY) +from lineitem_ix +where L_ORDERKEY = 7 +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +7 7 +PREHOOK: query: explain +select L_ORDERKEY, count(1) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(1) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(1) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(1) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 +34 3 +32 6 +38 1 +98 3 +96 2 +36 1 +66 2 +2 1 +70 6 +4 1 +68 7 +6 1 +69 6 +7 7 +67 6 +33 4 +97 3 +65 3 +71 6 +3 6 +39 6 +35 6 +1 6 +5 3 +37 3 +PREHOOK: query: explain +select count(L_ORDERKEY+1) +from lineitem_ix +PREHOOK: type: QUERY +POSTHOOK: query: explain +select count(L_ORDERKEY+1) +from lineitem_ix +POSTHOOK: type: QUERY +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, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count((l_orderkey + 1)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(L_ORDERKEY+1) +from lineitem_ix +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select count(L_ORDERKEY+1) +from lineitem_ix +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +100 +PREHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY+1) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY+1) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count((l_orderkey + 1)) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(L_ORDERKEY+1) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(L_ORDERKEY+1) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 +34 3 +32 6 +38 1 +98 3 +96 2 +36 1 +66 2 +2 1 +70 6 +4 1 +68 7 +6 1 +69 6 +7 7 +67 6 +33 4 +97 3 +65 3 +71 6 +3 6 +39 6 +35 6 +1 6 +5 3 +37 3 +PREHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY+1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY+1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count((((l_orderkey + 1) + l_orderkey) + 2)) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(L_ORDERKEY+1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(L_ORDERKEY+1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 +34 3 +32 6 +38 1 +98 3 +96 2 +36 1 +66 2 +2 1 +70 6 +4 1 +68 7 +6 1 +69 6 +7 7 +67 6 +33 4 +97 3 +65 3 +71 6 +3 6 +39 6 +35 6 +1 6 +5 3 +37 3 +PREHOOK: query: explain +select L_ORDERKEY, count(1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(((1 + l_orderkey) + 2)) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(1+L_ORDERKEY+2) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 +34 3 +32 6 +38 1 +98 3 +96 2 +36 1 +66 2 +2 1 +70 6 +4 1 +68 7 +6 1 +69 6 +7 7 +67 6 +33 4 +97 3 +65 3 +71 6 +3 6 +39 6 +35 6 +1 6 +5 3 +37 3 +PREHOOK: query: explain +select L_ORDERKEY as a, count(1) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY as a, count(1) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey < 7) (type: boolean) + Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1008 Data size: 4033 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 504 Data size: 2016 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 504 Data size: 2016 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 504 Data size: 2016 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY as a, count(1) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY as a, count(1) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +2 1 +4 1 +6 1 +3 6 +1 6 +5 3 +PREHOOK: query: explain +select L_ORDERKEY, count(keysum), sum(keysum) +from +(select L_ORDERKEY, L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(keysum), sum(keysum) +from +(select L_ORDERKEY, L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), (l_orderkey + l_partkey) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1), sum(_col1) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), sum(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(keysum), sum(keysum) +from +(select L_ORDERKEY, L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(keysum), sum(keysum) +from +(select L_ORDERKEY, L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 86015 +34 3 347422 +32 6 425147 +38 1 175877 +98 3 194959 +96 2 258658 +36 1 119803 +66 2 288739 +2 1 106172 +70 6 579033 +4 1 88039 +68 7 636998 +6 1 139642 +69 6 505146 +7 7 973580 +67 6 522264 +33 4 293374 +97 3 247035 +65 3 135092 +71 6 558240 +3 6 426418 +39 6 426444 +35 6 519350 +1 6 328000 +5 3 270043 +37 3 162426 +PREHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY), sum(L_ORDERKEY) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +POSTHOOK: query: explain +select L_ORDERKEY, count(L_ORDERKEY), sum(L_ORDERKEY) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(l_orderkey), sum(l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), sum(VALUE._col1) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, count(L_ORDERKEY), sum(L_ORDERKEY) +from lineitem_ix +group by L_ORDERKEY +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, count(L_ORDERKEY), sum(L_ORDERKEY) +from lineitem_ix +group by L_ORDERKEY +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 64 +34 3 102 +32 6 192 +38 1 38 +98 3 294 +96 2 192 +36 1 36 +66 2 132 +2 1 2 +70 6 420 +4 1 4 +68 7 476 +6 1 6 +69 6 414 +7 7 49 +67 6 402 +33 4 132 +97 3 291 +65 3 195 +71 6 426 +3 6 18 +39 6 234 +35 6 210 +1 6 6 +5 3 15 +37 3 111 +PREHOOK: query: explain +select colA, count(colA) +from (select L_ORDERKEY as colA from lineitem_ix) tabA +group by colA +PREHOOK: type: QUERY +POSTHOOK: query: explain +select colA, count(colA) +from (select L_ORDERKEY as colA from lineitem_ix) tabA +group by colA +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: taba:default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: _col0, _count_of_l_orderkey + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select colA, count(colA) +from (select L_ORDERKEY as colA from lineitem_ix) tabA +group by colA +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select colA, count(colA) +from (select L_ORDERKEY as colA from lineitem_ix) tabA +group by colA +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +64 1 +34 3 +32 6 +38 1 +98 3 +96 2 +36 1 +66 2 +2 1 +70 6 +4 1 +68 7 +6 1 +69 6 +7 7 +67 6 +33 4 +97 3 +65 3 +71 6 +3 6 +39 6 +35 6 +1 6 +5 3 +37 3 +PREHOOK: query: explain +select keysum, count(keysum) +from +(select L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by keysum +PREHOOK: type: QUERY +POSTHOOK: query: explain +select keysum, count(keysum) +from +(select L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by keysum +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (l_orderkey + l_partkey) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1512 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 756 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select keysum, count(keysum) +from +(select L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by keysum +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select keysum, count(keysum) +from +(select L_ORDERKEY+L_PARTKEY as keysum from lineitem_ix) tabA +group by keysum +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +183098 1 +169578 1 +115184 1 +137336 1 +175248 1 +88396 1 +7136 1 +135486 1 +137502 1 +85210 1 +119952 1 +24028 1 +20260 1 +94796 1 +106172 1 +44804 1 +79258 1 +67870 1 +62146 1 +145250 1 +82736 1 +123932 1 +89448 1 +195706 1 +115278 1 +12940 1 +64198 1 +4300 1 +40314 1 +139642 1 +37536 1 +62002 1 +128452 1 +77796 1 +123172 1 +35048 1 +60552 1 +163080 1 +15636 1 +40680 1 +103326 1 +119574 1 +82826 1 +96716 1 +196226 1 +54558 1 +45804 1 +73880 1 +2775 1 +102629 1 +151901 1 +2133 1 +55725 1 +186621 1 +119803 1 +92139 1 +175877 1 +37201 1 +37571 1 +109841 1 +11647 1 +29383 1 +173555 1 +22667 1 +86015 1 +88039 1 +21703 1 +85843 1 +87581 1 +2359 1 +65987 1 +44193 1 +157245 1 +182059 1 +139315 1 +67311 1 +178373 1 +161975 1 +33951 1 +94787 1 +34503 1 +120931 1 +19039 1 +485 1 +197953 1 +1453 1 +63701 1 +126819 1 +49665 1 +155191 1 +179879 1 +108575 1 +20629 1 +61369 1 +104249 1 +30797 1 +173667 1 +18573 1 +59759 1 +94407 1 +PREHOOK: query: explain +select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +PREHOOK: type: QUERY +POSTHOOK: query: explain +select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (l_orderkey + 1) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +34 4 +38 3 +98 3 +36 6 +72 6 +66 3 +2 6 +70 6 +4 6 +68 6 +6 3 +40 6 +8 7 +69 7 +7 1 +67 2 +33 6 +97 2 +65 1 +71 6 +3 1 +99 3 +39 1 +35 3 +5 1 +37 1 +PREHOOK: query: explain +select keysum, count(1) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +PREHOOK: type: QUERY +POSTHOOK: query: explain +select keysum, count(1) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: (l_orderkey + 1) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select keysum, count(1) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select keysum, count(1) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix) tabA +group by keysum +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +34 4 +38 3 +98 3 +36 6 +72 6 +66 3 +2 6 +70 6 +4 6 +68 6 +6 3 +40 6 +8 7 +69 7 +7 1 +67 2 +33 6 +97 2 +65 1 +71 6 +3 1 +99 3 +39 1 +35 3 +5 1 +37 1 +PREHOOK: query: explain +select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum +PREHOOK: type: QUERY +POSTHOOK: query: explain +select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey = 7) (type: boolean) + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 8 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select keysum, count(keysum) +from +(select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +8 7 +PREHOOK: query: explain +select ckeysum, count(ckeysum) +from +(select keysum, count(keysum) as ckeysum +from + (select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum) tabB +group by ckeysum +PREHOOK: type: QUERY +POSTHOOK: query: explain +select ckeysum, count(ckeysum) +from +(select keysum, count(keysum) as ckeysum +from + (select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum) tabB +group by ckeysum +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey = 7) (type: boolean) + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 8 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1) + keys: _col1 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 756 Data size: 3024 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 378 Data size: 1512 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 378 Data size: 1512 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 378 Data size: 1512 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ckeysum, count(ckeysum) +from +(select keysum, count(keysum) as ckeysum +from + (select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum) tabB +group by ckeysum +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select ckeysum, count(ckeysum) +from +(select keysum, count(keysum) as ckeysum +from + (select L_ORDERKEY+1 as keysum from lineitem_ix where L_ORDERKEY = 7) tabA +group by keysum) tabB +group by ckeysum +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +7 1 +PREHOOK: query: explain +select keysum, count(keysum) as ckeysum +from +(select L_ORDERKEY, count(L_ORDERKEY) as keysum +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY)tabA +group by keysum +PREHOOK: type: QUERY +POSTHOOK: query: explain +select keysum, count(keysum) as ckeysum +from +(select L_ORDERKEY, count(L_ORDERKEY) as keysum +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY)tabA +group by keysum +POSTHOOK: type: QUERY +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, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: taba:default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey < 7) (type: boolean) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col1) + keys: _col1 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 200 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select keysum, count(keysum) as ckeysum +from +(select L_ORDERKEY, count(L_ORDERKEY) as keysum +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY)tabA +group by keysum +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select keysum, count(keysum) as ckeysum +from +(select L_ORDERKEY, count(L_ORDERKEY) as keysum +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY)tabA +group by keysum +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +6 2 +3 1 +1 3 +PREHOOK: query: DROP INDEX IF EXISTS src_key_idx on src +PREHOOK: type: DROPINDEX +PREHOOK: Input: default@src +POSTHOOK: query: DROP INDEX IF EXISTS src_key_idx on src +POSTHOOK: type: DROPINDEX +POSTHOOK: Input: default@src +PREHOOK: query: CREATE INDEX src_key_idx ON TABLE src(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +PREHOOK: type: CREATEINDEX +PREHOOK: Input: default@src +POSTHOOK: query: CREATE INDEX src_key_idx ON TABLE src(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)") +POSTHOOK: type: CREATEINDEX +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src_key_idx__ +PREHOOK: query: ALTER INDEX src_key_idx ON src REBUILD +PREHOOK: type: ALTERINDEX_REBUILD +PREHOOK: Input: default@src +PREHOOK: Output: default@default__src_src_key_idx__ +POSTHOOK: query: ALTER INDEX src_key_idx ON src REBUILD +POSTHOOK: type: ALTERINDEX_REBUILD +POSTHOOK: Input: default@src +POSTHOOK: Output: default@default__src_src_key_idx__ +POSTHOOK: Lineage: default__src_src_key_idx__._bucketname SIMPLE [(src)src.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: default__src_src_key_idx__._count_of_key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: default__src_src_key_idx__._offsets EXPRESSION [(src)src.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), ] +POSTHOOK: Lineage: default__src_src_key_idx__.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: explain +select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b) +POSTHOOK: type: QUERY +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, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: taba:default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey < 7) (type: boolean) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: tabb:default.default__src_src_key_idx__ + Statistics: Num rows: 309 Data size: 23663 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 309 Data size: 23663 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 309 Data size: 23663 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: 309 Data size: 23663 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: bigint) + sort order: + + Map-reduce partition columns: _col1 (type: bigint) + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 169 Data size: 12972 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 169 Data size: 12972 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 169 Data size: 12972 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 154 Data size: 11793 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 154 Data size: 11793 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: bigint) + sort order: + + Map-reduce partition columns: _col1 (type: bigint) + Statistics: Num rows: 154 Data size: 11793 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b) +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@default__src_src_key_idx__ +PREHOOK: Input: default@lineitem_ix +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@default__src_src_key_idx__ +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Input: default@src +#### A masked pattern was here #### +2 1 8 1 +2 1 378 1 +2 1 435 1 +2 1 392 1 +2 1 495 1 +2 1 455 1 +2 1 11 1 +2 1 448 1 +2 1 189 1 +2 1 222 1 +2 1 145 1 +2 1 260 1 +2 1 77 1 +2 1 336 1 +2 1 82 1 +2 1 178 1 +2 1 17 1 +2 1 244 1 +2 1 457 1 +2 1 497 1 +2 1 266 1 +2 1 365 1 +2 1 262 1 +2 1 4 1 +2 1 286 1 +2 1 192 1 +2 1 163 1 +2 1 86 1 +2 1 66 1 +2 1 257 1 +2 1 156 1 +2 1 394 1 +2 1 2 1 +2 1 136 1 +2 1 460 1 +2 1 248 1 +2 1 80 1 +2 1 235 1 +2 1 194 1 +2 1 493 1 +2 1 226 1 +2 1 53 1 +2 1 475 1 +2 1 33 1 +2 1 183 1 +2 1 28 1 +2 1 196 1 +2 1 437 1 +2 1 446 1 +2 1 64 1 +2 1 482 1 +2 1 389 1 +2 1 150 1 +2 1 202 1 +2 1 310 1 +2 1 228 1 +2 1 116 1 +2 1 402 1 +2 1 444 1 +2 1 190 1 +2 1 484 1 +2 1 400 1 +2 1 341 1 +2 1 305 1 +2 1 170 1 +2 1 105 1 +2 1 323 1 +2 1 477 1 +2 1 44 1 +2 1 114 1 +2 1 419 1 +2 1 20 1 +2 1 143 1 +2 1 338 1 +2 1 411 1 +2 1 57 1 +2 1 453 1 +2 1 284 1 +2 1 275 1 +2 1 291 1 +2 1 356 1 +2 1 345 1 +2 1 491 1 +2 1 158 1 +2 1 332 1 +2 1 479 1 +2 1 181 1 +2 1 19 1 +2 1 374 1 +2 1 166 1 +2 1 54 1 +2 1 258 1 +2 1 302 1 +2 1 306 1 +2 1 494 1 +2 1 30 1 +2 1 315 1 +2 1 41 1 +2 1 407 1 +2 1 292 1 +2 1 452 1 +2 1 362 1 +2 1 252 1 +2 1 201 1 +2 1 285 1 +2 1 168 1 +2 1 180 1 +2 1 360 1 +2 1 10 1 +2 1 339 1 +2 1 283 1 +2 1 483 1 +2 1 368 1 +2 1 111 1 +2 1 443 1 +2 1 162 1 +2 1 335 1 +2 1 373 1 +2 1 186 1 +2 1 65 1 +2 1 126 1 +2 1 496 1 +2 1 487 1 +2 1 427 1 +2 1 418 1 +2 1 485 1 +2 1 366 1 +2 1 87 1 +2 1 157 1 +2 1 43 1 +2 1 155 1 +2 1 490 1 +2 1 131 1 +2 1 472 1 +2 1 432 1 +2 1 296 1 +2 1 34 1 +2 1 421 1 +2 1 386 1 +2 1 470 1 +2 1 214 1 +2 1 289 1 +2 1 27 1 +2 1 85 1 +2 1 379 1 +2 1 160 1 +2 1 287 1 +2 1 92 1 +2 1 393 1 +2 1 436 1 +2 1 47 1 +2 1 78 1 +2 1 364 1 +2 1 274 1 +2 1 153 1 +2 1 177 1 +2 1 74 1 +2 1 218 1 +2 1 247 1 +2 1 133 1 +2 1 351 1 +2 1 467 1 +2 1 449 1 +2 1 375 1 +2 1 308 1 +2 1 377 1 +2 1 481 1 +2 1 263 1 +2 1 69 1 +2 1 241 1 +2 1 249 1 +2 1 9 1 +2 1 96 1 +4 1 8 1 +4 1 378 1 +4 1 435 1 +4 1 392 1 +4 1 495 1 +4 1 455 1 +4 1 11 1 +4 1 448 1 +4 1 189 1 +4 1 222 1 +4 1 145 1 +4 1 260 1 +4 1 77 1 +4 1 336 1 +4 1 82 1 +4 1 178 1 +4 1 17 1 +4 1 244 1 +4 1 457 1 +4 1 497 1 +4 1 266 1 +4 1 365 1 +4 1 262 1 +4 1 4 1 +4 1 286 1 +4 1 192 1 +4 1 163 1 +4 1 86 1 +4 1 66 1 +4 1 257 1 +4 1 156 1 +4 1 394 1 +4 1 2 1 +4 1 136 1 +4 1 460 1 +4 1 248 1 +4 1 80 1 +4 1 235 1 +4 1 194 1 +4 1 493 1 +4 1 226 1 +4 1 53 1 +4 1 475 1 +4 1 33 1 +4 1 183 1 +4 1 28 1 +4 1 196 1 +4 1 437 1 +4 1 446 1 +4 1 64 1 +4 1 482 1 +4 1 389 1 +4 1 150 1 +4 1 202 1 +4 1 310 1 +4 1 228 1 +4 1 116 1 +4 1 402 1 +4 1 444 1 +4 1 190 1 +4 1 484 1 +4 1 400 1 +4 1 341 1 +4 1 305 1 +4 1 170 1 +4 1 105 1 +4 1 323 1 +4 1 477 1 +4 1 44 1 +4 1 114 1 +4 1 419 1 +4 1 20 1 +4 1 143 1 +4 1 338 1 +4 1 411 1 +4 1 57 1 +4 1 453 1 +4 1 284 1 +4 1 275 1 +4 1 291 1 +4 1 356 1 +4 1 345 1 +4 1 491 1 +4 1 158 1 +4 1 332 1 +4 1 479 1 +4 1 181 1 +4 1 19 1 +4 1 374 1 +4 1 166 1 +4 1 54 1 +4 1 258 1 +4 1 302 1 +4 1 306 1 +4 1 494 1 +4 1 30 1 +4 1 315 1 +4 1 41 1 +4 1 407 1 +4 1 292 1 +4 1 452 1 +4 1 362 1 +4 1 252 1 +4 1 201 1 +4 1 285 1 +4 1 168 1 +4 1 180 1 +4 1 360 1 +4 1 10 1 +4 1 339 1 +4 1 283 1 +4 1 483 1 +4 1 368 1 +4 1 111 1 +4 1 443 1 +4 1 162 1 +4 1 335 1 +4 1 373 1 +4 1 186 1 +4 1 65 1 +4 1 126 1 +4 1 496 1 +4 1 487 1 +4 1 427 1 +4 1 418 1 +4 1 485 1 +4 1 366 1 +4 1 87 1 +4 1 157 1 +4 1 43 1 +4 1 155 1 +4 1 490 1 +4 1 131 1 +4 1 472 1 +4 1 432 1 +4 1 296 1 +4 1 34 1 +4 1 421 1 +4 1 386 1 +4 1 470 1 +4 1 214 1 +4 1 289 1 +4 1 27 1 +4 1 85 1 +4 1 379 1 +4 1 160 1 +4 1 287 1 +4 1 92 1 +4 1 393 1 +4 1 436 1 +4 1 47 1 +4 1 78 1 +4 1 364 1 +4 1 274 1 +4 1 153 1 +4 1 177 1 +4 1 74 1 +4 1 218 1 +4 1 247 1 +4 1 133 1 +4 1 351 1 +4 1 467 1 +4 1 449 1 +4 1 375 1 +4 1 308 1 +4 1 377 1 +4 1 481 1 +4 1 263 1 +4 1 69 1 +4 1 241 1 +4 1 249 1 +4 1 9 1 +4 1 96 1 +6 1 8 1 +6 1 378 1 +6 1 435 1 +6 1 392 1 +6 1 495 1 +6 1 455 1 +6 1 11 1 +6 1 448 1 +6 1 189 1 +6 1 222 1 +6 1 145 1 +6 1 260 1 +6 1 77 1 +6 1 336 1 +6 1 82 1 +6 1 178 1 +6 1 17 1 +6 1 244 1 +6 1 457 1 +6 1 497 1 +6 1 266 1 +6 1 365 1 +6 1 262 1 +6 1 4 1 +6 1 286 1 +6 1 192 1 +6 1 163 1 +6 1 86 1 +6 1 66 1 +6 1 257 1 +6 1 156 1 +6 1 394 1 +6 1 2 1 +6 1 136 1 +6 1 460 1 +6 1 248 1 +6 1 80 1 +6 1 235 1 +6 1 194 1 +6 1 493 1 +6 1 226 1 +6 1 53 1 +6 1 475 1 +6 1 33 1 +6 1 183 1 +6 1 28 1 +6 1 196 1 +6 1 437 1 +6 1 446 1 +6 1 64 1 +6 1 482 1 +6 1 389 1 +6 1 150 1 +6 1 202 1 +6 1 310 1 +6 1 228 1 +6 1 116 1 +6 1 402 1 +6 1 444 1 +6 1 190 1 +6 1 484 1 +6 1 400 1 +6 1 341 1 +6 1 305 1 +6 1 170 1 +6 1 105 1 +6 1 323 1 +6 1 477 1 +6 1 44 1 +6 1 114 1 +6 1 419 1 +6 1 20 1 +6 1 143 1 +6 1 338 1 +6 1 411 1 +6 1 57 1 +6 1 453 1 +6 1 284 1 +6 1 275 1 +6 1 291 1 +6 1 356 1 +6 1 345 1 +6 1 491 1 +6 1 158 1 +6 1 332 1 +6 1 479 1 +6 1 181 1 +6 1 19 1 +6 1 374 1 +6 1 166 1 +6 1 54 1 +6 1 258 1 +6 1 302 1 +6 1 306 1 +6 1 494 1 +6 1 30 1 +6 1 315 1 +6 1 41 1 +6 1 407 1 +6 1 292 1 +6 1 452 1 +6 1 362 1 +6 1 252 1 +6 1 201 1 +6 1 285 1 +6 1 168 1 +6 1 180 1 +6 1 360 1 +6 1 10 1 +6 1 339 1 +6 1 283 1 +6 1 483 1 +6 1 368 1 +6 1 111 1 +6 1 443 1 +6 1 162 1 +6 1 335 1 +6 1 373 1 +6 1 186 1 +6 1 65 1 +6 1 126 1 +6 1 496 1 +6 1 487 1 +6 1 427 1 +6 1 418 1 +6 1 485 1 +6 1 366 1 +6 1 87 1 +6 1 157 1 +6 1 43 1 +6 1 155 1 +6 1 490 1 +6 1 131 1 +6 1 472 1 +6 1 432 1 +6 1 296 1 +6 1 34 1 +6 1 421 1 +6 1 386 1 +6 1 470 1 +6 1 214 1 +6 1 289 1 +6 1 27 1 +6 1 85 1 +6 1 379 1 +6 1 160 1 +6 1 287 1 +6 1 92 1 +6 1 393 1 +6 1 436 1 +6 1 47 1 +6 1 78 1 +6 1 364 1 +6 1 274 1 +6 1 153 1 +6 1 177 1 +6 1 74 1 +6 1 218 1 +6 1 247 1 +6 1 133 1 +6 1 351 1 +6 1 467 1 +6 1 449 1 +6 1 375 1 +6 1 308 1 +6 1 377 1 +6 1 481 1 +6 1 263 1 +6 1 69 1 +6 1 241 1 +6 1 249 1 +6 1 9 1 +6 1 96 1 +5 3 396 3 +5 3 316 3 +5 3 431 3 +5 3 273 3 +5 3 167 3 +5 3 466 3 +5 3 369 3 +5 3 187 3 +5 3 417 3 +5 3 318 3 +5 3 0 3 +5 3 35 3 +5 3 208 3 +5 3 327 3 +5 3 480 3 +5 3 384 3 +5 3 119 3 +5 3 199 3 +5 3 70 3 +5 3 430 3 +5 3 5 3 +5 3 438 3 +5 3 454 3 +5 3 128 3 +5 3 311 3 +5 3 193 3 +5 3 90 3 +5 3 298 3 +5 3 409 3 +5 3 498 3 +5 3 403 3 +PREHOOK: query: explain +select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b and tabB.a < '2') +PREHOOK: type: QUERY +POSTHOOK: query: explain +select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b and tabB.a < '2') +POSTHOOK: type: QUERY +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, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: taba:default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey < 7) (type: boolean) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: tabb:default.default__src_src_key_idx__ + Statistics: Num rows: 309 Data size: 23663 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key < '2') (type: boolean) + Statistics: Num rows: 103 Data size: 7887 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), _count_of_key (type: bigint) + outputColumnNames: key, _count_of_key + Statistics: Num rows: 103 Data size: 7887 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_key) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 103 Data size: 7887 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: 103 Data size: 7887 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: bigint) + sort order: + + Map-reduce partition columns: _col1 (type: bigint) + Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 56 Data size: 4295 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 56 Data size: 4295 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 56 Data size: 4295 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 51 Data size: 3905 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 51 Data size: 3905 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: bigint) + sort order: + + Map-reduce partition columns: _col1 (type: bigint) + Statistics: Num rows: 51 Data size: 3905 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b and tabB.a < '2') +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@default__src_src_key_idx__ +PREHOOK: Input: default@lineitem_ix +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select tabA.a, tabA.b, tabB.a, tabB.b +from +(select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY) tabA +join +(select key as a, count(key) as b +from src +group by key +) tabB +on (tabA.b=tabB.b and tabB.a < '2') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@default__src_src_key_idx__ +POSTHOOK: Input: default@lineitem_ix +POSTHOOK: Input: default@src +#### A masked pattern was here #### +2 1 183 1 +2 1 114 1 +2 1 196 1 +2 1 150 1 +2 1 143 1 +2 1 192 1 +2 1 11 1 +2 1 163 1 +2 1 136 1 +2 1 156 1 +2 1 189 1 +2 1 116 1 +2 1 145 1 +2 1 190 1 +2 1 158 1 +2 1 178 1 +2 1 170 1 +2 1 17 1 +2 1 194 1 +2 1 181 1 +2 1 105 1 +2 1 19 1 +2 1 153 1 +2 1 111 1 +2 1 177 1 +2 1 133 1 +2 1 162 1 +2 1 186 1 +2 1 126 1 +2 1 160 1 +2 1 168 1 +2 1 180 1 +2 1 157 1 +2 1 155 1 +2 1 131 1 +2 1 10 1 +2 1 166 1 +4 1 183 1 +4 1 114 1 +4 1 196 1 +4 1 150 1 +4 1 143 1 +4 1 192 1 +4 1 11 1 +4 1 163 1 +4 1 136 1 +4 1 156 1 +4 1 189 1 +4 1 116 1 +4 1 145 1 +4 1 190 1 +4 1 158 1 +4 1 178 1 +4 1 170 1 +4 1 17 1 +4 1 194 1 +4 1 181 1 +4 1 105 1 +4 1 19 1 +4 1 153 1 +4 1 111 1 +4 1 177 1 +4 1 133 1 +4 1 162 1 +4 1 186 1 +4 1 126 1 +4 1 160 1 +4 1 168 1 +4 1 180 1 +4 1 157 1 +4 1 155 1 +4 1 131 1 +4 1 10 1 +4 1 166 1 +6 1 183 1 +6 1 114 1 +6 1 196 1 +6 1 150 1 +6 1 143 1 +6 1 192 1 +6 1 11 1 +6 1 163 1 +6 1 136 1 +6 1 156 1 +6 1 189 1 +6 1 116 1 +6 1 145 1 +6 1 190 1 +6 1 158 1 +6 1 178 1 +6 1 170 1 +6 1 17 1 +6 1 194 1 +6 1 181 1 +6 1 105 1 +6 1 19 1 +6 1 153 1 +6 1 111 1 +6 1 177 1 +6 1 133 1 +6 1 162 1 +6 1 186 1 +6 1 126 1 +6 1 160 1 +6 1 168 1 +6 1 180 1 +6 1 157 1 +6 1 155 1 +6 1 131 1 +6 1 10 1 +6 1 166 1 +5 3 187 3 +5 3 0 3 +5 3 167 3 +5 3 193 3 +5 3 119 3 +5 3 199 3 +5 3 128 3 +PREHOOK: query: EXPLAIN +select L_ORDERKEY FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +select L_ORDERKEY FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int) + outputColumnNames: l_orderkey + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: l_orderkey (type: int), (l_orderkey + 1) (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +34 +32 +71 +96 +38 +3 +1 +6 +39 +98 +97 +65 +68 +35 +2 +7 +64 +66 +4 +5 +70 +37 +67 +69 +36 +33 +PREHOOK: query: EXPLAIN +select L_ORDERKEY, L_ORDERKEY+1, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +select L_ORDERKEY, L_ORDERKEY+1, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: l_orderkey (type: int), (l_orderkey + 1) (type: int) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY, L_ORDERKEY+1, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY, L_ORDERKEY+1, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY, L_ORDERKEY+1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +34 35 3 +32 33 6 +71 72 6 +96 97 2 +38 39 1 +3 4 6 +1 2 6 +6 7 1 +39 40 6 +98 99 3 +97 98 3 +65 66 3 +68 69 7 +35 36 6 +2 3 1 +7 8 7 +64 65 1 +66 67 2 +4 5 1 +5 6 3 +70 71 6 +37 38 3 +67 68 6 +69 70 6 +36 37 1 +33 34 4 +PREHOOK: query: EXPLAIN +select L_ORDERKEY+2, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY+2 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +select L_ORDERKEY+2, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY+2 +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: (l_orderkey + 2) (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1302 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select L_ORDERKEY+2, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY+2 +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select L_ORDERKEY+2, count(L_ORDERKEY) FROM lineitem_ix GROUP BY L_ORDERKEY+2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +34 6 +38 1 +98 2 +36 3 +72 6 +66 1 +70 7 +4 1 +68 2 +100 3 +6 1 +40 1 +8 1 +41 6 +69 6 +7 3 +67 3 +71 6 +3 6 +73 6 +9 7 +99 3 +39 3 +35 4 +5 6 +37 6 +PREHOOK: query: --with cbo on, the following query can use idx + +explain +select b, count(b) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by b +PREHOOK: type: QUERY +POSTHOOK: query: --with cbo on, the following query can use idx + +explain +select b, count(b) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by b +POSTHOOK: type: QUERY +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, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: null-subquery1:taba-subquery1:default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey < 7) (type: boolean) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: null-subquery2:taba-subquery2:default.default__lineitem_ix_lineitem_ix_l_partkey_idx__ + Statistics: Num rows: 100 Data size: 8937 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_partkey < 10) (type: boolean) + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_partkey (type: int), _count_of_l_partkey (type: bigint) + outputColumnNames: l_partkey, _count_of_l_partkey + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_partkey) + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Group By Operator + aggregations: count(_col1) + keys: _col1 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 914 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 914 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 914 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Select Operator + expressions: _col1 (type: bigint) + outputColumnNames: _col1 + Group By Operator + aggregations: count(_col1) + keys: _col1 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + value expressions: _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select b, count(b) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by b +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select b, count(b) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +6 2 +3 1 +1 3 +PREHOOK: query: --with cbo on, the following query can not use idx because AggFunc is empty here + +explain +select a, count(a) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by a +PREHOOK: type: QUERY +POSTHOOK: query: --with cbo on, the following query can not use idx because AggFunc is empty here + +explain +select a, count(a) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by a +POSTHOOK: type: QUERY +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, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 5 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: null-subquery1:taba-subquery1:default.default__lineitem_ix_lineitem_ix_l_orderkey_idx__ + Statistics: Num rows: 26 Data size: 2604 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_orderkey < 7) (type: boolean) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_orderkey (type: int), _count_of_l_orderkey (type: bigint) + outputColumnNames: l_orderkey, _count_of_l_orderkey + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_orderkey) + keys: l_orderkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 8 Data size: 801 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: null-subquery2:taba-subquery2:default.default__lineitem_ix_lineitem_ix_l_partkey_idx__ + Statistics: Num rows: 100 Data size: 8937 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (l_partkey < 10) (type: boolean) + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: l_partkey (type: int), _count_of_l_partkey (type: bigint) + outputColumnNames: l_partkey, _count_of_l_partkey + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(_count_of_l_partkey) + keys: l_partkey (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 33 Data size: 2949 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 914 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 914 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 914 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a, count(a) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by a +PREHOOK: type: QUERY +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +PREHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select a, count(a) as ckeysum +from +( +select L_ORDERKEY as a, count(L_ORDERKEY) as b +from lineitem_ix +where L_ORDERKEY < 7 +group by L_ORDERKEY +union all +select L_PARTKEY as a, count(L_PARTKEY) as b +from lineitem_ix +where L_PARTKEY < 10 +group by L_PARTKEY +) tabA +group by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_orderkey_idx__ +POSTHOOK: Input: default@default__lineitem_ix_lineitem_ix_l_partkey_idx__ +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +2 1 +4 1 +6 1 +3 1 +1 1 +5 1 +PREHOOK: query: explain +select a, count(a) +from ( +select case L_ORDERKEY when null then 1 else 1 END as a +from lineitem_ix)tab +group by a +PREHOOK: type: QUERY +POSTHOOK: query: explain +select a, count(a) +from ( +select case L_ORDERKEY when null then 1 else 1 END as a +from lineitem_ix)tab +group by a +POSTHOOK: type: QUERY +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, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: lineitem_ix + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: CASE (l_orderkey) WHEN (null) THEN (1) ELSE (1) END (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(_col0) + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3024 Data size: 12099 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1512 Data size: 6049 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a, count(a) +from ( +select case L_ORDERKEY when null then 1 else 1 END as a +from lineitem_ix)tab +group by a +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +POSTHOOK: query: select a, count(a) +from ( +select case L_ORDERKEY when null then 1 else 1 END as a +from lineitem_ix)tab +group by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem_ix +#### A masked pattern was here #### +1 100 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/quotedid_smb.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/quotedid_smb.q.out new file mode 100644 index 0000000..7b8777f --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/quotedid_smb.q.out @@ -0,0 +1,81 @@ +PREHOOK: query: create table src_b(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_b +POSTHOOK: query: create table src_b(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_b +PREHOOK: query: insert overwrite table src_b +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@src_b +POSTHOOK: query: insert overwrite table src_b +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@src_b +POSTHOOK: Lineage: src_b.!@#$%^&*()_q SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src_b.x+1 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: create table src_b2(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_b2 +POSTHOOK: query: create table src_b2(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_b2 +PREHOOK: query: insert overwrite table src_b2 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@src_b2 +POSTHOOK: query: insert overwrite table src_b2 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@src_b2 +POSTHOOK: Lineage: src_b2.!@#$%^&*()_q SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src_b2.x+1 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: select a.`x+1`, a.`!@#$%^&*()_q`, b.`x+1`, b.`!@#$%^&*()_q` +from src_b a join src_b2 b on a.`!@#$%^&*()_q` = b.`!@#$%^&*()_q` +where a.`x+1` < '11' +PREHOOK: type: QUERY +PREHOOK: Input: default@src_b +PREHOOK: Input: default@src_b2 +#### A masked pattern was here #### +POSTHOOK: query: select a.`x+1`, a.`!@#$%^&*()_q`, b.`x+1`, b.`!@#$%^&*()_q` +from src_b a join src_b2 b on a.`!@#$%^&*()_q` = b.`!@#$%^&*()_q` +where a.`x+1` < '11' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_b +POSTHOOK: Input: default@src_b2 +#### A masked pattern was here #### +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +105 val_105 105 val_105 +10 val_10 10 val_10 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/reduce_deduplicate.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/reduce_deduplicate.q.out new file mode 100644 index 0000000..9c65350 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/reduce_deduplicate.q.out @@ -0,0 +1,479 @@ +PREHOOK: query: CREATE TABLE bucket5_1(key string, value string) CLUSTERED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket5_1 +POSTHOOK: query: CREATE TABLE bucket5_1(key string, value string) CLUSTERED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket5_1 +PREHOOK: query: explain extended +insert overwrite table bucket5_1 +select * from src cluster by key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +insert overwrite table bucket5_1 +select * from src cluster by key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + bucket5_1 + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + TOK_CLUSTERBY + TOK_TABLE_OR_COL + key + + +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 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: string) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + /src [src] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 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: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket5_1 + serialization.ddl struct bucket5_1 { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket5_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true + + Stage: Stage-0 + Move Operator + tables: + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket5_1 + serialization.ddl struct bucket5_1 { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket5_1 + + Stage: Stage-2 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table bucket5_1 +select * from src cluster by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@bucket5_1 +POSTHOOK: query: insert overwrite table bucket5_1 +select * from src cluster by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@bucket5_1 +POSTHOOK: Lineage: bucket5_1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: bucket5_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: select sum(hash(key)),sum(hash(value)) from bucket5_1 +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket5_1 +#### A masked pattern was here #### +POSTHOOK: query: select sum(hash(key)),sum(hash(value)) from bucket5_1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket5_1 +#### A masked pattern was here #### +21025334 36210398070 +PREHOOK: query: select sum(hash(key)),sum(hash(value)) from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select sum(hash(key)),sum(hash(value)) from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +21025334 36210398070 +PREHOOK: query: create table complex_tbl_1(aid string, bid string, t int, ctime string, etime bigint, l string, et string) partitioned by (ds string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_tbl_1 +POSTHOOK: query: create table complex_tbl_1(aid string, bid string, t int, ctime string, etime bigint, l string, et string) partitioned by (ds string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_tbl_1 +PREHOOK: query: create table complex_tbl_2(aet string, aes string) partitioned by (ds string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_tbl_2 +POSTHOOK: query: create table complex_tbl_2(aet string, aes string) partitioned by (ds string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_tbl_2 +PREHOOK: query: explain extended +insert overwrite table complex_tbl_1 partition (ds='2010-03-29') +select s2.* from +( + select TRANSFORM (aid,bid,t,ctime,etime,l,et) + USING 'cat' + AS (aid string, bid string, t int, ctime string, etime bigint, l string, et string) + from + ( + select transform(aet,aes) + using 'cat' + as (aid string, bid string, t int, ctime string, etime bigint, l string, et string) + from complex_tbl_2 where ds ='2010-03-29' cluster by bid +)s +)s2 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +insert overwrite table complex_tbl_1 partition (ds='2010-03-29') +select s2.* from +( + select TRANSFORM (aid,bid,t,ctime,etime,l,et) + USING 'cat' + AS (aid string, bid string, t int, ctime string, etime bigint, l string, et string) + from + ( + select transform(aet,aes) + using 'cat' + as (aid string, bid string, t int, ctime string, etime bigint, l string, et string) + from complex_tbl_2 where ds ='2010-03-29' cluster by bid +)s +)s2 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_SUBQUERY + TOK_QUERY + TOK_FROM + TOK_SUBQUERY + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + complex_tbl_2 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TRANSFORM + TOK_EXPLIST + TOK_TABLE_OR_COL + aet + TOK_TABLE_OR_COL + aes + TOK_SERDE + TOK_RECORDWRITER + 'cat' + TOK_SERDE + TOK_RECORDREADER + TOK_TABCOLLIST + TOK_TABCOL + aid + TOK_STRING + TOK_TABCOL + bid + TOK_STRING + TOK_TABCOL + t + TOK_INT + TOK_TABCOL + ctime + TOK_STRING + TOK_TABCOL + etime + TOK_BIGINT + TOK_TABCOL + l + TOK_STRING + TOK_TABCOL + et + TOK_STRING + TOK_WHERE + = + TOK_TABLE_OR_COL + ds + '2010-03-29' + TOK_CLUSTERBY + TOK_TABLE_OR_COL + bid + s + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TRANSFORM + TOK_EXPLIST + TOK_TABLE_OR_COL + aid + TOK_TABLE_OR_COL + bid + TOK_TABLE_OR_COL + t + TOK_TABLE_OR_COL + ctime + TOK_TABLE_OR_COL + etime + TOK_TABLE_OR_COL + l + TOK_TABLE_OR_COL + et + TOK_SERDE + TOK_RECORDWRITER + 'cat' + TOK_SERDE + TOK_RECORDREADER + TOK_TABCOLLIST + TOK_TABCOL + aid + TOK_STRING + TOK_TABCOL + bid + TOK_STRING + TOK_TABCOL + t + TOK_INT + TOK_TABCOL + ctime + TOK_STRING + TOK_TABCOL + etime + TOK_BIGINT + TOK_TABCOL + l + TOK_STRING + TOK_TABCOL + et + TOK_STRING + s2 + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + complex_tbl_1 + TOK_PARTSPEC + TOK_PARTVAL + ds + '2010-03-29' + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + TOK_TABNAME + s2 + + +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 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: bigint), VALUE._col5 (type: string), VALUE._col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Transform Operator + command: cat + output info: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6 + columns.types string,string,int,string,bigint,string,string + field.delim 9 + serialization.format 9 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: string), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Static Partition Specification: ds=2010-03-29/ + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: + bucket_count -1 + columns aid,bid,t,ctime,etime,l,et + columns.comments + columns.types string:string:int:string:bigint:string:string +#### A masked pattern was here #### + name default.complex_tbl_1 + partition_columns ds + partition_columns.types string + serialization.ddl struct complex_tbl_1 { string aid, string bid, i32 t, string ctime, i64 etime, string l, string et} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.complex_tbl_1 + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2010-03-29 + replace: true +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns aid,bid,t,ctime,etime,l,et + columns.comments + columns.types string:string:int:string:bigint:string:string +#### A masked pattern was here #### + name default.complex_tbl_1 + partition_columns ds + partition_columns.types string + serialization.ddl struct complex_tbl_1 { string aid, string bid, i32 t, string ctime, i64 etime, string l, string et} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.complex_tbl_1 + + Stage: Stage-2 + Stats-Aggr Operator +#### A masked pattern was here #### + diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/remote_script.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/remote_script.q.out new file mode 100644 index 0000000..6ec9949 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/remote_script.q.out @@ -0,0 +1,49 @@ +PREHOOK: query: create table tmp_tmp(key string, value string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tmp_tmp +POSTHOOK: query: create table tmp_tmp(key string, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tmp_tmp +PREHOOK: query: insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python newline.py' AS key, value FROM src limit 6 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tmp_tmp +POSTHOOK: query: insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python newline.py' AS key, value FROM src limit 6 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tmp_tmp +POSTHOOK: Lineage: tmp_tmp.key SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp_tmp.value SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: select * from tmp_tmp ORDER BY key ASC, value ASC +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp_tmp +#### A masked pattern was here #### +POSTHOOK: query: select * from tmp_tmp ORDER BY key ASC, value ASC +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp_tmp +#### A masked pattern was here #### +1 2 NULL +1 2 NULL +1 NULL +2 NULL +1 NULL +2 NULL +1 NULL +2 NULL +1 NULL +2 NULL +#### A masked pattern was here #### +PREHOOK: query: drop table tmp_tmp +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tmp_tmp +PREHOOK: Output: default@tmp_tmp +POSTHOOK: query: drop table tmp_tmp +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tmp_tmp +POSTHOOK: Output: default@tmp_tmp diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/root_dir_external_table.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/root_dir_external_table.q.out new file mode 100644 index 0000000..238a409 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/root_dir_external_table.q.out @@ -0,0 +1,26 @@ +#### A masked pattern was here #### +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@roottable +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@roottable +PREHOOK: query: select count(*) from roottable +PREHOOK: type: QUERY +PREHOOK: Input: default@roottable +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from roottable +POSTHOOK: type: QUERY +POSTHOOK: Input: default@roottable +#### A masked pattern was here #### +20 +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/schemeAuthority.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/schemeAuthority.q.out new file mode 100644 index 0000000..9a6019c --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/schemeAuthority.q.out @@ -0,0 +1,84 @@ +PREHOOK: query: create external table dynPart (key string) partitioned by (value string) row format delimited fields terminated by '\\t' stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dynPart +POSTHOOK: query: create external table dynPart (key string) partitioned by (value string) row format delimited fields terminated by '\\t' stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dynPart +#### A masked pattern was here #### +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@dynpart +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@dynpart +POSTHOOK: Output: default@dynpart@value=0 +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@dynpart +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@dynpart +POSTHOOK: Output: default@dynpart@value=1 +PREHOOK: query: select count(*) from dynPart +PREHOOK: type: QUERY +PREHOOK: Input: default@dynpart +PREHOOK: Input: default@dynpart@value=0 +PREHOOK: Input: default@dynpart@value=1 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from dynPart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dynpart +POSTHOOK: Input: default@dynpart@value=0 +POSTHOOK: Input: default@dynpart@value=1 +#### A masked pattern was here #### +2 +PREHOOK: query: select key from dynPart +PREHOOK: type: QUERY +PREHOOK: Input: default@dynpart +PREHOOK: Input: default@dynpart@value=0 +PREHOOK: Input: default@dynpart@value=1 +#### A masked pattern was here #### +POSTHOOK: query: select key from dynPart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dynpart +POSTHOOK: Input: default@dynpart@value=0 +POSTHOOK: Input: default@dynpart@value=1 +#### A masked pattern was here #### +10 +20 +PREHOOK: query: select key from src where (key = 10) order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key from src where (key = 10) order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +10 +PREHOOK: query: select key from src where (key = 20) order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key from src where (key = 20) order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +20 +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/schemeAuthority2.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/schemeAuthority2.q.out new file mode 100644 index 0000000..60913f2 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/schemeAuthority2.q.out @@ -0,0 +1,53 @@ +PREHOOK: query: create external table dynPart (key string) partitioned by (value string, value2 string) row format delimited fields terminated by '\\t' stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dynPart +POSTHOOK: query: create external table dynPart (key string) partitioned by (value string, value2 string) row format delimited fields terminated by '\\t' stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dynPart +#### A masked pattern was here #### +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@dynpart +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@dynpart +POSTHOOK: Output: default@dynpart@value=0/value2=clusterA +#### A masked pattern was here #### +PREHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +PREHOOK: Output: default@dynpart +#### A masked pattern was here #### +POSTHOOK: type: ALTERTABLE_ADDPARTS +#### A masked pattern was here #### +POSTHOOK: Output: default@dynpart +POSTHOOK: Output: default@dynpart@value=0/value2=clusterB +PREHOOK: query: select value2, key from dynPart where value='0' +PREHOOK: type: QUERY +PREHOOK: Input: default@dynpart +PREHOOK: Input: default@dynpart@value=0/value2=clusterA +PREHOOK: Input: default@dynpart@value=0/value2=clusterB +#### A masked pattern was here #### +POSTHOOK: query: select value2, key from dynPart where value='0' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dynpart +POSTHOOK: Input: default@dynpart@value=0/value2=clusterA +POSTHOOK: Input: default@dynpart@value=0/value2=clusterB +#### A masked pattern was here #### +clusterA 10 +clusterB 20 +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/scriptfile1.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/scriptfile1.q.out new file mode 100644 index 0000000..d72ff5c --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/scriptfile1.q.out @@ -0,0 +1,53 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- EXCLUDE_OS_WINDOWS +CREATE TABLE dest1(key INT, value STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- EXCLUDE_OS_WINDOWS +CREATE TABLE dest1(key INT, value STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest1 +PREHOOK: query: FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'testgrep' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dest1 +POSTHOOK: query: FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'testgrep' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dest1 +POSTHOOK: Lineage: dest1.key SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.value SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: SELECT dest1.* FROM dest1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT dest1.* FROM dest1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest1 +#### A masked pattern was here #### +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +310 val_310 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/smb_mapjoin_8.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/smb_mapjoin_8.q.out new file mode 100644 index 0000000..babaff8 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/smb_mapjoin_8.q.out @@ -0,0 +1,468 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +create table smb_bucket_input (key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket_input +POSTHOOK: query: -- SORT_QUERY_RESULTS + +create table smb_bucket_input (key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket_input +PREHOOK: query: load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@smb_bucket_input +POSTHOOK: query: load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@smb_bucket_input +PREHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket4_2 +PREHOOK: query: CREATE TABLE smb_bucket4_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: CREATE TABLE smb_bucket4_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket4_3 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4 or key=2000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4 or key=2000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=484 or key=3000 or key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=484 or key=3000 or key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +2000 val_169 NULL NULL +4 val_356 NULL NULL +4000 val_125 NULL NULL +NULL NULL 3000 val_169 +NULL NULL 484 val_169 +NULL NULL 5000 val_125 +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +2000 val_169 NULL NULL +4 val_356 NULL NULL +4000 val_125 NULL NULL +NULL NULL 3000 val_169 +NULL NULL 484 val_169 +NULL NULL 5000 val_125 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=2000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=2000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=3000 or key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=3000 or key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +2000 val_169 NULL NULL +4000 val_125 NULL NULL +NULL NULL 3000 val_169 +NULL NULL 5000 val_125 +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +2000 val_169 NULL NULL +4000 val_125 NULL NULL +NULL NULL 3000 val_169 +NULL NULL 5000 val_125 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +4000 val_125 NULL NULL +NULL NULL 5000 val_125 +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +4000 val_125 NULL NULL +NULL NULL 5000 val_125 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +1000 val_1000 1000 val_1000 +4000 val_125 NULL NULL +NULL NULL 5000 val_125 +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +#### A masked pattern was here #### +1000 val_1000 1000 val_1000 +4000 val_125 NULL NULL +NULL NULL 5000 val_125 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_3 +POSTHOOK: Lineage: smb_bucket4_3.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_3.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +1000 val_1000 1000 val_1000 1000 val_1000 +4000 val_125 NULL NULL NULL NULL +NULL NULL 5000 val_125 NULL NULL +NULL NULL NULL NULL 5000 val_125 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_3 +POSTHOOK: Lineage: smb_bucket4_3.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_3.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +1000 val_1000 1000 val_1000 1000 val_1000 +4000 val_125 NULL NULL 4000 val_125 +NULL NULL 5000 val_125 NULL NULL +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_3 +POSTHOOK: Lineage: smb_bucket4_3.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_3.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +4000 val_125 NULL NULL 4000 val_125 +NULL NULL 5000 val_125 NULL NULL +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=00000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=00000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_3 +POSTHOOK: Lineage: smb_bucket4_3.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_3.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +NULL NULL 4000 val_125 NULL NULL +NULL NULL NULL NULL 5000 val_125 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_2 +POSTHOOK: Lineage: smb_bucket4_2.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_2.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=5000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=5000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_3 +POSTHOOK: Lineage: smb_bucket4_3.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_3.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +POSTHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_3 +#### A masked pattern was here #### +1000 val_1000 NULL NULL NULL NULL +NULL NULL 4000 val_125 NULL NULL +NULL NULL NULL NULL 5000 val_125 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/stats_counter.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/stats_counter.q.out new file mode 100644 index 0000000..e2980e8 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/stats_counter.q.out @@ -0,0 +1,104 @@ +PREHOOK: query: -- by analyze +create table dummy1 as select * from src +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@dummy1 +POSTHOOK: query: -- by analyze +create table dummy1 as select * from src +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dummy1 +PREHOOK: query: analyze table dummy1 compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@dummy1 +PREHOOK: Output: default@dummy1 +POSTHOOK: query: analyze table dummy1 compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dummy1 +POSTHOOK: Output: default@dummy1 +PREHOOK: query: desc formatted dummy1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy1 +POSTHOOK: query: desc formatted dummy1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy1 +# col_name data_type comment + +key string +value string + +# 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 1 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: -- by autogather +create table dummy2 as select * from src +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@dummy2 +POSTHOOK: query: -- by autogather +create table dummy2 as select * from src +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dummy2 +PREHOOK: query: desc formatted dummy2 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy2 +POSTHOOK: query: desc formatted dummy2 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy2 +# col_name data_type comment + +key string +value string + +# 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 1 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/stats_counter_partitioned.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/stats_counter_partitioned.q.out new file mode 100644 index 0000000..4b84eca --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/stats_counter_partitioned.q.out @@ -0,0 +1,481 @@ +PREHOOK: query: -- partitioned table analyze + +create table dummy (key string, value string) partitioned by (ds string, hr string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dummy +POSTHOOK: query: -- partitioned table analyze + +create table dummy (key string, value string) partitioned by (ds string, hr string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dummy +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='12') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@dummy +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='12') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@dummy +POSTHOOK: Output: default@dummy@ds=2008/hr=12 +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='11') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@dummy +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='11') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@dummy +POSTHOOK: Output: default@dummy@ds=2008/hr=11 +PREHOOK: query: analyze table dummy partition (ds,hr) compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@dummy +PREHOOK: Input: default@dummy@ds=2008/hr=11 +PREHOOK: Input: default@dummy@ds=2008/hr=12 +PREHOOK: Output: default@dummy +PREHOOK: Output: default@dummy@ds=2008/hr=11 +PREHOOK: Output: default@dummy@ds=2008/hr=12 +POSTHOOK: query: analyze table dummy partition (ds,hr) compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dummy +POSTHOOK: Input: default@dummy@ds=2008/hr=11 +POSTHOOK: Input: default@dummy@ds=2008/hr=12 +POSTHOOK: Output: default@dummy +POSTHOOK: Output: default@dummy@ds=2008/hr=11 +POSTHOOK: Output: default@dummy@ds=2008/hr=12 +PREHOOK: query: describe formatted dummy partition (ds='2008', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (ds='2008', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008, 11] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: describe formatted dummy partition (ds='2008', hr='12') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (ds='2008', hr='12') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008, 12] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: drop table dummy +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@dummy +PREHOOK: Output: default@dummy +POSTHOOK: query: drop table dummy +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@dummy +POSTHOOK: Output: default@dummy +PREHOOK: query: -- static partitioned table on insert + +create table dummy (key string, value string) partitioned by (ds string, hr string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dummy +POSTHOOK: query: -- static partitioned table on insert + +create table dummy (key string, value string) partitioned by (ds string, hr string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dummy +PREHOOK: query: insert overwrite table dummy partition (ds='10',hr='11') select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dummy@ds=10/hr=11 +POSTHOOK: query: insert overwrite table dummy partition (ds='10',hr='11') select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dummy@ds=10/hr=11 +POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=11).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=11).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert overwrite table dummy partition (ds='10',hr='12') select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dummy@ds=10/hr=12 +POSTHOOK: query: insert overwrite table dummy partition (ds='10',hr='12') select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dummy@ds=10/hr=12 +POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=12).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=12).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: describe formatted dummy partition (ds='10', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (ds='10', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [10, 11] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: describe formatted dummy partition (ds='10', hr='12') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (ds='10', hr='12') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key string +value string + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [10, 12] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 500 + rawDataSize 5312 + totalSize 5812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: drop table dummy +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@dummy +PREHOOK: Output: default@dummy +POSTHOOK: query: drop table dummy +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@dummy +POSTHOOK: Output: default@dummy +PREHOOK: query: -- dynamic partitioned table on insert + +create table dummy (key int) partitioned by (hr int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dummy +POSTHOOK: query: -- dynamic partitioned table on insert + +create table dummy (key int) partitioned by (hr int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dummy +PREHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl +POSTHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@tbl +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@tbl +PREHOOK: query: insert overwrite table dummy partition (hr) select * from tbl +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl +PREHOOK: Output: default@dummy +POSTHOOK: query: insert overwrite table dummy partition (hr) select * from tbl +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@dummy@hr=1994 +POSTHOOK: Output: default@dummy@hr=1996 +POSTHOOK: Output: default@dummy@hr=1997 +POSTHOOK: Output: default@dummy@hr=1998 +POSTHOOK: Lineage: dummy PARTITION(hr=1994).key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dummy PARTITION(hr=1996).key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dummy PARTITION(hr=1997).key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dummy PARTITION(hr=1998).key SIMPLE [(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ] +PREHOOK: query: describe formatted dummy partition (hr=1997) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (hr=1997) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key int + +# Partition Information +# col_name data_type comment + +hr int + +# Detailed Partition Information +Partition Value: [1997] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 0 + rawDataSize 0 + totalSize 12 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: describe formatted dummy partition (hr=1994) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (hr=1994) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key int + +# Partition Information +# col_name data_type comment + +hr int + +# Detailed Partition Information +Partition Value: [1994] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 0 + rawDataSize 0 + totalSize 2 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: describe formatted dummy partition (hr=1998) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (hr=1998) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key int + +# Partition Information +# col_name data_type comment + +hr int + +# Detailed Partition Information +Partition Value: [1998] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 0 + rawDataSize 0 + totalSize 4 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: describe formatted dummy partition (hr=1996) +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@dummy +POSTHOOK: query: describe formatted dummy partition (hr=1996) +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@dummy +# col_name data_type comment + +key int + +# Partition Information +# col_name data_type comment + +hr int + +# Detailed Partition Information +Partition Value: [1996] +Database: default +Table: dummy +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE true + numFiles 1 + numRows 0 + rawDataSize 0 + totalSize 2 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: drop table tbl +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@tbl +PREHOOK: Output: default@tbl +POSTHOOK: query: drop table tbl +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@tbl +POSTHOOK: Output: default@tbl +PREHOOK: query: drop table dummy +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@dummy +PREHOOK: Output: default@dummy +POSTHOOK: query: drop table dummy +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@dummy +POSTHOOK: Output: default@dummy diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/temp_table_external.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/temp_table_external.q.out new file mode 100644 index 0000000..001cd98 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/temp_table_external.q.out @@ -0,0 +1,34 @@ +Found 1 items +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@temp_table_external +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@temp_table_external +PREHOOK: query: select * from temp_table_external +PREHOOK: type: QUERY +PREHOOK: Input: default@temp_table_external +#### A masked pattern was here #### +POSTHOOK: query: select * from temp_table_external +POSTHOOK: type: QUERY +POSTHOOK: Input: default@temp_table_external +#### A masked pattern was here #### +NULL 35 +48 NULL +100 100 +PREHOOK: query: -- Even after we drop the table, the data directory should still be there +drop table temp_table_external +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@temp_table_external +PREHOOK: Output: default@temp_table_external +POSTHOOK: query: -- Even after we drop the table, the data directory should still be there +drop table temp_table_external +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@temp_table_external +POSTHOOK: Output: default@temp_table_external +Found 1 items +#### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/truncate_column_buckets.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/truncate_column_buckets.q.out new file mode 100644 index 0000000..7147c22 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/truncate_column_buckets.q.out @@ -0,0 +1,70 @@ +PREHOOK: query: -- Tests truncating columns from a bucketed table, table should remain bucketed + +CREATE TABLE test_tab (key STRING, value STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS RCFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_tab +POSTHOOK: query: -- Tests truncating columns from a bucketed table, table should remain bucketed + +CREATE TABLE test_tab (key STRING, value STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS RCFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_tab +PREHOOK: query: INSERT OVERWRITE TABLE test_tab SELECT * FROM src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_tab +POSTHOOK: query: INSERT OVERWRITE TABLE test_tab SELECT * FROM src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_tab +POSTHOOK: Lineage: test_tab.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_tab.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- Check how many rows there are in each bucket, there should be two rows +SELECT cnt FROM ( +SELECT INPUT__FILE__NAME file_name, count(*) cnt FROM +test_tab GROUP BY INPUT__FILE__NAME +ORDER BY file_name DESC)a +PREHOOK: type: QUERY +PREHOOK: Input: default@test_tab +#### A masked pattern was here #### +POSTHOOK: query: -- Check how many rows there are in each bucket, there should be two rows +SELECT cnt FROM ( +SELECT INPUT__FILE__NAME file_name, count(*) cnt FROM +test_tab GROUP BY INPUT__FILE__NAME +ORDER BY file_name DESC)a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_tab +#### A masked pattern was here #### +258 +242 +PREHOOK: query: -- Truncate a column on which the table is not bucketed +TRUNCATE TABLE test_tab COLUMNS (value) +PREHOOK: type: TRUNCATETABLE +PREHOOK: Input: default@test_tab +PREHOOK: Output: default@test_tab +POSTHOOK: query: -- Truncate a column on which the table is not bucketed +TRUNCATE TABLE test_tab COLUMNS (value) +POSTHOOK: type: TRUNCATETABLE +POSTHOOK: Input: default@test_tab +POSTHOOK: Output: default@test_tab +PREHOOK: query: -- Check how many rows there are in each bucket, this should produce the same rows as before +-- because truncate should not break bucketing +SELECT cnt FROM ( +SELECT INPUT__FILE__NAME file_name, count(*) cnt FROM +test_tab GROUP BY INPUT__FILE__NAME +ORDER BY file_name DESC)a +PREHOOK: type: QUERY +PREHOOK: Input: default@test_tab +#### A masked pattern was here #### +POSTHOOK: query: -- Check how many rows there are in each bucket, this should produce the same rows as before +-- because truncate should not break bucketing +SELECT cnt FROM ( +SELECT INPUT__FILE__NAME file_name, count(*) cnt FROM +test_tab GROUP BY INPUT__FILE__NAME +ORDER BY file_name DESC)a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_tab +#### A masked pattern was here #### +258 +242 diff --git ql/src/test/results/clientpositive/miniSparkOnYarn/uber_reduce.q.out ql/src/test/results/clientpositive/miniSparkOnYarn/uber_reduce.q.out new file mode 100644 index 0000000..2a29131 --- /dev/null +++ ql/src/test/results/clientpositive/miniSparkOnYarn/uber_reduce.q.out @@ -0,0 +1,31 @@ +PREHOOK: query: -- Uberized mode is a YARN option, ignore this test for non-YARN Hadoop versions +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20S) + +CREATE TABLE T1(key STRING, val STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T1 +POSTHOOK: query: -- Uberized mode is a YARN option, ignore this test for non-YARN Hadoop versions +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20S) + +CREATE TABLE T1(key STRING, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: SELECT count(*) FROM T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT count(*) FROM T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +6 diff --git shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java index b17f465..65ef211 100644 --- shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java +++ shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java @@ -232,6 +232,12 @@ public MiniMrShim getMiniTezCluster(Configuration conf, int numberOfTaskTrackers throw new IOException("Cannot run tez on current hadoop, Version: " + VersionInfo.getVersion()); } + @Override + public MiniMrShim getMiniSparkCluster(Configuration conf, int numberOfTaskTrackers, + String nameNode, int numDir) throws IOException { + throw new IOException("Cannot run Spark on YARN on current Hadoop, Version: " + VersionInfo.getVersion()); + } + /** * Shim for MiniMrCluster */ diff --git shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java index a61c3ac..17799e9 100644 --- shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java +++ shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java @@ -415,6 +415,73 @@ public void setupConfiguration(Configuration conf) { } } + /** + * Returns a shim to wrap MiniSparkOnYARNCluster + */ + @Override + public MiniMrShim getMiniSparkCluster(Configuration conf, int numberOfTaskTrackers, + String nameNode, int numDir) throws IOException { + return new MiniSparkShim(conf, numberOfTaskTrackers, nameNode, numDir); + } + + /** + * Shim for MiniSparkOnYARNCluster + */ + public class MiniSparkShim extends Hadoop23Shims.MiniMrShim { + + private final MiniSparkOnYARNCluster mr; + private final Configuration conf; + + public MiniSparkShim(Configuration conf, int numberOfTaskTrackers, + String nameNode, int numDir) throws IOException { + + mr = new MiniSparkOnYARNCluster("sparkOnYarn"); + conf.set("fs.defaultFS", nameNode); + mr.init(conf); + mr.start(); + this.conf = mr.getConfig(); + } + + @Override + public int getJobTrackerPort() throws UnsupportedOperationException { + String address = conf.get("yarn.resourcemanager.address"); + address = StringUtils.substringAfterLast(address, ":"); + + if (StringUtils.isBlank(address)) { + throw new IllegalArgumentException("Invalid YARN resource manager port."); + } + + return Integer.parseInt(address); + } + + @Override + public void shutdown() throws IOException { + mr.stop(); + } + + @Override + public void setupConfiguration(Configuration conf) { + Configuration config = mr.getConfig(); + for (Map.Entry pair : config) { + conf.set(pair.getKey(), pair.getValue()); + } + + Path jarPath = new Path("hdfs:///user/hive"); + Path hdfsPath = new Path("hdfs:///user/"); + try { + FileSystem fs = cluster.getFileSystem(); + jarPath = fs.makeQualified(jarPath); + conf.set("hive.jar.directory", jarPath.toString()); + fs.mkdirs(jarPath); + hdfsPath = fs.makeQualified(hdfsPath); + conf.set("hive.user.install.directory", hdfsPath.toString()); + fs.mkdirs(hdfsPath); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + // Don't move this code to the parent class. There's a binary // incompatibility between hadoop 1 and 2 wrt MiniDFSCluster and we // need to have two different shim classes even though they are diff --git shims/0.23/src/main/java/org/apache/hadoop/hive/shims/MiniSparkOnYARNCluster.java shims/0.23/src/main/java/org/apache/hadoop/hive/shims/MiniSparkOnYARNCluster.java new file mode 100644 index 0000000..45358f8 --- /dev/null +++ shims/0.23/src/main/java/org/apache/hadoop/hive/shims/MiniSparkOnYARNCluster.java @@ -0,0 +1,19 @@ +package org.apache.hadoop.hive.shims; + +import org.apache.hadoop.yarn.server.MiniYARNCluster; + +public class MiniSparkOnYARNCluster extends MiniYARNCluster { + + public MiniSparkOnYARNCluster(String testName) { + this(testName, 1, 1); + } + + public MiniSparkOnYARNCluster(String testName, int numResourceManagers, int numNodeManagers) { + this(testName, numResourceManagers, numNodeManagers, 1, 1); + } + + public MiniSparkOnYARNCluster(String testName, int numResourceManagers, + int numNodeManagers, int numLocalDirs, int numLogDirs) { + super(testName, numResourceManagers, numNodeManagers, numLocalDirs, numLogDirs); + } +} \ No newline at end of file diff --git shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java index 064304c..cdf3ea9 100644 --- shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java +++ shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java @@ -93,7 +93,10 @@ public MiniMrShim getMiniMrCluster(Configuration conf, int numberOfTaskTrackers, String nameNode, int numDir) throws IOException; public MiniMrShim getMiniTezCluster(Configuration conf, int numberOfTaskTrackers, - String nameNode, int numDir) throws IOException; + String nameNode, int numDir) throws IOException; + + public MiniMrShim getMiniSparkCluster(Configuration conf, int numberOfTaskTrackers, + String nameNode, int numDir) throws IOException; /** * Shim for MiniMrCluster diff --git spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java index aea90db..89cbdfb 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java @@ -350,6 +350,10 @@ public void run() { LOG.debug("Running client driver with argv: {}", Joiner.on(" ").join(argv)); ProcessBuilder pb = new ProcessBuilder(argv.toArray(new String[argv.size()])); + String isTesting = conf.get("spark.testing"); + if (isTesting != null) { + pb.environment().put("SPARK_TESTING", isTesting); + } final Process child = pb.start(); int childId = childIdGenerator.incrementAndGet();