Index: . =================================================================== --- . (revision 1674118) +++ . (working copy) Property changes on: . ___________________________________________________________________ Modified: svn:mergeinfo Merged /hive/branches/spark:r1660299-1673960 Index: bin/hive =================================================================== --- bin/hive (revision 1674118) +++ bin/hive (working copy) @@ -110,6 +110,13 @@ CLASSPATH=${CLASSPATH}:$f; done +# add Spark assembly jar to the classpath +if [[ -n "$SPARK_HOME" ]] +then + sparkAssemblyPath=`ls ${SPARK_HOME}/lib/spark-assembly-*.jar` + CLASSPATH="${CLASSPATH}:${sparkAssemblyPath}" +fi + # add the auxillary jars such as serdes if [ -d "${HIVE_AUX_JARS_PATH}" ]; then hive_aux_jars_abspath=`cd ${HIVE_AUX_JARS_PATH} && pwd` Index: hbase-handler/pom.xml =================================================================== --- hbase-handler/pom.xml (revision 1674118) +++ hbase-handler/pom.xml (working copy) Property changes on: hbase-handler/pom.xml ___________________________________________________________________ Modified: svn:mergeinfo Merged /hive/branches/spark/hbase-handler/pom.xml:r1660299-1673960 Index: itests/src/test/resources/testconfiguration.properties =================================================================== --- itests/src/test/resources/testconfiguration.properties (revision 1674118) +++ itests/src/test/resources/testconfiguration.properties (working copy) @@ -590,7 +590,16 @@ bucketsortoptimize_insert_6.q, \ bucketsortoptimize_insert_7.q, \ bucketsortoptimize_insert_8.q, \ + cbo_gby.q, \ cbo_gby_empty.q, \ + cbo_limit.q, \ + cbo_semijoin.q, \ + cbo_simple_select.q, \ + cbo_stats.q, \ + cbo_subq_in.q, \ + cbo_subq_not_in.q, \ + cbo_udf_udaf.q, \ + cbo_union.q, \ column_access_stats.q, \ count.q, \ create_merge_compressed.q, \ @@ -819,6 +828,7 @@ rcfile_bigdata.q, \ reduce_deduplicate_exclude_join.q, \ router_join_ppr.q, \ + runtime_skewjoin_mapjoin_spark.q, \ sample1.q, \ sample10.q, \ sample2.q, \ @@ -930,31 +940,48 @@ union.q, \ union10.q, \ union11.q, \ + union12.q, \ union13.q, \ union14.q, \ union15.q, \ union16.q, \ + union17.q, \ union18.q, \ union19.q, \ union2.q, \ + union20.q, \ + union21.q, \ + union22.q, \ union23.q, \ + union24.q, \ union25.q, \ + union26.q, \ + union27.q, \ union28.q, \ union29.q, \ union3.q, \ union30.q, \ + union31.q, \ + union32.q, \ union33.q, \ + union34.q, \ union4.q, \ union5.q, \ union6.q, \ union7.q, \ union8.q, \ union9.q, \ + union_date.q, \ + union_date_trim.q, \ + union_lateralview.q, \ union_null.q, \ union_ppr.q, \ union_remove_1.q, \ union_remove_10.q, \ union_remove_11.q, \ + union_remove_12.q, \ + union_remove_13.q, \ + union_remove_14.q, \ union_remove_15.q, \ union_remove_16.q, \ union_remove_17.q, \ @@ -963,15 +990,20 @@ union_remove_2.q, \ union_remove_20.q, \ union_remove_21.q, \ + union_remove_22.q, \ + union_remove_23.q, \ union_remove_24.q, \ union_remove_25.q, \ union_remove_3.q, \ union_remove_4.q, \ union_remove_5.q, \ union_remove_6.q, \ + union_remove_6_subq.q, \ union_remove_7.q, \ union_remove_8.q, \ union_remove_9.q, \ + union_script.q, \ + union_top_level.q, \ uniquejoin.q, \ union_view.q, \ varchar_join1.q, \ Index: itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java =================================================================== --- itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (revision 1674118) +++ itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (working copy) @@ -913,13 +913,15 @@ long endTime = System.currentTimeMillis() + 240000; while (sparkSession.getMemoryAndCores().getSecond() <= 1) { if (System.currentTimeMillis() >= endTime) { - LOG.error("Timed out waiting for Spark cluster to init"); - break; + String msg = "Timed out waiting for Spark cluster to init"; + throw new IllegalStateException(msg); } Thread.sleep(100); } } catch (Exception e) { - LOG.error(e); + String msg = "Error trying to obtain executor info: " + e; + LOG.error(msg, e); + throw new IllegalStateException(msg, e); } } } Index: metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java =================================================================== --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java (revision 1674118) +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java (working copy) @@ -716,7 +716,7 @@ struct.partitionVals = new ArrayList(_list516.size); for (int _i517 = 0; _i517 < _list516.size; ++_i517) { - String _elem518; // optional + String _elem518; // required _elem518 = iprot.readString(); struct.partitionVals.add(_elem518); } @@ -846,7 +846,7 @@ struct.partitionVals = new ArrayList(_list521.size); for (int _i522 = 0; _i522 < _list521.size; ++_i522) { - String _elem523; // optional + String _elem523; // required _elem523 = iprot.readString(); struct.partitionVals.add(_elem523); } Index: metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java =================================================================== --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java (revision 1674118) +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java (working copy) @@ -354,7 +354,7 @@ struct.filesAdded = new ArrayList(_list508.size); for (int _i509 = 0; _i509 < _list508.size; ++_i509) { - String _elem510; // optional + String _elem510; // required _elem510 = iprot.readString(); struct.filesAdded.add(_elem510); } @@ -424,7 +424,7 @@ struct.filesAdded = new ArrayList(_list513.size); for (int _i514 = 0; _i514 < _list513.size; ++_i514) { - String _elem515; // optional + String _elem515; // required _elem515 = iprot.readString(); struct.filesAdded.add(_elem515); } Index: metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java =================================================================== --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java (revision 1674118) +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java (working copy) @@ -354,7 +354,7 @@ struct.events = new ArrayList(_list500.size); for (int _i501 = 0; _i501 < _list500.size; ++_i501) { - NotificationEvent _elem502; // optional + NotificationEvent _elem502; // required _elem502 = new NotificationEvent(); _elem502.read(iprot); struct.events.add(_elem502); @@ -425,7 +425,7 @@ struct.events = new ArrayList(_list505.size); for (int _i506 = 0; _i506 < _list505.size; ++_i506) { - NotificationEvent _elem507; // optional + NotificationEvent _elem507; // required _elem507 = new NotificationEvent(); _elem507.read(iprot); struct.events.add(_elem507); Index: metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java =================================================================== --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java (revision 1674118) +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java (working copy) @@ -17328,7 +17328,7 @@ struct.success = new ArrayList(_list524.size); for (int _i525 = 0; _i525 < _list524.size; ++_i525) { - String _elem526; // optional + String _elem526; // required _elem526 = iprot.readString(); struct.success.add(_elem526); } @@ -17427,7 +17427,7 @@ struct.success = new ArrayList(_list529.size); for (int _i530 = 0; _i530 < _list529.size; ++_i530) { - String _elem531; // optional + String _elem531; // required _elem531 = iprot.readString(); struct.success.add(_elem531); } @@ -18090,7 +18090,7 @@ struct.success = new ArrayList(_list532.size); for (int _i533 = 0; _i533 < _list532.size; ++_i533) { - String _elem534; // optional + String _elem534; // required _elem534 = iprot.readString(); struct.success.add(_elem534); } @@ -18189,7 +18189,7 @@ struct.success = new ArrayList(_list537.size); for (int _i538 = 0; _i538 < _list537.size; ++_i538) { - String _elem539; // optional + String _elem539; // required _elem539 = iprot.readString(); struct.success.add(_elem539); } @@ -23950,7 +23950,7 @@ struct.success = new ArrayList(_list550.size); for (int _i551 = 0; _i551 < _list550.size; ++_i551) { - FieldSchema _elem552; // optional + FieldSchema _elem552; // required _elem552 = new FieldSchema(); _elem552.read(iprot); struct.success.add(_elem552); @@ -24090,7 +24090,7 @@ struct.success = new ArrayList(_list555.size); for (int _i556 = 0; _i556 < _list555.size; ++_i556) { - FieldSchema _elem557; // optional + FieldSchema _elem557; // required _elem557 = new FieldSchema(); _elem557.read(iprot); struct.success.add(_elem557); @@ -25251,7 +25251,7 @@ struct.success = new ArrayList(_list558.size); for (int _i559 = 0; _i559 < _list558.size; ++_i559) { - FieldSchema _elem560; // optional + FieldSchema _elem560; // required _elem560 = new FieldSchema(); _elem560.read(iprot); struct.success.add(_elem560); @@ -25391,7 +25391,7 @@ struct.success = new ArrayList(_list563.size); for (int _i564 = 0; _i564 < _list563.size; ++_i564) { - FieldSchema _elem565; // optional + FieldSchema _elem565; // required _elem565 = new FieldSchema(); _elem565.read(iprot); struct.success.add(_elem565); @@ -26443,7 +26443,7 @@ struct.success = new ArrayList(_list566.size); for (int _i567 = 0; _i567 < _list566.size; ++_i567) { - FieldSchema _elem568; // optional + FieldSchema _elem568; // required _elem568 = new FieldSchema(); _elem568.read(iprot); struct.success.add(_elem568); @@ -26583,7 +26583,7 @@ struct.success = new ArrayList(_list571.size); for (int _i572 = 0; _i572 < _list571.size; ++_i572) { - FieldSchema _elem573; // optional + FieldSchema _elem573; // required _elem573 = new FieldSchema(); _elem573.read(iprot); struct.success.add(_elem573); @@ -27744,7 +27744,7 @@ struct.success = new ArrayList(_list574.size); for (int _i575 = 0; _i575 < _list574.size; ++_i575) { - FieldSchema _elem576; // optional + FieldSchema _elem576; // required _elem576 = new FieldSchema(); _elem576.read(iprot); struct.success.add(_elem576); @@ -27884,7 +27884,7 @@ struct.success = new ArrayList(_list579.size); for (int _i580 = 0; _i580 < _list579.size; ++_i580) { - FieldSchema _elem581; // optional + FieldSchema _elem581; // required _elem581 = new FieldSchema(); _elem581.read(iprot); struct.success.add(_elem581); @@ -33134,7 +33134,7 @@ struct.success = new ArrayList(_list582.size); for (int _i583 = 0; _i583 < _list582.size; ++_i583) { - String _elem584; // optional + String _elem584; // required _elem584 = iprot.readString(); struct.success.add(_elem584); } @@ -33233,7 +33233,7 @@ struct.success = new ArrayList(_list587.size); for (int _i588 = 0; _i588 < _list587.size; ++_i588) { - String _elem589; // optional + String _elem589; // required _elem589 = iprot.readString(); struct.success.add(_elem589); } @@ -34008,7 +34008,7 @@ struct.success = new ArrayList(_list590.size); for (int _i591 = 0; _i591 < _list590.size; ++_i591) { - String _elem592; // optional + String _elem592; // required _elem592 = iprot.readString(); struct.success.add(_elem592); } @@ -34107,7 +34107,7 @@ struct.success = new ArrayList(_list595.size); for (int _i596 = 0; _i596 < _list595.size; ++_i596) { - String _elem597; // optional + String _elem597; // required _elem597 = iprot.readString(); struct.success.add(_elem597); } @@ -35569,7 +35569,7 @@ struct.tbl_names = new ArrayList(_list598.size); for (int _i599 = 0; _i599 < _list598.size; ++_i599) { - String _elem600; // optional + String _elem600; // required _elem600 = iprot.readString(); struct.tbl_names.add(_elem600); } @@ -35663,7 +35663,7 @@ struct.tbl_names = new ArrayList(_list603.size); for (int _i604 = 0; _i604 < _list603.size; ++_i604) { - String _elem605; // optional + String _elem605; // required _elem605 = iprot.readString(); struct.tbl_names.add(_elem605); } @@ -36237,7 +36237,7 @@ struct.success = new ArrayList(_list606.size); for (int _i607 = 0; _i607 < _list606.size; ++_i607) { - Table _elem608; // optional + Table _elem608; // required _elem608 = new Table(); _elem608.read(iprot); struct.success.add(_elem608); @@ -36377,7 +36377,7 @@ struct.success = new ArrayList
(_list611.size); for (int _i612 = 0; _i612 < _list611.size; ++_i612) { - Table _elem613; // optional + Table _elem613; // required _elem613 = new Table(); _elem613.read(iprot); struct.success.add(_elem613); @@ -37533,7 +37533,7 @@ struct.success = new ArrayList(_list614.size); for (int _i615 = 0; _i615 < _list614.size; ++_i615) { - String _elem616; // optional + String _elem616; // required _elem616 = iprot.readString(); struct.success.add(_elem616); } @@ -37672,7 +37672,7 @@ struct.success = new ArrayList(_list619.size); for (int _i620 = 0; _i620 < _list619.size; ++_i620) { - String _elem621; // optional + String _elem621; // required _elem621 = iprot.readString(); struct.success.add(_elem621); } @@ -43537,7 +43537,7 @@ struct.new_parts = new ArrayList(_list622.size); for (int _i623 = 0; _i623 < _list622.size; ++_i623) { - Partition _elem624; // optional + Partition _elem624; // required _elem624 = new Partition(); _elem624.read(iprot); struct.new_parts.add(_elem624); @@ -43617,7 +43617,7 @@ struct.new_parts = new ArrayList(_list627.size); for (int _i628 = 0; _i628 < _list627.size; ++_i628) { - Partition _elem629; // optional + Partition _elem629; // required _elem629 = new Partition(); _elem629.read(iprot); struct.new_parts.add(_elem629); @@ -44625,7 +44625,7 @@ struct.new_parts = new ArrayList(_list630.size); for (int _i631 = 0; _i631 < _list630.size; ++_i631) { - PartitionSpec _elem632; // optional + PartitionSpec _elem632; // required _elem632 = new PartitionSpec(); _elem632.read(iprot); struct.new_parts.add(_elem632); @@ -44705,7 +44705,7 @@ struct.new_parts = new ArrayList(_list635.size); for (int _i636 = 0; _i636 < _list635.size; ++_i636) { - PartitionSpec _elem637; // optional + PartitionSpec _elem637; // required _elem637 = new PartitionSpec(); _elem637.read(iprot); struct.new_parts.add(_elem637); @@ -45891,7 +45891,7 @@ struct.part_vals = new ArrayList(_list638.size); for (int _i639 = 0; _i639 < _list638.size; ++_i639) { - String _elem640; // optional + String _elem640; // required _elem640 = iprot.readString(); struct.part_vals.add(_elem640); } @@ -46000,7 +46000,7 @@ struct.part_vals = new ArrayList(_list643.size); for (int _i644 = 0; _i644 < _list643.size; ++_i644) { - String _elem645; // optional + String _elem645; // required _elem645 = iprot.readString(); struct.part_vals.add(_elem645); } @@ -48318,7 +48318,7 @@ struct.part_vals = new ArrayList(_list646.size); for (int _i647 = 0; _i647 < _list646.size; ++_i647) { - String _elem648; // optional + String _elem648; // required _elem648 = iprot.readString(); struct.part_vals.add(_elem648); } @@ -48447,7 +48447,7 @@ struct.part_vals = new ArrayList(_list651.size); for (int _i652 = 0; _i652 < _list651.size; ++_i652) { - String _elem653; // optional + String _elem653; // required _elem653 = iprot.readString(); struct.part_vals.add(_elem653); } @@ -52326,7 +52326,7 @@ struct.part_vals = new ArrayList(_list654.size); for (int _i655 = 0; _i655 < _list654.size; ++_i655) { - String _elem656; // optional + String _elem656; // required _elem656 = iprot.readString(); struct.part_vals.add(_elem656); } @@ -52452,7 +52452,7 @@ struct.part_vals = new ArrayList(_list659.size); for (int _i660 = 0; _i660 < _list659.size; ++_i660) { - String _elem661; // optional + String _elem661; // required _elem661 = iprot.readString(); struct.part_vals.add(_elem661); } @@ -53700,7 +53700,7 @@ struct.part_vals = new ArrayList(_list662.size); for (int _i663 = 0; _i663 < _list662.size; ++_i663) { - String _elem664; // optional + String _elem664; // required _elem664 = iprot.readString(); struct.part_vals.add(_elem664); } @@ -53846,7 +53846,7 @@ struct.part_vals = new ArrayList(_list667.size); for (int _i668 = 0; _i668 < _list667.size; ++_i668) { - String _elem669; // optional + String _elem669; // required _elem669 = iprot.readString(); struct.part_vals.add(_elem669); } @@ -58457,7 +58457,7 @@ struct.part_vals = new ArrayList(_list670.size); for (int _i671 = 0; _i671 < _list670.size; ++_i671) { - String _elem672; // optional + String _elem672; // required _elem672 = iprot.readString(); struct.part_vals.add(_elem672); } @@ -58566,7 +58566,7 @@ struct.part_vals = new ArrayList(_list675.size); for (int _i676 = 0; _i676 < _list675.size; ++_i676) { - String _elem677; // optional + String _elem677; // required _elem677 = iprot.readString(); struct.part_vals.add(_elem677); } @@ -61456,7 +61456,7 @@ struct.part_vals = new ArrayList(_list688.size); for (int _i689 = 0; _i689 < _list688.size; ++_i689) { - String _elem690; // optional + String _elem690; // required _elem690 = iprot.readString(); struct.part_vals.add(_elem690); } @@ -61482,7 +61482,7 @@ struct.group_names = new ArrayList(_list691.size); for (int _i692 = 0; _i692 < _list691.size; ++_i692) { - String _elem693; // optional + String _elem693; // required _elem693 = iprot.readString(); struct.group_names.add(_elem693); } @@ -61626,7 +61626,7 @@ struct.part_vals = new ArrayList(_list698.size); for (int _i699 = 0; _i699 < _list698.size; ++_i699) { - String _elem700; // optional + String _elem700; // required _elem700 = iprot.readString(); struct.part_vals.add(_elem700); } @@ -61643,7 +61643,7 @@ struct.group_names = new ArrayList(_list701.size); for (int _i702 = 0; _i702 < _list701.size; ++_i702) { - String _elem703; // optional + String _elem703; // required _elem703 = iprot.readString(); struct.group_names.add(_elem703); } @@ -64418,7 +64418,7 @@ struct.success = new ArrayList(_list704.size); for (int _i705 = 0; _i705 < _list704.size; ++_i705) { - Partition _elem706; // optional + Partition _elem706; // required _elem706 = new Partition(); _elem706.read(iprot); struct.success.add(_elem706); @@ -64538,7 +64538,7 @@ struct.success = new ArrayList(_list709.size); for (int _i710 = 0; _i710 < _list709.size; ++_i710) { - Partition _elem711; // optional + Partition _elem711; // required _elem711 = new Partition(); _elem711.read(iprot); struct.success.add(_elem711); @@ -65238,7 +65238,7 @@ struct.group_names = new ArrayList(_list712.size); for (int _i713 = 0; _i713 < _list712.size; ++_i713) { - String _elem714; // optional + String _elem714; // required _elem714 = iprot.readString(); struct.group_names.add(_elem714); } @@ -65375,7 +65375,7 @@ struct.group_names = new ArrayList(_list717.size); for (int _i718 = 0; _i718 < _list717.size; ++_i718) { - String _elem719; // optional + String _elem719; // required _elem719 = iprot.readString(); struct.group_names.add(_elem719); } @@ -65868,7 +65868,7 @@ struct.success = new ArrayList(_list720.size); for (int _i721 = 0; _i721 < _list720.size; ++_i721) { - Partition _elem722; // optional + Partition _elem722; // required _elem722 = new Partition(); _elem722.read(iprot); struct.success.add(_elem722); @@ -65988,7 +65988,7 @@ struct.success = new ArrayList(_list725.size); for (int _i726 = 0; _i726 < _list725.size; ++_i726) { - Partition _elem727; // optional + Partition _elem727; // required _elem727 = new Partition(); _elem727.read(iprot); struct.success.add(_elem727); @@ -67058,7 +67058,7 @@ struct.success = new ArrayList(_list728.size); for (int _i729 = 0; _i729 < _list728.size; ++_i729) { - PartitionSpec _elem730; // optional + PartitionSpec _elem730; // required _elem730 = new PartitionSpec(); _elem730.read(iprot); struct.success.add(_elem730); @@ -67178,7 +67178,7 @@ struct.success = new ArrayList(_list733.size); for (int _i734 = 0; _i734 < _list733.size; ++_i734) { - PartitionSpec _elem735; // optional + PartitionSpec _elem735; // required _elem735 = new PartitionSpec(); _elem735.read(iprot); struct.success.add(_elem735); @@ -68167,7 +68167,7 @@ struct.success = new ArrayList(_list736.size); for (int _i737 = 0; _i737 < _list736.size; ++_i737) { - String _elem738; // optional + String _elem738; // required _elem738 = iprot.readString(); struct.success.add(_elem738); } @@ -68266,7 +68266,7 @@ struct.success = new ArrayList(_list741.size); for (int _i742 = 0; _i742 < _list741.size; ++_i742) { - String _elem743; // optional + String _elem743; // required _elem743 = iprot.readString(); struct.success.add(_elem743); } @@ -68863,7 +68863,7 @@ struct.part_vals = new ArrayList(_list744.size); for (int _i745 = 0; _i745 < _list744.size; ++_i745) { - String _elem746; // optional + String _elem746; // required _elem746 = iprot.readString(); struct.part_vals.add(_elem746); } @@ -68989,7 +68989,7 @@ struct.part_vals = new ArrayList(_list749.size); for (int _i750 = 0; _i750 < _list749.size; ++_i750) { - String _elem751; // optional + String _elem751; // required _elem751 = iprot.readString(); struct.part_vals.add(_elem751); } @@ -69486,7 +69486,7 @@ struct.success = new ArrayList(_list752.size); for (int _i753 = 0; _i753 < _list752.size; ++_i753) { - Partition _elem754; // optional + Partition _elem754; // required _elem754 = new Partition(); _elem754.read(iprot); struct.success.add(_elem754); @@ -69606,7 +69606,7 @@ struct.success = new ArrayList(_list757.size); for (int _i758 = 0; _i758 < _list757.size; ++_i758) { - Partition _elem759; // optional + Partition _elem759; // required _elem759 = new Partition(); _elem759.read(iprot); struct.success.add(_elem759); @@ -70391,7 +70391,7 @@ struct.part_vals = new ArrayList(_list760.size); for (int _i761 = 0; _i761 < _list760.size; ++_i761) { - String _elem762; // optional + String _elem762; // required _elem762 = iprot.readString(); struct.part_vals.add(_elem762); } @@ -70425,7 +70425,7 @@ struct.group_names = new ArrayList(_list763.size); for (int _i764 = 0; _i764 < _list763.size; ++_i764) { - String _elem765; // optional + String _elem765; // required _elem765 = iprot.readString(); struct.group_names.add(_elem765); } @@ -70578,7 +70578,7 @@ struct.part_vals = new ArrayList(_list770.size); for (int _i771 = 0; _i771 < _list770.size; ++_i771) { - String _elem772; // optional + String _elem772; // required _elem772 = iprot.readString(); struct.part_vals.add(_elem772); } @@ -70599,7 +70599,7 @@ struct.group_names = new ArrayList(_list773.size); for (int _i774 = 0; _i774 < _list773.size; ++_i774) { - String _elem775; // optional + String _elem775; // required _elem775 = iprot.readString(); struct.group_names.add(_elem775); } @@ -71092,7 +71092,7 @@ struct.success = new ArrayList(_list776.size); for (int _i777 = 0; _i777 < _list776.size; ++_i777) { - Partition _elem778; // optional + Partition _elem778; // required _elem778 = new Partition(); _elem778.read(iprot); struct.success.add(_elem778); @@ -71212,7 +71212,7 @@ struct.success = new ArrayList(_list781.size); for (int _i782 = 0; _i782 < _list781.size; ++_i782) { - Partition _elem783; // optional + Partition _elem783; // required _elem783 = new Partition(); _elem783.read(iprot); struct.success.add(_elem783); @@ -71815,7 +71815,7 @@ struct.part_vals = new ArrayList(_list784.size); for (int _i785 = 0; _i785 < _list784.size; ++_i785) { - String _elem786; // optional + String _elem786; // required _elem786 = iprot.readString(); struct.part_vals.add(_elem786); } @@ -71941,7 +71941,7 @@ struct.part_vals = new ArrayList(_list789.size); for (int _i790 = 0; _i790 < _list789.size; ++_i790) { - String _elem791; // optional + String _elem791; // required _elem791 = iprot.readString(); struct.part_vals.add(_elem791); } @@ -72438,7 +72438,7 @@ struct.success = new ArrayList(_list792.size); for (int _i793 = 0; _i793 < _list792.size; ++_i793) { - String _elem794; // optional + String _elem794; // required _elem794 = iprot.readString(); struct.success.add(_elem794); } @@ -72557,7 +72557,7 @@ struct.success = new ArrayList(_list797.size); for (int _i798 = 0; _i798 < _list797.size; ++_i798) { - String _elem799; // optional + String _elem799; // required _elem799 = iprot.readString(); struct.success.add(_elem799); } @@ -73730,7 +73730,7 @@ struct.success = new ArrayList(_list800.size); for (int _i801 = 0; _i801 < _list800.size; ++_i801) { - Partition _elem802; // optional + Partition _elem802; // required _elem802 = new Partition(); _elem802.read(iprot); struct.success.add(_elem802); @@ -73850,7 +73850,7 @@ struct.success = new ArrayList(_list805.size); for (int _i806 = 0; _i806 < _list805.size; ++_i806) { - Partition _elem807; // optional + Partition _elem807; // required _elem807 = new Partition(); _elem807.read(iprot); struct.success.add(_elem807); @@ -75024,7 +75024,7 @@ struct.success = new ArrayList(_list808.size); for (int _i809 = 0; _i809 < _list808.size; ++_i809) { - PartitionSpec _elem810; // optional + PartitionSpec _elem810; // required _elem810 = new PartitionSpec(); _elem810.read(iprot); struct.success.add(_elem810); @@ -75144,7 +75144,7 @@ struct.success = new ArrayList(_list813.size); for (int _i814 = 0; _i814 < _list813.size; ++_i814) { - PartitionSpec _elem815; // optional + PartitionSpec _elem815; // required _elem815 = new PartitionSpec(); _elem815.read(iprot); struct.success.add(_elem815); @@ -76602,7 +76602,7 @@ struct.names = new ArrayList(_list816.size); for (int _i817 = 0; _i817 < _list816.size; ++_i817) { - String _elem818; // optional + String _elem818; // required _elem818 = iprot.readString(); struct.names.add(_elem818); } @@ -76711,7 +76711,7 @@ struct.names = new ArrayList(_list821.size); for (int _i822 = 0; _i822 < _list821.size; ++_i822) { - String _elem823; // optional + String _elem823; // required _elem823 = iprot.readString(); struct.names.add(_elem823); } @@ -77204,7 +77204,7 @@ struct.success = new ArrayList(_list824.size); for (int _i825 = 0; _i825 < _list824.size; ++_i825) { - Partition _elem826; // optional + Partition _elem826; // required _elem826 = new Partition(); _elem826.read(iprot); struct.success.add(_elem826); @@ -77324,7 +77324,7 @@ struct.success = new ArrayList(_list829.size); for (int _i830 = 0; _i830 < _list829.size; ++_i830) { - Partition _elem831; // optional + Partition _elem831; // required _elem831 = new Partition(); _elem831.read(iprot); struct.success.add(_elem831); @@ -78881,7 +78881,7 @@ struct.new_parts = new ArrayList(_list832.size); for (int _i833 = 0; _i833 < _list832.size; ++_i833) { - Partition _elem834; // optional + Partition _elem834; // required _elem834 = new Partition(); _elem834.read(iprot); struct.new_parts.add(_elem834); @@ -78991,7 +78991,7 @@ struct.new_parts = new ArrayList(_list837.size); for (int _i838 = 0; _i838 < _list837.size; ++_i838) { - Partition _elem839; // optional + Partition _elem839; // required _elem839 = new Partition(); _elem839.read(iprot); struct.new_parts.add(_elem839); @@ -81197,7 +81197,7 @@ struct.part_vals = new ArrayList(_list840.size); for (int _i841 = 0; _i841 < _list840.size; ++_i841) { - String _elem842; // optional + String _elem842; // required _elem842 = iprot.readString(); struct.part_vals.add(_elem842); } @@ -81326,7 +81326,7 @@ struct.part_vals = new ArrayList(_list845.size); for (int _i846 = 0; _i846 < _list845.size; ++_i846) { - String _elem847; // optional + String _elem847; // required _elem847 = iprot.readString(); struct.part_vals.add(_elem847); } @@ -82209,7 +82209,7 @@ struct.part_vals = new ArrayList(_list848.size); for (int _i849 = 0; _i849 < _list848.size; ++_i849) { - String _elem850; // optional + String _elem850; // required _elem850 = iprot.readString(); struct.part_vals.add(_elem850); } @@ -82305,7 +82305,7 @@ struct.part_vals = new ArrayList(_list853.size); for (int _i854 = 0; _i854 < _list853.size; ++_i854) { - String _elem855; // optional + String _elem855; // required _elem855 = iprot.readString(); struct.part_vals.add(_elem855); } @@ -84469,7 +84469,7 @@ struct.success = new ArrayList(_list856.size); for (int _i857 = 0; _i857 < _list856.size; ++_i857) { - String _elem858; // optional + String _elem858; // required _elem858 = iprot.readString(); struct.success.add(_elem858); } @@ -84568,7 +84568,7 @@ struct.success = new ArrayList(_list861.size); for (int _i862 = 0; _i862 < _list861.size; ++_i862) { - String _elem863; // optional + String _elem863; // required _elem863 = iprot.readString(); struct.success.add(_elem863); } @@ -94564,7 +94564,7 @@ struct.success = new ArrayList(_list894.size); for (int _i895 = 0; _i895 < _list894.size; ++_i895) { - Index _elem896; // optional + Index _elem896; // required _elem896 = new Index(); _elem896.read(iprot); struct.success.add(_elem896); @@ -94684,7 +94684,7 @@ struct.success = new ArrayList(_list899.size); for (int _i900 = 0; _i900 < _list899.size; ++_i900) { - Index _elem901; // optional + Index _elem901; // required _elem901 = new Index(); _elem901.read(iprot); struct.success.add(_elem901); @@ -95673,7 +95673,7 @@ struct.success = new ArrayList(_list902.size); for (int _i903 = 0; _i903 < _list902.size; ++_i903) { - String _elem904; // optional + String _elem904; // required _elem904 = iprot.readString(); struct.success.add(_elem904); } @@ -95772,7 +95772,7 @@ struct.success = new ArrayList(_list907.size); for (int _i908 = 0; _i908 < _list907.size; ++_i908) { - String _elem909; // optional + String _elem909; // required _elem909 = iprot.readString(); struct.success.add(_elem909); } @@ -111516,7 +111516,7 @@ struct.success = new ArrayList(_list910.size); for (int _i911 = 0; _i911 < _list910.size; ++_i911) { - String _elem912; // optional + String _elem912; // required _elem912 = iprot.readString(); struct.success.add(_elem912); } @@ -111615,7 +111615,7 @@ struct.success = new ArrayList(_list915.size); for (int _i916 = 0; _i916 < _list915.size; ++_i916) { - String _elem917; // optional + String _elem917; // required _elem917 = iprot.readString(); struct.success.add(_elem917); } @@ -114964,7 +114964,7 @@ struct.success = new ArrayList(_list918.size); for (int _i919 = 0; _i919 < _list918.size; ++_i919) { - String _elem920; // optional + String _elem920; // required _elem920 = iprot.readString(); struct.success.add(_elem920); } @@ -115063,7 +115063,7 @@ struct.success = new ArrayList(_list923.size); for (int _i924 = 0; _i924 < _list923.size; ++_i924) { - String _elem925; // optional + String _elem925; // required _elem925 = iprot.readString(); struct.success.add(_elem925); } @@ -118360,7 +118360,7 @@ struct.success = new ArrayList(_list926.size); for (int _i927 = 0; _i927 < _list926.size; ++_i927) { - Role _elem928; // optional + Role _elem928; // required _elem928 = new Role(); _elem928.read(iprot); struct.success.add(_elem928); @@ -118460,7 +118460,7 @@ struct.success = new ArrayList(_list931.size); for (int _i932 = 0; _i932 < _list931.size; ++_i932) { - Role _elem933; // optional + Role _elem933; // required _elem933 = new Role(); _elem933.read(iprot); struct.success.add(_elem933); @@ -121475,7 +121475,7 @@ struct.group_names = new ArrayList(_list934.size); for (int _i935 = 0; _i935 < _list934.size; ++_i935) { - String _elem936; // optional + String _elem936; // required _elem936 = iprot.readString(); struct.group_names.add(_elem936); } @@ -121585,7 +121585,7 @@ struct.group_names = new ArrayList(_list939.size); for (int _i940 = 0; _i940 < _list939.size; ++_i940) { - String _elem941; // optional + String _elem941; // required _elem941 = iprot.readString(); struct.group_names.add(_elem941); } @@ -123049,7 +123049,7 @@ struct.success = new ArrayList(_list942.size); for (int _i943 = 0; _i943 < _list942.size; ++_i943) { - HiveObjectPrivilege _elem944; // optional + HiveObjectPrivilege _elem944; // required _elem944 = new HiveObjectPrivilege(); _elem944.read(iprot); struct.success.add(_elem944); @@ -123149,7 +123149,7 @@ struct.success = new ArrayList(_list947.size); for (int _i948 = 0; _i948 < _list947.size; ++_i948) { - HiveObjectPrivilege _elem949; // optional + HiveObjectPrivilege _elem949; // required _elem949 = new HiveObjectPrivilege(); _elem949.read(iprot); struct.success.add(_elem949); @@ -126061,7 +126061,7 @@ struct.group_names = new ArrayList(_list950.size); for (int _i951 = 0; _i951 < _list950.size; ++_i951) { - String _elem952; // optional + String _elem952; // required _elem952 = iprot.readString(); struct.group_names.add(_elem952); } @@ -126155,7 +126155,7 @@ struct.group_names = new ArrayList(_list955.size); for (int _i956 = 0; _i956 < _list955.size; ++_i956) { - String _elem957; // optional + String _elem957; // required _elem957 = iprot.readString(); struct.group_names.add(_elem957); } @@ -126567,7 +126567,7 @@ struct.success = new ArrayList(_list958.size); for (int _i959 = 0; _i959 < _list958.size; ++_i959) { - String _elem960; // optional + String _elem960; // required _elem960 = iprot.readString(); struct.success.add(_elem960); } @@ -126666,7 +126666,7 @@ struct.success = new ArrayList(_list963.size); for (int _i964 = 0; _i964 < _list963.size; ++_i964) { - String _elem965; // optional + String _elem965; // required _elem965 = iprot.readString(); struct.success.add(_elem965); } Index: metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java =================================================================== --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (revision 1674118) +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (working copy) @@ -6062,6 +6062,11 @@ ShimLoader.getHadoopShims().startPauseMonitor(conf); try { + startPauseMonitor(conf); + } catch (Throwable t) { + LOG.warn("Error starting the JVM pause monitor", t); + } + try { // Per the javadocs on Condition, do not depend on the condition alone as a start gate // since spurious wake ups are possible. while (!startedServing.get()) startCondition.await(); @@ -6080,6 +6085,18 @@ t.start(); } + private static void startPauseMonitor(HiveConf conf) throws Exception { + try { + Class.forName("org.apache.hadoop.util.JvmPauseMonitor"); + org.apache.hadoop.util.JvmPauseMonitor pauseMonitor = + new org.apache.hadoop.util.JvmPauseMonitor(conf); + pauseMonitor.start(); + } catch (Throwable t) { + LOG.warn("Could not initiate the JvmPauseMonitor thread." + + " GCs and Pauses may not be warned upon.", t); + } + } + private static void startCompactorInitiator(HiveConf conf) throws Exception { if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_INITIATOR_ON)) { MetaStoreThread initiator = Index: pom.xml =================================================================== --- pom.xml (revision 1674118) +++ pom.xml (working copy) @@ -156,7 +156,7 @@ 4.0.4 0.5.2 2.2.0 - 1.2.0 + 1.3.0 2.10 2.10.4 1.1 @@ -216,6 +216,16 @@ false + + spark-1.3 + https://s3-us-west-1.amazonaws.com/hive-spark/maven2/spark_2.10-1.3-rc1/ + + true + + + false + + Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java (working copy) @@ -69,9 +69,9 @@ import org.apache.hadoop.mapred.JobConfigurable; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hive.common.util.AnnotationUtils; -import org.apache.hive.common.util.ReflectionUtil; import com.google.common.collect.Iterators; @@ -204,12 +204,13 @@ JobConf conf) throws IOException { if (Configurable.class.isAssignableFrom(inputFormatClass) || JobConfigurable.class.isAssignableFrom(inputFormatClass)) { - return ReflectionUtil.newInstance(inputFormatClass, conf); + return ReflectionUtils + .newInstance(inputFormatClass, conf); } InputFormat format = inputFormats.get(inputFormatClass.getName()); if (format == null) { try { - format = ReflectionUtil.newInstance(inputFormatClass, conf); + format = ReflectionUtils.newInstance(inputFormatClass, conf); inputFormats.put(inputFormatClass.getName(), format); } catch (Exception e) { throw new IOException("Cannot create an instance of InputFormat class " Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java (working copy) @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.CommandNeedRetryException; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; import org.apache.hadoop.hive.ql.io.HiveInputFormat; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; @@ -79,6 +80,7 @@ fetch = new FetchOperator(work, job, source, getVirtualColumns(source)); source.initialize(conf, new ObjectInspector[]{fetch.getOutputObjectInspector()}); totalRows = 0; + ExecMapper.setDone(false); } catch (Exception e) { // Bail out ungracefully - we should never hit Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java (working copy) @@ -61,7 +61,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Writable; -import org.apache.hive.common.util.ReflectionUtil; +import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.hive.ql.exec.persistence.HybridHashTableContainer.HashPartition; import static org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer.KeyValueHelper; @@ -229,8 +229,8 @@ try { TableDesc keyTableDesc = conf.getKeyTblDesc(); - SerDe keySerializer = (SerDe) ReflectionUtil.newInstance( - keyTableDesc.getDeserializerClass(), null); + SerDe keySerializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc.getDeserializerClass(), + null); SerDeUtils.initializeSerDe(keySerializer, null, keyTableDesc.getProperties(), null); MapJoinObjectSerDeContext keyContext = new MapJoinObjectSerDeContext(keySerializer, false); for (int pos = 0; pos < order.length; pos++) { @@ -243,8 +243,8 @@ } else { valueTableDesc = conf.getValueFilteredTblDescs().get(pos); } - SerDe valueSerDe = (SerDe) ReflectionUtil.newInstance( - valueTableDesc.getDeserializerClass(), null); + SerDe valueSerDe = + (SerDe) ReflectionUtils.newInstance(valueTableDesc.getDeserializerClass(), null); SerDeUtils.initializeSerDe(valueSerDe, null, valueTableDesc.getProperties(), null); MapJoinObjectSerDeContext valueContext = new MapJoinObjectSerDeContext(valueSerDe, hasFilter(pos)); Index: ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java (working copy) @@ -18,7 +18,6 @@ package org.apache.hadoop.hive.ql.exec; import java.io.BufferedOutputStream; -import java.io.IOException; import java.io.ObjectOutputStream; import java.io.OutputStream; import java.io.Serializable; @@ -85,8 +84,18 @@ if (mapJoinTables == null || mapJoinTables.length < tag || mapJoinTables[tag] == null) { LOG.debug("mapJoinTable is null"); + } else if (abort) { + if (LOG.isDebugEnabled()) { + LOG.debug("Aborting, skip dumping side-table for tag: " + tag); + } } else { - flushToFile(mapJoinTables[tag], tag); + String method = PerfLogger.SPARK_FLUSH_HASHTABLE + getName(); + perfLogger.PerfLogBegin(CLASS_NAME, method); + try { + flushToFile(mapJoinTables[tag], tag); + } finally { + perfLogger.PerfLogEnd(CLASS_NAME, method); + } } super.closeOp(abort); } catch (HiveException e) { @@ -97,8 +106,7 @@ } protected void flushToFile(MapJoinPersistableTableContainer tableContainer, - byte tag) throws IOException, HiveException { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_FLUSH_HASHTABLE + this.getName()); + byte tag) throws Exception { MapredLocalWork localWork = getExecContext().getLocalWork(); BucketMapJoinContext mapJoinCtx = localWork.getBucketMapjoinContext(); Path inputPath = getExecContext().getCurrentInputPath(); @@ -135,12 +143,11 @@ } // TODO find out numOfPartitions for the big table int numOfPartitions = replication; - replication = (short) Math.min(MIN_REPLICATION, numOfPartitions); + replication = (short) Math.max(MIN_REPLICATION, numOfPartitions); } htsOperator.console.printInfo(Utilities.now() + "\tDump the side-table for tag: " + tag + " with group count: " + tableContainer.size() + " into file: " + path); // get the hashtable file and path - // get the hashtable file and path OutputStream os = null; ObjectOutputStream out = null; try { @@ -148,6 +155,18 @@ out = new ObjectOutputStream(new BufferedOutputStream(os, 4096)); MapJoinTableContainerSerDe mapJoinTableSerde = htsOperator.mapJoinTableSerdes[tag]; mapJoinTableSerde.persist(out, tableContainer); + FileStatus status = fs.getFileStatus(path); + htsOperator.console.printInfo(Utilities.now() + "\tUploaded 1 File to: " + path + + " (" + status.getLen() + " bytes)"); + } catch (Exception e) { + // Failed to dump the side-table, remove the partial file + try { + fs.delete(path, false); + } catch (Exception ex) { + LOG.warn("Got exception in deleting partial side-table dump for tag: " + + tag + ", file " + path, ex); + } + throw e; } finally { if (out != null) { out.close(); @@ -156,10 +175,6 @@ } } tableContainer.clear(); - FileStatus status = fs.getFileStatus(path); - htsOperator.console.printInfo(Utilities.now() + "\tUploaded 1 File to: " + path - + " (" + status.getLen() + " bytes)"); - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_FLUSH_HASHTABLE + this.getName()); } public void setTag(byte tag) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (working copy) @@ -84,6 +84,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.WordUtils; import org.apache.commons.lang3.StringEscapeUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -152,6 +153,8 @@ import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.PlanUtils.ExpressionTypes; import org.apache.hadoop.hive.ql.plan.ReduceWork; +import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; +import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.api.Adjacency; import org.apache.hadoop.hive.ql.plan.api.Graph; @@ -1101,6 +1104,28 @@ fld.removeField(fieldName); kryo.register(type, fld); } + + public static ThreadLocal sparkSerializationKryo = new ThreadLocal() { + @Override + protected synchronized Kryo initialValue() { + Kryo kryo = new Kryo(); + kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); + kryo.register(java.sql.Date.class, new SqlDateSerializer()); + kryo.register(java.sql.Timestamp.class, new TimestampSerializer()); + kryo.register(Path.class, new PathSerializer()); + kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); + removeField(kryo, Operator.class, "colExprMap"); + removeField(kryo, ColumnInfo.class, "objectInspector"); + kryo.register(SparkEdgeProperty.class); + kryo.register(MapWork.class); + kryo.register(ReduceWork.class); + kryo.register(SparkWork.class); + kryo.register(TableDesc.class); + kryo.register(Pair.class); + return kryo; + }; + }; + private static ThreadLocal cloningQueryPlanKryo = new ThreadLocal() { @Override protected synchronized Kryo initialValue() { @@ -3672,7 +3697,7 @@ public static boolean isVectorMode(Configuration conf) { if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED) && Utilities.getPlanPath(conf) != null && Utilities - .getMapRedWork(conf).getMapWork().getVectorMode()) { + .getMapWork(conf).getVectorMode()) { return true; } return false; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java (working copy) @@ -21,28 +21,35 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.util.Arrays; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.concurrent.TimeUnit; +import java.util.Set; import org.apache.commons.compress.utils.CharsetNames; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.io.HiveKey; +import org.apache.hadoop.io.BytesWritable; import org.apache.hive.spark.client.rpc.RpcConfiguration; import org.apache.spark.SparkConf; import org.apache.spark.SparkException; +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import com.google.common.base.Strings; +import com.google.common.collect.Sets; + public class HiveSparkClientFactory { protected static final transient Log LOG = LogFactory.getLog(HiveSparkClientFactory.class); private static final String SPARK_DEFAULT_CONF_FILE = "spark-defaults.conf"; - private static final String SPARK_DEFAULT_MASTER = "local"; + private static final String SPARK_DEFAULT_MASTER = "yarn-cluster"; private static final String SPARK_DEFAULT_APP_NAME = "Hive on Spark"; private static final String SPARK_DEFAULT_SERIALIZER = "org.apache.spark.serializer.KryoSerializer"; + private static final String SPARK_DEFAULT_REFERENCE_TRACKING = "false"; public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf) throws IOException, SparkException { @@ -66,6 +73,7 @@ sparkConf.put("spark.master", SPARK_DEFAULT_MASTER); sparkConf.put("spark.app.name", SPARK_DEFAULT_APP_NAME); sparkConf.put("spark.serializer", SPARK_DEFAULT_SERIALIZER); + sparkConf.put("spark.kryo.referenceTracking", SPARK_DEFAULT_REFERENCE_TRACKING); // load properties from spark-defaults.conf. InputStream inputStream = null; @@ -105,6 +113,9 @@ if (sparkMaster == null) { sparkMaster = sparkConf.get("spark.master"); } + if (sparkMaster.equals("yarn-cluster")) { + sparkConf.put("spark.yarn.maxAppAttempts", "1"); + } for (Map.Entry entry : hiveConf) { String propertyName = entry.getKey(); if (propertyName.startsWith("spark")) { @@ -133,6 +144,15 @@ } } + Set classes = Sets.newHashSet( + Splitter.on(",").trimResults().omitEmptyStrings().split( + Strings.nullToEmpty(sparkConf.get("spark.kryo.classesToRegister")))); + classes.add(VectorizedRowBatch.class.getName()); + classes.add(BytesWritable.class.getName()); + classes.add(HiveKey.class.getName()); + sparkConf.put( + "spark.kryo.classesToRegister", Joiner.on(",").join(classes)); + return sparkConf; } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java (working copy) @@ -39,14 +39,14 @@ ByteArrayOutputStream stream = new ByteArrayOutputStream(); Output output = new Output(stream); - Utilities.runtimeSerializationKryo.get().writeObject(output, object); + Utilities.sparkSerializationKryo.get().writeObject(output, object); output.close(); // close() also calls flush() return stream.toByteArray(); } public static T deserialize(byte[] buffer, Class clazz) { - return Utilities.runtimeSerializationKryo.get().readObject( + return Utilities.sparkSerializationKryo.get().readObject( new Input(new ByteArrayInputStream(buffer)), clazz); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java (working copy) @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.Serializable; +import java.io.StringWriter; import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; @@ -33,6 +34,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; @@ -165,7 +167,9 @@ try { URI fileUri = SparkUtilities.getURI(addedFile); if (fileUri != null && !localFiles.contains(fileUri)) { - fileUri = SparkUtilities.uploadToHDFS(fileUri, hiveConf); + if (SparkUtilities.needUploadToHDFS(fileUri, sparkConf)) { + fileUri = SparkUtilities.uploadToHDFS(fileUri, hiveConf); + } localFiles.add(fileUri); remoteClient.addFile(fileUri); } @@ -180,7 +184,9 @@ try { URI jarUri = SparkUtilities.getURI(addedJar); if (jarUri != null && !localJars.contains(jarUri)) { - jarUri = SparkUtilities.uploadToHDFS(jarUri, hiveConf); + if (SparkUtilities.needUploadToHDFS(jarUri, sparkConf)) { + jarUri = SparkUtilities.uploadToHDFS(jarUri, hiveConf); + } localJars.add(jarUri); remoteClient.addJar(jarUri); } @@ -220,12 +226,14 @@ // may need to load classes from this jar in other threads. List addedJars = jc.getAddedJars(); if (addedJars != null && !addedJars.isEmpty()) { - SparkClientUtilities.addToClassPath(addedJars.toArray(new String[addedJars.size()])); + SparkClientUtilities.addToClassPath(addedJars.toArray(new String[addedJars.size()]), + localJobConf, jc.getLocalTmpDir()); localJobConf.set(Utilities.HIVE_ADDED_JARS, StringUtils.join(addedJars, ";")); } Path localScratchDir = KryoSerializer.deserialize(scratchDirBytes, Path.class); SparkWork localSparkWork = KryoSerializer.deserialize(sparkWorkBytes, SparkWork.class); + logConfigurations(localJobConf); SparkCounters sparkCounters = new SparkCounters(jc.sc()); Map> prefixes = localSparkWork.getRequiredCounterPrefix(); @@ -250,6 +258,18 @@ jc.monitor(future, sparkCounters, plan.getCachedRDDIds()); return null; } + + private void logConfigurations(JobConf localJobConf) { + if (LOG.isInfoEnabled()) { + LOG.info("Logging job configuration: "); + StringWriter outWriter = new StringWriter(); + try { + Configuration.dumpConfiguration(localJobConf, outWriter); + } catch (IOException e) { + LOG.warn("Error logging job configuration", e); + } + LOG.info(outWriter.toString()); + } + } } - } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java (working copy) @@ -137,6 +137,7 @@ } } catch (Throwable e) { abort = true; + Utilities.setMapWork(jc, null); if (e instanceof OutOfMemoryError) { // Don't create a new object if we are already out of memory throw (OutOfMemoryError) e; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java (working copy) @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import com.google.common.base.Preconditions; @@ -37,6 +38,8 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; import org.apache.hadoop.hive.ql.exec.mr.ExecReducer; @@ -96,14 +99,20 @@ workToTranMap.clear(); workToParentWorkTranMap.clear(); - for (BaseWork work : sparkWork.getAllWork()) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_CREATE_TRAN + work.getName()); - SparkTran tran = generate(work); - SparkTran parentTran = generateParentTran(sparkPlan, sparkWork, work); - sparkPlan.addTran(tran); - sparkPlan.connect(parentTran, tran); - workToTranMap.put(work, tran); - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_CREATE_TRAN + work.getName()); + try { + for (BaseWork work : sparkWork.getAllWork()) { + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_CREATE_TRAN + work.getName()); + SparkTran tran = generate(work); + SparkTran parentTran = generateParentTran(sparkPlan, sparkWork, work); + sparkPlan.addTran(tran); + sparkPlan.connect(parentTran, tran); + workToTranMap.put(work, tran); + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_CREATE_TRAN + work.getName()); + } + } finally { + // clear all ThreadLocal cached MapWork/ReduceWork after plan generation + // as this may executed in a pool thread. + Utilities.clearWorkMap(); } perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_BUILD_PLAN); @@ -200,6 +209,7 @@ private SparkTran generate(BaseWork work) throws Exception { initStatsPublisher(work); JobConf newJobConf = cloneJobConf(work); + checkSpecs(work, newJobConf); byte[] confBytes = KryoSerializer.serializeJobConf(newJobConf); if (work instanceof MapWork) { MapTran mapTran = new MapTran(); @@ -217,6 +227,15 @@ } } + private void checkSpecs(BaseWork work, JobConf jc) throws Exception { + Set> opList = work.getAllOperators(); + for (Operator op : opList) { + if (op instanceof FileSinkOperator) { + ((FileSinkOperator) op).checkOutputSpecs(null, jc); + } + } + } + @SuppressWarnings({ "unchecked" }) private JobConf cloneJobConf(BaseWork work) throws Exception { if (workToJobConf.containsKey(work)) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java (working copy) @@ -283,6 +283,7 @@ } catch (Throwable e) { abort = true; + Utilities.setReduceWork(jc, null); if (e instanceof OutOfMemoryError) { // Don't create a new object if we are already out of memory throw (OutOfMemoryError) e; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java (working copy) @@ -28,6 +28,8 @@ import java.util.List; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.Warehouse; @@ -51,6 +53,7 @@ import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; +import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; @@ -65,6 +68,8 @@ import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.StatsWork; import org.apache.hadoop.hive.ql.plan.api.StageType; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.hadoop.hive.ql.stats.StatsFactory; import org.apache.hadoop.util.StringUtils; import org.apache.hive.spark.counter.SparkCounters; @@ -73,6 +78,8 @@ public class SparkTask extends Task { private static final String CLASS_NAME = SparkTask.class.getName(); + private static final Log LOG = LogFactory.getLog(CLASS_NAME); + private static final LogHelper console = new LogHelper(LOG); private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); private static final long serialVersionUID = 1L; private SparkCounters sparkCounters; @@ -100,6 +107,7 @@ SparkJobRef jobRef = sparkSession.submit(driverContext, sparkWork); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); + addToHistory(jobRef); rc = jobRef.monitorJob(); SparkJobStatus sparkJobStatus = jobRef.getSparkJobStatus(); if (rc == 0) { @@ -136,6 +144,14 @@ return rc; } + private void addToHistory(SparkJobRef jobRef) { + console.printInfo("Starting Spark Job = " + jobRef.getJobId()); + if (SessionState.get() != null) { + SessionState.get().getHiveHistory() + .setQueryProperty(SessionState.get().getQueryId(), Keys.SPARK_JOB_ID, jobRef.getJobId()); + } + } + private void logSparkStatistic(SparkStatistics sparkStatistic) { Iterator groupIterator = sparkStatistic.getStatisticGroups(); while (groupIterator.hasNext()) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java (working copy) @@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.io.BytesWritable; +import org.apache.spark.SparkConf; /** * Contains utilities methods used as part of Spark tasks. @@ -67,7 +68,7 @@ } /** - * Copies local file to HDFS in yarn-cluster mode. + * Uploads a local file to HDFS * * @param source * @param conf @@ -75,20 +76,20 @@ * @throws IOException */ public static URI uploadToHDFS(URI source, HiveConf conf) throws IOException { - URI result = source; - if (conf.get("spark.master").equals("yarn-cluster")) { - if (!source.getScheme().equals("hdfs")) { - Path tmpDir = SessionState.getHDFSSessionPath(conf); - FileSystem fileSystem = FileSystem.get(conf); - fileSystem.copyFromLocalFile(new Path(source.getPath()), tmpDir); - String filePath = tmpDir + File.separator + getFileName(source); - Path fullPath = fileSystem.getFileStatus(new Path(filePath)).getPath(); - result = fullPath.toUri(); - } - } - return result; + Path tmpDir = SessionState.getHDFSSessionPath(conf); + FileSystem fileSystem = FileSystem.get(conf); + fileSystem.copyFromLocalFile(new Path(source.getPath()), tmpDir); + String filePath = tmpDir + File.separator + getFileName(source); + Path fullPath = fileSystem.getFileStatus(new Path(filePath)).getPath(); + return fullPath.toUri(); } + // checks if a resource has to be uploaded to HDFS for yarn-cluster mode + public static boolean needUploadToHDFS(URI source, SparkConf sparkConf) { + return sparkConf.get("spark.master").equals("yarn-cluster") && + !source.getScheme().equals("hdfs"); + } + private static String getFileName(URI uri) { if (uri == null) { return null; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java (working copy) @@ -39,6 +39,8 @@ import org.apache.spark.scheduler.SparkListenerTaskGettingResult; import org.apache.spark.scheduler.SparkListenerTaskStart; import org.apache.spark.scheduler.SparkListenerUnpersistRDD; +import org.apache.spark.scheduler.SparkListenerExecutorRemoved; +import org.apache.spark.scheduler.SparkListenerExecutorAdded; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -52,6 +54,16 @@ private final Map>> allJobMetrics = Maps.newHashMap(); @Override + public void onExecutorRemoved(SparkListenerExecutorRemoved removed) { + + } + + @Override + public void onExecutorAdded(SparkListenerExecutorAdded added) { + + } + + @Override public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java (working copy) @@ -166,7 +166,7 @@ HiveException { Map pathToPartitionInfo = Utilities - .getMapRedWork(hiveConf).getMapWork().getPathToPartitionInfo(); + .getMapWork(hiveConf).getPathToPartitionInfo(); PartitionDesc part = HiveFileFormatUtils .getPartitionDescFromPathRecursively(pathToPartitionInfo, Index: ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java (working copy) @@ -68,7 +68,8 @@ TASK_COUNTERS, TASK_NUM_MAPPERS, TASK_NUM_REDUCERS, - ROWS_INSERTED + ROWS_INSERTED, + SPARK_JOB_ID }; /** Index: ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java (working copy) @@ -580,6 +580,10 @@ if (oldPaths != null) { job.set(HiveConf.ConfVars.HADOOPMAPREDINPUTDIR.varname, oldPaths); } + + // clear work from ThreadLocal after splits generated in case of thread is reused in pool. + Utilities.clearWorkMapForConf(job); + LOG.info("Number of all splits " + result.size()); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS); return result.toArray(new InputSplit[result.size()]); Index: ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java (working copy) @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.ql.lib; +import org.apache.hadoop.hive.ql.exec.ConditionalTask; +import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.parse.SemanticException; /** @@ -58,6 +60,12 @@ for (Node n : nd.getChildren()) { walk(n); } + } else if (nd instanceof ConditionalTask) { + for (Task n : ((ConditionalTask) nd).getListTasks()) { + if (n.getParentTasks() == null || n.getParentTasks().isEmpty()) { + walk(n); + } + } } opStack.pop(); Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java (working copy) @@ -335,6 +335,7 @@ if (child != null) { currTask.removeDependentTask(child); listTasks.add(child); + listWorks.add(child.getWork()); } ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx context = new ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx(bigKeysDirToTaskMap, child); @@ -413,12 +414,4 @@ } mapJoinDesc.setHashTableMemoryUsage(hashtableMemoryUsage); } - - // check this before calling processSkewJoin - public static boolean supportRuntimeSkewJoin(JoinOperator joinOp, - Task currTask, HiveConf hiveConf) { - List> children = currTask.getChildTasks(); - return GenMRSkewJoinProcessor.skewJoinEnabled(hiveConf, joinOp) - && (children == null || children.size() <= 1); - } } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java (working copy) @@ -59,6 +59,9 @@ public class SparkMapJoinResolver implements PhysicalPlanResolver { + // prevents a task from being processed multiple times + private final Set> visitedTasks = new HashSet<>(); + @Override public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { @@ -78,6 +81,15 @@ return matchingOps != null && !matchingOps.isEmpty(); } + private boolean containsOp(SparkWork sparkWork, Class clazz) { + for (BaseWork work : sparkWork.getAllWorkUnsorted()) { + if (containsOp(work, clazz)) { + return true; + } + } + return false; + } + public static Set> getOp(BaseWork work, Class clazz) { Set> ops = new HashSet>(); if (work instanceof MapWork) { @@ -172,70 +184,68 @@ private void generateLocalWork(SparkTask originalTask) { SparkWork originalWork = originalTask.getWork(); - Collection allBaseWorks = originalWork.getAllWorkUnsorted(); - - for (BaseWork work : allBaseWorks) { - if (containsOp(work, SparkHashTableSinkOperator.class) || - containsOp(work, MapJoinOperator.class)) { - work.setMapRedLocalWork(new MapredLocalWork()); - } - } - + Collection allBaseWorks = originalWork.getAllWork(); Context ctx = physicalContext.getContext(); for (BaseWork work : allBaseWorks) { - Set> ops = getOp(work, MapJoinOperator.class); - if (ops == null || ops.isEmpty()) { - continue; - } - Path tmpPath = Utilities.generateTmpPath(ctx.getMRTmpPath(), originalTask.getId()); - MapredLocalWork bigTableLocalWork = work.getMapRedLocalWork(); - List> dummyOps = - new ArrayList>(work.getDummyOps()); - bigTableLocalWork.setDummyParentOp(dummyOps); - bigTableLocalWork.setTmpPath(tmpPath); + if (work.getMapRedLocalWork() == null) { + if (containsOp(work, SparkHashTableSinkOperator.class) || + containsOp(work, MapJoinOperator.class)) { + work.setMapRedLocalWork(new MapredLocalWork()); + } + Set> ops = getOp(work, MapJoinOperator.class); + if (ops == null || ops.isEmpty()) { + continue; + } + Path tmpPath = Utilities.generateTmpPath(ctx.getMRTmpPath(), originalTask.getId()); + MapredLocalWork bigTableLocalWork = work.getMapRedLocalWork(); + List> dummyOps = + new ArrayList>(work.getDummyOps()); + bigTableLocalWork.setDummyParentOp(dummyOps); + bigTableLocalWork.setTmpPath(tmpPath); - // In one work, only one map join operator can be bucketed - SparkBucketMapJoinContext bucketMJCxt = null; - for (Operator op: ops) { - MapJoinOperator mapJoinOp = (MapJoinOperator) op; - MapJoinDesc mapJoinDesc = mapJoinOp.getConf(); - if (mapJoinDesc.isBucketMapJoin()) { - bucketMJCxt = new SparkBucketMapJoinContext(mapJoinDesc); - bucketMJCxt.setBucketMatcherClass( - org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); - bucketMJCxt.setPosToAliasMap(mapJoinOp.getPosToAliasMap()); - ((MapWork) work).setUseBucketizedHiveInputFormat(true); - bigTableLocalWork.setBucketMapjoinContext(bucketMJCxt); - bigTableLocalWork.setInputFileChangeSensitive(true); - break; + // In one work, only one map join operator can be bucketed + SparkBucketMapJoinContext bucketMJCxt = null; + for (Operator op : ops) { + MapJoinOperator mapJoinOp = (MapJoinOperator) op; + MapJoinDesc mapJoinDesc = mapJoinOp.getConf(); + if (mapJoinDesc.isBucketMapJoin()) { + bucketMJCxt = new SparkBucketMapJoinContext(mapJoinDesc); + bucketMJCxt.setBucketMatcherClass( + org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); + bucketMJCxt.setPosToAliasMap(mapJoinOp.getPosToAliasMap()); + ((MapWork) work).setUseBucketizedHiveInputFormat(true); + bigTableLocalWork.setBucketMapjoinContext(bucketMJCxt); + bigTableLocalWork.setInputFileChangeSensitive(true); + break; + } } - } - for (BaseWork parentWork : originalWork.getParents(work)) { - Set> hashTableSinkOps = - getOp(parentWork, SparkHashTableSinkOperator.class); - if (hashTableSinkOps == null || hashTableSinkOps.isEmpty()) { - continue; - } - MapredLocalWork parentLocalWork = parentWork.getMapRedLocalWork(); - parentLocalWork.setTmpHDFSPath(tmpPath); - if (bucketMJCxt != null) { - // We only need to update the work with the hashtable - // sink operator with the same mapjoin desc. We can tell - // that by comparing the bucket file name mapping map - // instance. They should be exactly the same one due to - // the way how the bucket mapjoin context is constructed. - for (Operator op: hashTableSinkOps) { - SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator) op; - SparkHashTableSinkDesc hashTableSinkDesc = hashTableSinkOp.getConf(); - BucketMapJoinContext original = hashTableSinkDesc.getBucketMapjoinContext(); - if (original != null && original.getBucketFileNameMapping() - == bucketMJCxt.getBucketFileNameMapping()) { - ((MapWork) parentWork).setUseBucketizedHiveInputFormat(true); - parentLocalWork.setBucketMapjoinContext(bucketMJCxt); - parentLocalWork.setInputFileChangeSensitive(true); - break; + for (BaseWork parentWork : originalWork.getParents(work)) { + Set> hashTableSinkOps = + getOp(parentWork, SparkHashTableSinkOperator.class); + if (hashTableSinkOps == null || hashTableSinkOps.isEmpty()) { + continue; + } + MapredLocalWork parentLocalWork = parentWork.getMapRedLocalWork(); + parentLocalWork.setTmpHDFSPath(tmpPath); + if (bucketMJCxt != null) { + // We only need to update the work with the hashtable + // sink operator with the same mapjoin desc. We can tell + // that by comparing the bucket file name mapping map + // instance. They should be exactly the same one due to + // the way how the bucket mapjoin context is constructed. + for (Operator op : hashTableSinkOps) { + SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator) op; + SparkHashTableSinkDesc hashTableSinkDesc = hashTableSinkOp.getConf(); + BucketMapJoinContext original = hashTableSinkDesc.getBucketMapjoinContext(); + if (original != null && original.getBucketFileNameMapping() + == bucketMJCxt.getBucketFileNameMapping()) { + ((MapWork) parentWork).setUseBucketizedHiveInputFormat(true); + parentLocalWork.setBucketMapjoinContext(bucketMJCxt); + parentLocalWork.setInputFileChangeSensitive(true); + break; + } } } } @@ -296,10 +306,12 @@ for (Task tsk : taskList) { if (tsk instanceof SparkTask) { processCurrentTask((SparkTask) tsk, (ConditionalTask) currentTask); + visitedTasks.add(tsk); } } } else if (currentTask instanceof SparkTask) { processCurrentTask((SparkTask) currentTask, null); + visitedTasks.add(currentTask); } } @@ -312,32 +324,47 @@ * wrapped in its task list. */ private void processCurrentTask(SparkTask sparkTask, ConditionalTask conditionalTask) { - dependencyGraph.clear(); - sparkWorkMap.clear(); SparkWork sparkWork = sparkTask.getWork(); + if (!visitedTasks.contains(sparkTask)) { + dependencyGraph.clear(); + sparkWorkMap.clear(); - // Generate MapredLocalWorks for MJ and HTS - generateLocalWork(sparkTask); + // Generate MapredLocalWorks for MJ and HTS + generateLocalWork(sparkTask); - dependencyGraph.put(sparkWork, new ArrayList()); - Set leaves = sparkWork.getLeaves(); - for (BaseWork leaf : leaves) { - moveWork(sparkWork, leaf, sparkWork); - } + dependencyGraph.put(sparkWork, new ArrayList()); + Set leaves = sparkWork.getLeaves(); + for (BaseWork leaf : leaves) { + moveWork(sparkWork, leaf, sparkWork); + } - // Now remove all BaseWorks in all the childSparkWorks that we created - // from the original SparkWork - for (SparkWork newSparkWork : sparkWorkMap.values()) { - for (BaseWork work : newSparkWork.getAllWorkUnsorted()) { - sparkWork.remove(work); + // Now remove all BaseWorks in all the childSparkWorks that we created + // from the original SparkWork + for (SparkWork newSparkWork : sparkWorkMap.values()) { + for (BaseWork work : newSparkWork.getAllWorkUnsorted()) { + sparkWork.remove(work); + } } - } - Map createdTaskMap = new LinkedHashMap(); + Map createdTaskMap = new LinkedHashMap(); - // Now create SparkTasks from the SparkWorks, also set up dependency - for (SparkWork work : dependencyGraph.keySet()) { - createSparkTask(sparkTask, work, createdTaskMap, conditionalTask); + // Now create SparkTasks from the SparkWorks, also set up dependency + for (SparkWork work : dependencyGraph.keySet()) { + createSparkTask(sparkTask, work, createdTaskMap, conditionalTask); + } + } else if (conditionalTask != null) { + // We may need to update the conditional task's list. This happens when a common map join + // task exists in the task list and has already been processed. In such a case, + // the current task is the map join task and we need to replace it with + // its parent, i.e. the small table task. + if (sparkTask.getParentTasks() != null && sparkTask.getParentTasks().size() == 1 && + sparkTask.getParentTasks().get(0) instanceof SparkTask) { + SparkTask parent = (SparkTask) sparkTask.getParentTasks().get(0); + if (containsOp(sparkWork, MapJoinOperator.class) && + containsOp(parent.getWork(), SparkHashTableSinkOperator.class)) { + updateConditionalTask(conditionalTask, sparkTask, parent); + } + } } } @@ -382,6 +409,10 @@ } } context.setDirToTaskMap(newbigKeysDirToTaskMap); + // update no skew task + if (context.getNoSkewTask() != null && context.getNoSkewTask().equals(originalTask)) { + context.setNoSkewTask(newTask); + } } } } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java (working copy) @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -34,6 +35,7 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessor; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; +import org.apache.hadoop.hive.ql.optimizer.physical.GenMRSkewJoinProcessor; import org.apache.hadoop.hive.ql.optimizer.physical.GenSparkSkewJoinProcessor; import org.apache.hadoop.hive.ql.optimizer.physical.SkewJoinProcFactory; import org.apache.hadoop.hive.ql.optimizer.physical.SparkMapJoinResolver; @@ -50,6 +52,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc; import java.io.Serializable; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.Stack; @@ -58,6 +61,9 @@ * Spark-version of SkewJoinProcFactory. */ public class SparkSkewJoinProcFactory { + // let's remember the join operators we have processed + private static final Set visitedJoinOp = new HashSet(); + private SparkSkewJoinProcFactory() { // prevent instantiation } @@ -81,13 +87,12 @@ JoinOperator op = (JoinOperator) nd; ReduceWork reduceWork = context.getReducerToReduceWork().get(op); ParseContext parseContext = context.getParseCtx(); - if (!op.getConf().isFixedAsSorted() && currentTsk instanceof SparkTask - && reduceWork != null && ((SparkTask) currentTsk).getWork().contains(reduceWork) - && GenSparkSkewJoinProcessor.supportRuntimeSkewJoin( - op, currentTsk, parseContext.getConf())) { + if (reduceWork != null && !visitedJoinOp.contains(op) && + supportRuntimeSkewJoin(op, reduceWork, currentTsk, parseContext.getConf())) { // first we try to split the task splitTask((SparkTask) currentTsk, reduceWork, parseContext); GenSparkSkewJoinProcessor.processSkewJoin(op, currentTsk, reduceWork, parseContext); + visitedJoinOp.add(op); } return null; } @@ -112,8 +117,7 @@ SparkWork newWork = new SparkWork(parseContext.getConf().getVar(HiveConf.ConfVars.HIVEQUERYID)); newWork.add(childWork); - copyWorkGraph(currentWork, newWork, childWork, true); - copyWorkGraph(currentWork, newWork, childWork, false); + copyWorkGraph(currentWork, newWork, childWork); // remove them from current spark work for (BaseWork baseWork : newWork.getAllWorkUnsorted()) { currentWork.remove(baseWork); @@ -196,22 +200,39 @@ /** * Copy a sub-graph from originWork to newWork. */ - private static void copyWorkGraph(SparkWork originWork, SparkWork newWork, - BaseWork baseWork, boolean upWards) { - if (upWards) { - for (BaseWork parent : originWork.getParents(baseWork)) { - newWork.add(parent); - SparkEdgeProperty edgeProperty = originWork.getEdgeProperty(parent, baseWork); - newWork.connect(parent, baseWork, edgeProperty); - copyWorkGraph(originWork, newWork, parent, true); - } - } else { - for (BaseWork child : originWork.getChildren(baseWork)) { + private static void copyWorkGraph(SparkWork originWork, SparkWork newWork, BaseWork baseWork) { + for (BaseWork child : originWork.getChildren(baseWork)) { + if (!newWork.contains(child)) { newWork.add(child); SparkEdgeProperty edgeProperty = originWork.getEdgeProperty(baseWork, child); newWork.connect(baseWork, child, edgeProperty); - copyWorkGraph(originWork, newWork, child, false); + copyWorkGraph(originWork, newWork, child); } } + for (BaseWork parent : originWork.getParents(baseWork)) { + if (!newWork.contains(parent)) { + newWork.add(parent); + SparkEdgeProperty edgeProperty = originWork.getEdgeProperty(parent, baseWork); + newWork.connect(parent, baseWork, edgeProperty); + copyWorkGraph(originWork, newWork, parent); + } + } } + + public static Set getVisitedJoinOp() { + return visitedJoinOp; + } + + private static boolean supportRuntimeSkewJoin(JoinOperator joinOp, ReduceWork reduceWork, + Task currTask, HiveConf hiveConf) { + if (currTask instanceof SparkTask && + GenMRSkewJoinProcessor.skewJoinEnabled(hiveConf, joinOp)) { + SparkWork sparkWork = ((SparkTask) currTask).getWork(); + List> children = currTask.getChildTasks(); + return !joinOp.getConf().isFixedAsSorted() && sparkWork.contains(reduceWork) && + (children == null || children.size() <= 1) && + SparkMapJoinResolver.getOp(reduceWork, CommonJoinOperator.class).size() == 1; + } + return false; + } } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java (working copy) @@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.lib.GraphWalker; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.PreOrderWalker; import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.ql.lib.RuleRegExp; import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; @@ -53,8 +54,10 @@ public class SparkSkewJoinResolver implements PhysicalPlanResolver { @Override public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { + SparkSkewJoinProcFactory.getVisitedJoinOp().clear(); Dispatcher disp = new SparkSkewJoinTaskDispatcher(pctx); - GraphWalker ogw = new DefaultGraphWalker(disp); + // since we may split current task, use a pre-order walker + GraphWalker ogw = new PreOrderWalker(disp); ArrayList topNodes = new ArrayList(); topNodes.addAll(pctx.getRootTasks()); ogw.startWalking(topNodes, null); Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (working copy) @@ -6954,7 +6954,7 @@ } // Create a reduceSink operator followed by another limit - curr = genReduceSinkPlan(dest, qb, curr, 1); + curr = genReduceSinkPlan(dest, qb, curr, 1, false); return genLimitPlan(dest, qb, curr, limit); } @@ -7092,7 +7092,7 @@ } private Operator genReduceSinkPlan(String dest, QB qb, Operator input, - int numReducers) throws SemanticException { + int numReducers, boolean hasOrderBy) throws SemanticException { RowResolver inputRR = opParseCtx.get(input).getRowResolver(); @@ -7153,7 +7153,14 @@ sortCols.add(exprNode); } } - return genReduceSinkPlan(input, partCols, sortCols, order.toString(), numReducers, Operation.NOT_ACID); + Operator result = genReduceSinkPlan( + input, partCols, sortCols, order.toString(), numReducers, Operation.NOT_ACID); + if (result.getParentOperators().size() == 1 && + result.getParentOperators().get(0) instanceof ReduceSinkOperator) { + ((ReduceSinkOperator) result.getParentOperators().get(0)) + .getConf().setHasOrderBy(hasOrderBy); + } + return result; } @SuppressWarnings("nls") @@ -8869,6 +8876,7 @@ // Reduce sink is needed if the query contains a cluster by, distribute by, // order by or a sort by clause. boolean genReduceSink = false; + boolean hasOrderBy = false; // Currently, expressions are not allowed in cluster by, distribute by, // order by or a sort by clause. For each of the above clause types, check @@ -8883,6 +8891,7 @@ if (qbp.getOrderByForClause(dest) != null) { genReduceSink = true; + hasOrderBy = true; } if (qbp.getSortByForClause(dest) != null) { @@ -8893,11 +8902,11 @@ int numReducers = -1; // Use only 1 reducer if order by is present - if (qbp.getOrderByForClause(dest) != null) { + if (hasOrderBy) { numReducers = 1; } - curr = genReduceSinkPlan(dest, qb, curr, numReducers); + curr = genReduceSinkPlan(dest, qb, curr, numReducers, hasOrderBy); } @@ -8905,15 +8914,14 @@ if (limit != null) { // In case of order by, only 1 reducer is used, so no need of // another shuffle - curr = genLimitMapRedPlan(dest, qb, curr, limit.intValue(), qbp - .getOrderByForClause(dest) != null ? false : true); + curr = genLimitMapRedPlan(dest, qb, curr, limit.intValue(), !hasOrderBy); } } else { // exact limit can be taken care of by the fetch operator if (limit != null) { boolean extraMRStep = true; - if (qbp.getOrderByForClause(dest) != null || + if (hasOrderBy || qb.getIsQuery() && qbp.getClusterByForClause(dest) == null && qbp.getSortByForClause(dest) == null) { extraMRStep = false; Index: ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java (working copy) @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.FetchTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.ForwardOperator; import org.apache.hadoop.hive.ql.exec.GroupByOperator; import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; @@ -342,9 +343,7 @@ edgeProperty.setNumPartitions(reduceWork.getNumReduceTasks()); String sortOrder = Strings.nullToEmpty(reduceSink.getConf().getOrder()).trim(); - // test if we need group-by shuffle - if (reduceSink.getChildOperators().size() == 1 - && reduceSink.getChildOperators().get(0) instanceof GroupByOperator) { + if (hasGBYOperator(reduceSink)) { edgeProperty.setShuffleGroup(); // test if the group by needs partition level sort, if so, use the MR style shuffle // SHUFFLE_SORT shouldn't be used for this purpose, see HIVE-8542 @@ -369,13 +368,12 @@ } } - // test if we need total order, if so, we can either use MR shuffle + set #reducer to 1, - // or we can use SHUFFLE_SORT + // test if we need partition/global order, SHUFFLE_SORT should only be used for global order if (edgeProperty.isShuffleNone() && !sortOrder.isEmpty()) { - if (reduceSink.getConf().getPartitionCols() == null - || reduceSink.getConf().getPartitionCols().isEmpty() - || isSame(reduceSink.getConf().getPartitionCols(), - reduceSink.getConf().getKeyCols())) { + if ((reduceSink.getConf().getPartitionCols() == null + || reduceSink.getConf().getPartitionCols().isEmpty() + || isSame(reduceSink.getConf().getPartitionCols(), reduceSink.getConf().getKeyCols())) + && reduceSink.getConf().hasOrderBy()) { edgeProperty.setShuffleSort(); } else { edgeProperty.setMRShuffle(); @@ -475,4 +473,21 @@ public synchronized int getNextSeqNumber() { return ++sequenceNumber; } + + // test if we need group-by shuffle + private static boolean hasGBYOperator(ReduceSinkOperator rs) { + if (rs.getChildOperators().size() == 1) { + if (rs.getChildOperators().get(0) instanceof GroupByOperator) { + return true; + } else if (rs.getChildOperators().get(0) instanceof ForwardOperator) { + for (Operator grandChild : rs.getChildOperators().get(0).getChildOperators()) { + if (!(grandChild instanceof GroupByOperator)) { + return false; + } + } + return true; + } + } + return false; + } } Index: ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java (working copy) @@ -117,6 +117,9 @@ // whether we'll enforce the sort order of the RS private transient boolean enforceSort = false; + // used by spark mode to decide whether global order is needed + private transient boolean hasOrderBy = false; + private static transient Log LOG = LogFactory.getLog(ReduceSinkDesc.class); public ReduceSinkDesc() { } @@ -172,6 +175,7 @@ desc.setSkipTag(skipTag); desc.reduceTraits = reduceTraits.clone(); desc.setEnforceSort(enforceSort); + desc.setHasOrderBy(hasOrderBy); return desc; } @@ -437,4 +441,12 @@ public void setEnforceSort(boolean isDeduplicated) { this.enforceSort = isDeduplicated; } + + public boolean hasOrderBy() { + return hasOrderBy; + } + + public void setHasOrderBy(boolean hasOrderBy) { + this.hasOrderBy = hasOrderBy; + } } Index: ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java (revision 1674118) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java (working copy) @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.LinkedList; import java.util.LinkedHashMap; import java.util.List; @@ -47,8 +48,8 @@ private static int counter; private final String name; - private final Set roots = new HashSet(); - private final Set leaves = new HashSet(); + private final Set roots = new LinkedHashSet(); + private final Set leaves = new LinkedHashSet<>(); protected final Map> workGraph = new HashMap>(); @@ -182,14 +183,14 @@ * getRoots returns all nodes that do not have a parent. */ public Set getRoots() { - return new HashSet(roots); + return new LinkedHashSet(roots); } /** * getLeaves returns all nodes that do not have a child */ public Set getLeaves() { - return new HashSet(leaves); + return new LinkedHashSet(leaves); } public void setRequiredCounterPrefix(Map> requiredCounterPrefix) { @@ -236,6 +237,7 @@ List parents = getParents(work); for (BaseWork w: children) { + edgeProperties.remove(new ImmutablePair(work, w)); invertedWorkGraph.get(w).remove(work); if (invertedWorkGraph.get(w).size() == 0) { roots.add(w); @@ -243,6 +245,7 @@ } for (BaseWork w: parents) { + edgeProperties.remove(new ImmutablePair(w, work)); workGraph.get(w).remove(work); if (workGraph.get(w).size() == 0) { leaves.add(w); Index: ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java =================================================================== --- ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java (revision 1674118) +++ ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java (working copy) @@ -42,6 +42,7 @@ @Test public void testSingleSessionMultipleUse() throws Exception { HiveConf conf = new HiveConf(); + conf.set("spark.master", "local"); SparkSessionManager sessionManager = SparkSessionManagerImpl.getInstance(); SparkSession sparkSession1 = sessionManager.getSession(null, conf, true); @@ -68,6 +69,8 @@ sessionManagerHS2.shutdown(); HiveConf hiveConf = new HiveConf(); + hiveConf.set("spark.master", "local"); + sessionManagerHS2.setup(hiveConf); List threadList = new ArrayList(); @@ -105,6 +108,8 @@ String threadName = Thread.currentThread().getName(); System.out.println(threadName + " started."); HiveConf conf = new HiveConf(); + conf.set("spark.master", "local"); + SparkSession prevSession = null; SparkSession currentSession = null; Index: ql/src/test/queries/clientpositive/cbo_limit.q =================================================================== --- ql/src/test/queries/clientpositive/cbo_limit.q (revision 1674118) +++ ql/src/test/queries/clientpositive/cbo_limit.q (working copy) @@ -12,4 +12,4 @@ select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c limit 5; -select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc limit 5; +select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5; Index: ql/src/test/queries/clientpositive/cbo_subq_in.q =================================================================== --- ql/src/test/queries/clientpositive/cbo_subq_in.q (revision 1674118) +++ ql/src/test/queries/clientpositive/cbo_subq_in.q (working copy) @@ -8,7 +8,7 @@ -- non agg, non corr select * from src_cbo -where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key ; -- agg, corr @@ -21,7 +21,7 @@ (select distinct a.key from src_cbo a where b.value = a.value and a.key > '9' - ) + ) order by b.key ; -- non agg, corr, with join in Parent Query @@ -41,7 +41,7 @@ from src_cbo b where b.key in (select key from src_cbo where src_cbo.key > '8') group by key, value -having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key ; -- non agg, non corr, windowing @@ -49,6 +49,6 @@ from part group by p_mfgr, p_name having p_name in - (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr ; Index: ql/src/test/queries/clientpositive/cbo_subq_not_in.q =================================================================== --- ql/src/test/queries/clientpositive/cbo_subq_not_in.q (revision 1674118) +++ ql/src/test/queries/clientpositive/cbo_subq_not_in.q (working copy) @@ -21,7 +21,7 @@ (select p_name from (select p_mfgr, p_name, p_size as r from part) a where r < 10 and b.p_mfgr = a.p_mfgr - ) + ) order by p_mfgr,p_size ; -- agg, non corr @@ -48,7 +48,7 @@ from lineitem li where li.l_linenumber = 1 and li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') -group by li.l_partkey +group by li.l_partkey order by li.l_partkey ; -- add null check test from sq_notin.q once HIVE-7721 resolved. Index: ql/src/test/queries/clientpositive/cbo_udf_udaf.q =================================================================== --- ql/src/test/queries/clientpositive/cbo_udf_udaf.q (revision 1674118) +++ ql/src/test/queries/clientpositive/cbo_udf_udaf.q (working copy) @@ -6,12 +6,12 @@ -- 8. Test UDF/UDAF select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int) from cbo_t1; -select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int; +select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a; select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from cbo_t1) cbo_t1; -select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1; +select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a; select f,a,e,b from (select count(*) as a, count(c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from cbo_t1) cbo_t1; select f,a,e,b from (select count(*) as a, count(distinct c_int) as b, sum(distinct c_int) as c, avg(distinct c_int) as d, max(distinct c_int) as e, min(distinct c_int) as f from cbo_t1) cbo_t1; -select count(c_int) as a, avg(c_float), key from cbo_t1 group by key; -select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float; -select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int; -select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int; +select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a; +select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a; +select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a; +select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a; Index: ql/src/test/queries/clientpositive/runtime_skewjoin_mapjoin_spark.q =================================================================== --- ql/src/test/queries/clientpositive/runtime_skewjoin_mapjoin_spark.q (revision 0) +++ ql/src/test/queries/clientpositive/runtime_skewjoin_mapjoin_spark.q (working copy) @@ -0,0 +1,24 @@ +set hive.optimize.skewjoin = true; +set hive.skewjoin.key = 4; +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=50; + +-- This is mainly intended for spark, to test runtime skew join together with map join + +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +EXPLAIN +SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key; + +SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key; Index: ql/src/test/queries/clientpositive/union12.q =================================================================== --- ql/src/test/queries/clientpositive/union12.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union12.q (working copy) @@ -1,5 +1,7 @@ set hive.map.aggr = true; +-- SORT_QUERY_RESULTS + -- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink create table tmptable(key string, value int); Index: ql/src/test/queries/clientpositive/union17.q =================================================================== --- ql/src/test/queries/clientpositive/union17.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union17.q (working copy) @@ -1,7 +1,7 @@ CREATE TABLE DEST1(key STRING, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key STRING, val1 STRING, val2 STRING) STORED AS TEXTFILE; --- SORT_BEFORE_DIFF +-- SORT_QUERY_RESULTS -- union case:map-reduce sub-queries followed by multi-table insert explain Index: ql/src/test/queries/clientpositive/union20.q =================================================================== --- ql/src/test/queries/clientpositive/union20.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union20.q (working copy) @@ -1,4 +1,4 @@ --- SORT_BEFORE_DIFF +-- SORT_QUERY_RESULTS -- union :map-reduce sub-queries followed by join explain Index: ql/src/test/queries/clientpositive/union21.q =================================================================== --- ql/src/test/queries/clientpositive/union21.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union21.q (working copy) @@ -1,4 +1,4 @@ --- SORT_BEFORE_DIFF +-- SORT_QUERY_RESULTS -- union of constants, udf outputs, and columns from text table and thrift table explain Index: ql/src/test/queries/clientpositive/union27.q =================================================================== --- ql/src/test/queries/clientpositive/union27.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union27.q (working copy) @@ -1,4 +1,4 @@ --- SORT_BEFORE_DIFF +-- SORT_QUERY_RESULTS create table jackson_sev_same as select * from src; create table dim_pho as select * from src; create table jackson_sev_add as select * from src; Index: ql/src/test/queries/clientpositive/union_remove_22.q =================================================================== --- ql/src/test/queries/clientpositive/union_remove_22.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union_remove_22.q (working copy) @@ -7,6 +7,8 @@ set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; +-- SORT_QUERY_RESULTS + -- This is to test the union->selectstar->filesink optimization -- Union of 2 map-reduce subqueries is performed followed by select and a file sink -- However, some columns are repeated. So, union cannot be removed. Index: ql/src/test/queries/clientpositive/union_remove_6_subq.q =================================================================== --- ql/src/test/queries/clientpositive/union_remove_6_subq.q (revision 1674118) +++ ql/src/test/queries/clientpositive/union_remove_6_subq.q (working copy) @@ -6,6 +6,8 @@ set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; +-- SORT_QUERY_RESULTS + -- This is to test the union->selectstar->filesink optimization -- Union of 2 subqueries is performed (all of which are mapred queries) -- followed by select star and a file sink in 2 output tables. Index: ql/src/test/results/clientpositive/cbo_limit.q.out =================================================================== --- ql/src/test/results/clientpositive/cbo_limit.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/cbo_limit.q.out (working copy) @@ -70,7 +70,7 @@ #### A masked pattern was here #### 1 12 6 1 2 6 -PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc limit 5 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5 PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 @@ -78,7 +78,7 @@ PREHOOK: Input: default@cbo_t2@dt=2014 PREHOOK: Input: default@cbo_t3 #### A masked pattern was here #### -POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc limit 5 +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5 POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 Index: ql/src/test/results/clientpositive/cbo_subq_in.q.out =================================================================== --- ql/src/test/results/clientpositive/cbo_subq_in.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/cbo_subq_in.q.out (working copy) @@ -2,7 +2,7 @@ -- non agg, non corr select * from src_cbo -where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key PREHOOK: type: QUERY PREHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -10,7 +10,7 @@ -- non agg, non corr select * from src_cbo -where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -35,7 +35,7 @@ (select distinct a.key from src_cbo a where b.value = a.value and a.key > '9' - ) + ) order by b.key PREHOOK: type: QUERY PREHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -49,7 +49,7 @@ (select distinct a.key from src_cbo a where b.value = a.value and a.key > '9' - ) + ) order by b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -92,7 +92,7 @@ from src_cbo b where b.key in (select key from src_cbo where src_cbo.key > '8') group by key, value -having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key PREHOOK: type: QUERY PREHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -106,30 +106,30 @@ from src_cbo b where b.key in (select key from src_cbo where src_cbo.key > '8') group by key, value -having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src_cbo #### A masked pattern was here #### 80 val_80 1 -96 val_96 1 -92 val_92 1 -9 val_9 1 -87 val_87 1 -86 val_86 1 -85 val_85 1 82 val_82 1 +83 val_83 2 84 val_84 2 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 3 +92 val_92 1 95 val_95 2 -83 val_83 2 +96 val_96 1 +97 val_97 2 98 val_98 2 -97 val_97 2 -90 val_90 3 PREHOOK: query: -- non agg, non corr, windowing select p_mfgr, p_name, avg(p_size) from part group by p_mfgr, p_name having p_name in - (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr PREHOOK: type: QUERY PREHOOK: Input: default@part #### A masked pattern was here #### @@ -138,12 +138,12 @@ from part group by p_mfgr, p_name having p_name in - (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### Manufacturer#1 almond antique burnished rose metallic 2.0 +Manufacturer#2 almond aquamarine midnight light salmon 2.0 Manufacturer#3 almond antique misty red olive 1.0 +Manufacturer#4 almond aquamarine yellow dodger mint 7.0 Manufacturer#5 almond antique sky peru orange 2.0 -Manufacturer#2 almond aquamarine midnight light salmon 2.0 -Manufacturer#4 almond aquamarine yellow dodger mint 7.0 Index: ql/src/test/results/clientpositive/cbo_subq_not_in.q.out =================================================================== --- ql/src/test/results/clientpositive/cbo_subq_not_in.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/cbo_subq_not_in.q.out (working copy) @@ -146,7 +146,7 @@ (select p_name from (select p_mfgr, p_name, p_size as r from part) a where r < 10 and b.p_mfgr = a.p_mfgr - ) + ) order by p_mfgr,p_size PREHOOK: type: QUERY PREHOOK: Input: default@part #### A masked pattern was here #### @@ -157,28 +157,28 @@ (select p_name from (select p_mfgr, p_name, p_size as r from part) a where r < 10 and b.p_mfgr = a.p_mfgr - ) + ) order by p_mfgr,p_size POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -Manufacturer#5 almond antique blue firebrick mint 31 -Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#1 almond aquamarine burnished black steel 28 Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond antique violet turquoise frosted 40 Manufacturer#3 almond antique forest lavender goldenrod 14 -Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#3 almond antique chartreuse khaki white 17 Manufacturer#3 almond antique metallic orange dim 19 Manufacturer#3 almond antique olive coral navajo 45 -Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 Manufacturer#4 almond antique violet mint lemon 39 -Manufacturer#2 almond antique violet turquoise frosted 40 -Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#5 almond azure blanched chiffon midnight 23 +Manufacturer#5 almond antique blue firebrick mint 31 Manufacturer#5 almond aquamarine dodger light gainsboro 46 -Manufacturer#4 almond aquamarine floral ivory bisque 27 -Manufacturer#1 almond aquamarine pink moccasin thistle 42 -Manufacturer#2 almond aquamarine rose maroon antique 25 -Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 -Manufacturer#4 almond azure aquamarine papaya violet 12 -Manufacturer#5 almond azure blanched chiffon midnight 23 PREHOOK: query: -- agg, non corr select p_name, p_size from @@ -272,7 +272,7 @@ from lineitem li where li.l_linenumber = 1 and li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') -group by li.l_partkey +group by li.l_partkey order by li.l_partkey PREHOOK: type: QUERY PREHOOK: Input: default@lineitem #### A masked pattern was here #### @@ -281,7 +281,7 @@ from lineitem li where li.l_linenumber = 1 and li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') -group by li.l_partkey +group by li.l_partkey order by li.l_partkey POSTHOOK: type: QUERY POSTHOOK: Input: default@lineitem #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/cbo_udf_udaf.q.out =================================================================== --- ql/src/test/results/clientpositive/cbo_udf_udaf.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/cbo_udf_udaf.q.out (working copy) @@ -11,12 +11,12 @@ POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 20 18 18 1.0 1 1 -PREHOOK: query: select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int +PREHOOK: query: select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int +POSTHOOK: query: select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 @@ -34,12 +34,12 @@ POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 20 1 18 1.0 1 1 -PREHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 +PREHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 +POSTHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 @@ -68,51 +68,51 @@ POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 1 20 1 1 -PREHOOK: query: select count(c_int) as a, avg(c_float), key from cbo_t1 group by key +PREHOOK: query: select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(c_int) as a, avg(c_float), key from cbo_t1 group by key +POSTHOOK: query: select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -0 NULL NULL -2 1.0 1 -2 1.0 1 -12 1.0 1 -2 1.0 1 -PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float +NULL 0 NULL +1 2 1.0 + 1 2 1.0 + 1 2 1.0 +1 12 1.0 +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 0 NULL 1 1.0 -PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 0 NULL 1 1.0 -PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 Index: ql/src/test/results/clientpositive/runtime_skewjoin_mapjoin_spark.q.out =================================================================== --- ql/src/test/results/clientpositive/runtime_skewjoin_mapjoin_spark.q.out (revision 0) +++ ql/src/test/results/clientpositive/runtime_skewjoin_mapjoin_spark.q.out (working copy) @@ -0,0 +1,669 @@ +PREHOOK: query: -- This is mainly intended for spark, to test runtime skew join together with map join + +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T1 +POSTHOOK: query: -- This is mainly intended for spark, to test runtime skew join together with map join + +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE +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: EXPLAIN +SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-18 is a root stage , consists of Stage-27, Stage-28, Stage-2 + Stage-27 has a backup stage: Stage-2 + Stage-16 depends on stages: Stage-27 + Stage-8 depends on stages: Stage-2, Stage-13, Stage-14, Stage-16, Stage-17, Stage-22, Stage-23 , consists of Stage-26, Stage-3 + Stage-26 + Stage-7 depends on stages: Stage-26 + Stage-3 depends on stages: Stage-7 + Stage-28 has a backup stage: Stage-2 + Stage-17 depends on stages: Stage-28 + Stage-2 + Stage-21 is a root stage , consists of Stage-32, Stage-33, Stage-1 + Stage-32 has a backup stage: Stage-1 + Stage-19 depends on stages: Stage-32 + Stage-10 depends on stages: Stage-1, Stage-19, Stage-20 , consists of Stage-31, Stage-2 + Stage-31 + Stage-9 depends on stages: Stage-31 + Stage-15 depends on stages: Stage-9, Stage-11 , consists of Stage-29, Stage-30, Stage-2 + Stage-29 has a backup stage: Stage-2 + Stage-13 depends on stages: Stage-29 + Stage-30 has a backup stage: Stage-2 + Stage-14 depends on stages: Stage-30 + Stage-33 has a backup stage: Stage-1 + Stage-20 depends on stages: Stage-33 + Stage-1 + Stage-24 is a root stage , consists of Stage-34, Stage-35, Stage-2 + Stage-34 has a backup stage: Stage-2 + Stage-22 depends on stages: Stage-34 + Stage-35 has a backup stage: Stage-2 + Stage-23 depends on stages: Stage-35 + Stage-37 is a root stage + Stage-25 depends on stages: Stage-37 + Stage-12 depends on stages: Stage-25 , consists of Stage-36, Stage-2 + Stage-36 + Stage-11 depends on stages: Stage-36 + Stage-0 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-18 + Conditional Operator + + Stage: Stage-27 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-16 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-8 + Conditional Operator + + Stage: Stage-26 + Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan + HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + + Stage: Stage-7 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-3 + Map Reduce + Map Operator Tree: + TableScan + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + 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 + 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-28 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-17 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + 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 + TableScan + 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + handleSkewJoin: true + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-21 + Conditional Operator + + Stage: Stage-32 + Map Reduce Local Work + Alias -> Map Local Tables: + a:src2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:src2 + TableScan + alias: src2 + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + + Stage: Stage-19 + Map Reduce + Map Operator Tree: + TableScan + alias: src1 + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-10 + Conditional Operator + + Stage: Stage-31 + Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan + HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + + Stage: Stage-9 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-15 + Conditional Operator + + Stage: Stage-29 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-13 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-30 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-14 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-33 + Map Reduce Local Work + Alias -> Map Local Tables: + a:src1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:src1 + TableScan + alias: src1 + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + + Stage: Stage-20 + Map Reduce + Map Operator Tree: + TableScan + alias: src2 + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + 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 + 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 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + handleSkewJoin: true + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-24 + Conditional Operator + + Stage: Stage-34 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-22 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-35 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + TableScan + HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + + Stage: Stage-23 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-37 + Map Reduce Local Work + Alias -> Map Local Tables: + b:t1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:t1 + TableScan + alias: t1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + + Stage: Stage-25 + Map Reduce + Map Operator Tree: + TableScan + alias: src + 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 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-12 + Conditional Operator + + Stage: Stage-36 + Map Reduce Local Work + Alias -> Map Local Tables: + 1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + 1 + TableScan + HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + + Stage: Stage-11 + Map Reduce + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/auto_join0.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join0.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join0.q.out (working copy) @@ -53,7 +53,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join15.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join15.q.out (working copy) @@ -42,7 +42,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join20.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join20.q.out (working copy) @@ -66,7 +66,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -225,7 +225,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join21.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join21.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join21.q.out (working copy) @@ -57,7 +57,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/auto_join23.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join23.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join23.q.out (working copy) @@ -37,7 +37,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/auto_join28.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join28.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join28.q.out (working copy) @@ -53,7 +53,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -166,7 +166,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -279,7 +279,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 2) + Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 2 @@ -389,7 +389,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/auto_join29.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join29.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join29.q.out (working copy) @@ -57,7 +57,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -678,7 +678,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1299,7 +1299,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 2) + Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 2 @@ -1929,7 +1929,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -2554,7 +2554,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2674,7 +2674,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -3295,7 +3295,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3416,7 +3416,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 2) + Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 2 @@ -3554,7 +3554,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/auto_join30.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join30.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join30.q.out (working copy) @@ -23,7 +23,7 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -59,7 +59,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -171,7 +171,7 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -204,7 +204,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -313,7 +313,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -345,7 +345,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -461,8 +461,8 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 2) - Reducer 7 <- Map 6 (SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -529,7 +529,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -656,8 +656,8 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 2) - Reducer 7 <- Map 6 (SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -718,7 +718,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -842,8 +842,8 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 4 (SORT, 2) - Reducer 7 <- Map 6 (SORT, 2) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 @@ -904,7 +904,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1028,8 +1028,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 4 <- Map 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1089,7 +1089,7 @@ Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2) Reducer 7 <- Reducer 6 (GROUP, 1) #### A masked pattern was here #### Vertices: @@ -1214,8 +1214,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 4 <- Map 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1275,7 +1275,7 @@ Stage: Stage-1 Spark Edges: - Reducer 6 <- Map 5 (SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2) Reducer 7 <- Reducer 6 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/auto_join31.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/auto_join31.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/auto_join31.q.out (working copy) @@ -29,8 +29,8 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 7 <- Map 6 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -90,7 +90,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (GROUP, 1) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/bucket5.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/bucket5.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/bucket5.q.out (working copy) @@ -78,7 +78,7 @@ Spark Edges: Reducer 2 <- Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Map 5 (SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 Index: ql/src/test/results/clientpositive/spark/cbo_gby.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_gby.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_gby.q.out (working copy) @@ -0,0 +1,120 @@ +PREHOOK: query: -- 6. Test Select + TS + Join + Fil + GB + GB Having +select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 6. Test Select + TS + Join + Fil + GB + GB Having +select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 4 12 +NULL NULL NULL + 1 4 2 +1 4 2 + 1 4 2 +PREHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +NULL NULL 1 +5.0 12 1 +5.0 2 3 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 12 6 +1 2 6 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 2 6 +1 12 6 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 2 6 +1 12 6 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) cbo_t1 full outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) cbo_t1 full outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by p+q desc, r asc) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 2 6 +1 12 6 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 2 6 +1 12 6 Index: ql/src/test/results/clientpositive/spark/cbo_limit.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_limit.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_limit.q.out (working copy) @@ -0,0 +1,90 @@ +PREHOOK: query: -- 7. Test Select + TS + Join + Fil + GB + GB Having + Limit +select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key order by x limit 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 7. Test Select + TS + Join + Fil + GB + GB Having + Limit +select key, (c_int+1)+2 as x, sum(c_int) from cbo_t1 group by c_float, cbo_t1.c_int, key order by x limit 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +NULL NULL NULL +PREHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x order by x,y limit 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from cbo_t1 group by c_float, cbo_t1.c_int, key) R group by y, x order by x,y limit 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +NULL NULL 1 +PREHOOK: query: select key from(select key from (select key from cbo_t1 limit 5)cbo_t2 limit 5)cbo_t3 limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select key from(select key from (select key from cbo_t1 limit 5)cbo_t2 limit 5)cbo_t3 limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 +1 +1 +1 +1 +PREHOOK: query: select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 order by c_int limit 5)cbo_t1 order by c_int limit 5)cbo_t2 order by c_int limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 order by c_int limit 5)cbo_t1 order by c_int limit 5)cbo_t2 order by c_int limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +NULL NULL +NULL NULL +1 1 +1 1 +1 1 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c order by cbo_t3.c_int+c desc, c limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 12 6 +1 2 6 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 12 6 +1 2 6 Index: ql/src/test/results/clientpositive/spark/cbo_semijoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_semijoin.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_semijoin.q.out (working copy) @@ -0,0 +1,440 @@ +PREHOOK: query: -- 12. SemiJoin +select cbo_t1.c_int from cbo_t1 left semi join cbo_t2 on cbo_t1.key=cbo_t2.key +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 12. SemiJoin +select cbo_t1.c_int from cbo_t1 left semi join cbo_t2 on cbo_t1.key=cbo_t2.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +PREHOOK: query: select cbo_t1.c_int from cbo_t1 left semi join cbo_t2 on cbo_t1.key=cbo_t2.key where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t1.c_int from cbo_t1 left semi join cbo_t2 on cbo_t1.key=cbo_t2.key where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +PREHOOK: query: select * from (select c, b, a from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c >= 0)) R where (b + 1 = 2) and (R.b > 0 or c >= 0) +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select c, b, a from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c >= 0)) R where (b + 1 = 2) and (R.b > 0 or c >= 0) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +1.0 1 1 +PREHOOK: query: select * from (select cbo_t3.c_int, cbo_t1.c, b from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 = 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t3.c_int == 2) and (b > 0 or c_int >= 0)) R where (R.c_int + 1 = 2) and (R.b > 0 or c_int >= 0) +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select cbo_t3.c_int, cbo_t1.c, b from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 = 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t3.c_int == 2) and (b > 0 or c_int >= 0)) R where (R.c_int + 1 = 2) and (R.b > 0 or c_int >= 0) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +1 1.0 1 +PREHOOK: query: select * from (select c_int, b, cbo_t1.c from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c_int >= 0)) R where (c + 1 = 2) and (R.b > 0 or c_int >= 0) +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select c_int, b, cbo_t1.c from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c_int >= 0)) R where (c + 1 = 2) and (R.b > 0 or c_int >= 0) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +PREHOOK: query: select * from (select c_int, b, cbo_t1.c from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c_int >= 0)) R where (c + 1 = 2) and (R.b > 0 or c_int >= 0) +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select c_int, b, cbo_t1.c from (select key as a, c_int as b, cbo_t1.c_float as c from cbo_t1 where (cbo_t1.c_int + 1 == 2) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)) cbo_t1 left semi join (select cbo_t2.key as p, cbo_t2.c_int as q, c_float as r from cbo_t2 where (cbo_t2.c_int + 1 == 2) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)) cbo_t2 on cbo_t1.a=p full outer join cbo_t3 on cbo_t1.a=key where (b + 1 == 2) and (b > 0 or c_int >= 0)) R where (c + 1 = 2) and (R.b > 0 or c_int >= 0) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +1 1 1.0 +PREHOOK: query: select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 >= 0) and (b > 0 or a >= 0) group by a, c having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 >= 0) and (b > 0 or a >= 0) group by a, c having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### + 1 2 1 + 1 2 1 +1 2 1 +1 12 1 +PREHOOK: query: select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc limit 5) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p limit 5) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 >= 0) and (b > 0 or a >= 0) group by a, c having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc limit 5) cbo_t1 left semi join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by q+r/10 desc, p limit 5) cbo_t2 on cbo_t1.a=p left semi join cbo_t3 on cbo_t1.a=key where (b + 1 >= 0) and (b > 0 or a >= 0) group by a, c having a > 0 and (a >=1 or c >= 1) and (a + c) >= 0 order by c, a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### + 1 2 1 + 1 2 1 +1 2 1 +1 12 1 Index: ql/src/test/results/clientpositive/spark/cbo_simple_select.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_simple_select.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_simple_select.q.out (working copy) @@ -0,0 +1,745 @@ +PREHOOK: query: -- 1. Test Select + TS +select * from cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 1. Test Select + TS +select * from cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +NULL NULL NULL NULL NULL 2014 +NULL NULL NULL NULL NULL 2014 +PREHOOK: query: select * from cbo_t1 as cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from cbo_t1 as cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +NULL NULL NULL NULL NULL 2014 +NULL NULL NULL NULL NULL 2014 +PREHOOK: query: select * from cbo_t1 as cbo_t2 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from cbo_t1 as cbo_t2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +NULL NULL NULL NULL NULL 2014 +NULL NULL NULL NULL NULL 2014 +PREHOOK: query: select cbo_t1.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t1.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +NULL NULL NULL +NULL NULL NULL +PREHOOK: query: -- 2. Test Select + TS + FIL +select * from cbo_t1 where cbo_t1.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 2. Test Select + TS + FIL +select * from cbo_t1 where cbo_t1.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +PREHOOK: query: -- 3 Test Select + Select + TS + FIL +select * from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 3 Test Select + Select + TS + FIL +select * from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +PREHOOK: query: select * from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and y+c_int >= 0 or x <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and y+c_int >= 0 or x <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +PREHOOK: query: select cbo_t1.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t1.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +PREHOOK: query: select cbo_t2.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t2 where cbo_t2.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t2.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t2 where cbo_t2.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +PREHOOK: query: select * from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select * from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +PREHOOK: query: select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and y+c_int >= 0 or x <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from cbo_t1 as cbo_t2 where cbo_t2.c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as cbo_t1 where cbo_t1.c_int >= 0 and y+c_int >= 0 or x <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 + 1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +1 1 25.0 +PREHOOK: query: select cbo_t1.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t1.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +PREHOOK: query: select cbo_t2.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t2 where cbo_t2.c_int >= 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select cbo_t2.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t2 where cbo_t2.c_int >= 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +2.0 1 25.0 +PREHOOK: query: -- 13. null expr in select list +select null from cbo_t3 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: -- 13. null expr in select list +select null from cbo_t3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +PREHOOK: query: -- 14. unary operator +select key from cbo_t1 where c_int = -6 or c_int = +6 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 14. unary operator +select key from cbo_t1 where c_int = -6 or c_int = +6 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +PREHOOK: query: -- 15. query referencing only partition columns +select count(cbo_t1.dt) from cbo_t1 join cbo_t2 on cbo_t1.dt = cbo_t2.dt where cbo_t1.dt = '2014' +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 15. query referencing only partition columns +select count(cbo_t1.dt) from cbo_t1 join cbo_t2 on cbo_t1.dt = cbo_t2.dt where cbo_t1.dt = '2014' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +400 Index: ql/src/test/results/clientpositive/spark/cbo_stats.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_stats.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_stats.q.out (working copy) @@ -0,0 +1,14 @@ +PREHOOK: query: -- 20. Test get stats with empty partition list +select cbo_t1.value from cbo_t1 join cbo_t2 on cbo_t1.key = cbo_t2.key where cbo_t1.dt = '10' and cbo_t1.c_boolean = true +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 20. Test get stats with empty partition list +select cbo_t1.value from cbo_t1 join cbo_t2 on cbo_t1.key = cbo_t2.key where cbo_t1.dt = '10' and cbo_t1.c_boolean = true +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/cbo_subq_in.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_subq_in.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_subq_in.q.out (working copy) @@ -0,0 +1,149 @@ +PREHOOK: query: -- 17. SubQueries In +-- non agg, non corr +select * +from src_cbo +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src_cbo +#### A masked pattern was here #### +POSTHOOK: query: -- 17. SubQueries In +-- non agg, non corr +select * +from src_cbo +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_cbo +#### A masked pattern was here #### +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +PREHOOK: query: -- agg, corr +-- add back once rank issue fixed for cbo + +-- distinct, corr +select * +from src_cbo b +where b.key in + (select distinct a.key + from src_cbo a + where b.value = a.value and a.key > '9' + ) order by b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src_cbo +#### A masked pattern was here #### +POSTHOOK: query: -- agg, corr +-- add back once rank issue fixed for cbo + +-- distinct, corr +select * +from src_cbo b +where b.key in + (select distinct a.key + from src_cbo a + where b.value = a.value and a.key > '9' + ) order by b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_cbo +#### A masked pattern was here #### +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +PREHOOK: query: -- non agg, corr, with join in Parent Query +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +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: -- non agg, corr, with join in Parent Query +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +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 #### +4297 1798 +108570 8571 +PREHOOK: query: -- where and having +-- Plan is: +-- Stage 1: b semijoin sq1:src_cbo (subquery in where) +-- Stage 2: group by Stage 1 o/p +-- Stage 5: group by on sq2:src_cbo (subquery in having) +-- Stage 6: Stage 2 o/p semijoin Stage 5 +select key, value, count(*) +from src_cbo b +where b.key in (select key from src_cbo where src_cbo.key > '8') +group by key, value +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src_cbo +#### A masked pattern was here #### +POSTHOOK: query: -- where and having +-- Plan is: +-- Stage 1: b semijoin sq1:src_cbo (subquery in where) +-- Stage 2: group by Stage 1 o/p +-- Stage 5: group by on sq2:src_cbo (subquery in having) +-- Stage 6: Stage 2 o/p semijoin Stage 5 +select key, value, count(*) +from src_cbo b +where b.key in (select key from src_cbo where src_cbo.key > '8') +group by key, value +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_cbo +#### A masked pattern was here #### +80 val_80 1 +82 val_82 1 +83 val_83 2 +84 val_84 2 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 3 +92 val_92 1 +95 val_95 2 +96 val_96 1 +97 val_97 2 +98 val_98 2 +PREHOOK: query: -- non agg, non corr, windowing +select p_mfgr, p_name, avg(p_size) +from part +group by p_mfgr, p_name +having p_name in + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- non agg, non corr, windowing +select p_mfgr, p_name, avg(p_size) +from part +group by p_mfgr, p_name +having p_name in + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2.0 +Manufacturer#2 almond aquamarine midnight light salmon 2.0 +Manufacturer#3 almond antique misty red olive 1.0 +Manufacturer#4 almond aquamarine yellow dodger mint 7.0 +Manufacturer#5 almond antique sky peru orange 2.0 Index: ql/src/test/results/clientpositive/spark/cbo_subq_not_in.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_subq_not_in.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_subq_not_in.q.out (working copy) @@ -0,0 +1,365 @@ +PREHOOK: query: -- 16. SubQueries Not In +-- non agg, non corr +select * +from src_cbo +where src_cbo.key not in + ( select key from src_cbo s1 + where s1.key > '2' + ) order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src_cbo +#### A masked pattern was here #### +POSTHOOK: query: -- 16. SubQueries Not In +-- non agg, non corr +select * +from src_cbo +where src_cbo.key not in + ( select key from src_cbo s1 + where s1.key > '2' + ) order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_cbo +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +150 val_150 +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 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +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 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +PREHOOK: query: -- non agg, corr +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size as r from part) a + where r < 10 and b.p_mfgr = a.p_mfgr + ) order by p_mfgr,p_size +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- non agg, corr +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size as r from part) a + where r < 10 and b.p_mfgr = a.p_mfgr + ) order by p_mfgr,p_size +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#3 almond antique forest lavender goldenrod 14 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#3 almond antique metallic orange dim 19 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +Manufacturer#4 almond antique violet mint lemon 39 +Manufacturer#5 almond azure blanched chiffon midnight 23 +Manufacturer#5 almond antique blue firebrick mint 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 +PREHOOK: query: -- agg, non corr +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size from part) a + where p_size < 10 + ) order by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- agg, non corr +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size from part) a + where p_size < 10 + ) order by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +almond antique blue firebrick mint 31 +almond antique burnished rose metallic 2 +almond antique burnished rose metallic 2 +almond antique chartreuse khaki white 17 +almond antique chartreuse lavender yellow 34 +almond antique forest lavender goldenrod 14 +almond antique gainsboro frosted violet 10 +almond antique medium spring khaki 6 +almond antique metallic orange dim 19 +almond antique misty red olive 1 +almond antique olive coral navajo 45 +almond antique salmon chartreuse burlywood 6 +almond antique sky peru orange 2 +almond antique violet chocolate turquoise 14 +almond antique violet mint lemon 39 +almond antique violet turquoise frosted 40 +almond aquamarine burnished black steel 28 +almond aquamarine dodger light gainsboro 46 +almond aquamarine floral ivory bisque 27 +almond aquamarine midnight light salmon 2 +almond aquamarine pink moccasin thistle 42 +almond aquamarine rose maroon antique 25 +almond aquamarine sandy cyan gainsboro 18 +almond aquamarine yellow dodger mint 7 +almond azure aquamarine papaya violet 12 +almond azure blanched chiffon midnight 23 +PREHOOK: query: -- agg, corr +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size from part) a + where p_size < 10 and b.p_mfgr = a.p_mfgr + ) order by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- agg, corr +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size from part) a + where p_size < 10 and b.p_mfgr = a.p_mfgr + ) order by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#5 almond antique blue firebrick mint 31 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#3 almond antique forest lavender goldenrod 14 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#5 almond antique medium spring khaki 6 +Manufacturer#3 almond antique metallic orange dim 19 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#4 almond antique violet mint lemon 39 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#5 almond azure blanched chiffon midnight 23 +PREHOOK: query: -- non agg, non corr, Group By in Parent Query +select li.l_partkey, count(*) +from lineitem li +where li.l_linenumber = 1 and + li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') +group by li.l_partkey order by li.l_partkey +PREHOOK: type: QUERY +PREHOOK: Input: default@lineitem +#### A masked pattern was here #### +POSTHOOK: query: -- non agg, non corr, Group By in Parent Query +select li.l_partkey, count(*) +from lineitem li +where li.l_linenumber = 1 and + li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') +group by li.l_partkey order by li.l_partkey +POSTHOOK: type: QUERY +POSTHOOK: Input: default@lineitem +#### A masked pattern was here #### +450 1 +7068 1 +21636 1 +22630 1 +59694 1 +61931 1 +85951 1 +88035 1 +88362 1 +106170 1 +119477 1 +119767 1 +123076 1 +139636 1 +175839 1 +182052 1 +PREHOOK: query: -- add null check test from sq_notin.q once HIVE-7721 resolved. + +-- non agg, corr, having +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) + order by b.p_mfgr +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- add null check test from sq_notin.q once HIVE-7721 resolved. + +-- non agg, corr, having +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) + order by b.p_mfgr +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 1173.15 +Manufacturer#2 1690.68 +PREHOOK: query: -- agg, non corr, having +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from part a + group by p_mfgr + having max(p_retailprice) - min(p_retailprice) > 600 + ) + order by b.p_mfgr +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- agg, non corr, having +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from part a + group by p_mfgr + having max(p_retailprice) - min(p_retailprice) > 600 + ) + order by b.p_mfgr +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 1173.15 +Manufacturer#2 1690.68 Index: ql/src/test/results/clientpositive/spark/cbo_udf_udaf.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_udf_udaf.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_udf_udaf.q.out (working copy) @@ -0,0 +1,121 @@ +PREHOOK: query: -- 8. Test UDF/UDAF +select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int) from cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- 8. Test UDF/UDAF +select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int) from cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +20 18 18 1.0 1 1 +PREHOOK: query: select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +2 0 NULL NULL NULL NULL 3 6 +18 18 18 1.0 1 1 2 36 +PREHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from cbo_t1) cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from cbo_t1) cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +20 1 18 1.0 1 1 +PREHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +2 0 NULL NULL NULL NULL 3 6 +18 1 18 1.0 1 1 2 36 +PREHOOK: query: select f,a,e,b from (select count(*) as a, count(c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from cbo_t1) cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select f,a,e,b from (select count(*) as a, count(c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from cbo_t1) cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 20 1 18 +PREHOOK: query: select f,a,e,b from (select count(*) as a, count(distinct c_int) as b, sum(distinct c_int) as c, avg(distinct c_int) as d, max(distinct c_int) as e, min(distinct c_int) as f from cbo_t1) cbo_t1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select f,a,e,b from (select count(*) as a, count(distinct c_int) as b, sum(distinct c_int) as c, avg(distinct c_int) as d, max(distinct c_int) as e, min(distinct c_int) as f from cbo_t1) cbo_t1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +1 20 1 1 +PREHOOK: query: select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +NULL 0 NULL + 1 2 1.0 + 1 2 1.0 +1 2 1.0 +1 12 1.0 +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +0 NULL +1 1.0 +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +0 NULL +1 1.0 +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +#### A masked pattern was here #### +0 NULL +1 1.0 Index: ql/src/test/results/clientpositive/spark/cbo_union.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/cbo_union.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/cbo_union.q.out (working copy) @@ -0,0 +1,920 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- 11. Union All +select * from (select * from cbo_t1 order by key, c_boolean, value, dt)a union all select * from (select * from cbo_t2 order by key, c_boolean, value, dt)b +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- 11. Union All +select * from (select * from cbo_t1 order by key, c_boolean, value, dt)a union all select * from (select * from cbo_t2 order by key, c_boolean, value, dt)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +#### A masked pattern was here #### + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 + 1 1 1 1.0 true 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 false 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +1 1 1 1.0 true 2014 +2 2 2 2.0 true 2014 +2 2 2 2.0 true 2014 +2 2 2 2.0 true 2014 +2 2 2 2.0 true 2014 +2 2 2 2.0 true 2014 +NULL NULL NULL NULL NULL 2014 +NULL NULL NULL NULL NULL 2014 +NULL NULL NULL NULL NULL 2014 +NULL NULL NULL NULL NULL 2014 +PREHOOK: query: select key from (select key, c_int from (select * from cbo_t1 union all select * from cbo_t2 where cbo_t2.key >=0)r1 union all select key, c_int from cbo_t3)r2 where key >=0 order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select key from (select key, c_int from (select * from cbo_t1 union all select * from cbo_t2 where cbo_t2.key >=0)r1 union all select key, c_int from cbo_t3)r2 where key >=0 order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### 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 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +2 +2 +2 +2 +2 +2 +2 +2 +3 +3 +3 +PREHOOK: query: select r2.key from (select key, c_int from (select key, c_int from cbo_t1 union all select key, c_int from cbo_t3 )r1 union all select key, c_int from cbo_t3)r2 join (select key, c_int from (select * from cbo_t1 union all select * from cbo_t2 where cbo_t2.key >=0)r1 union all select key, c_int from cbo_t3)r3 on r2.key=r3.key where r3.key >=0 order by r2.key +PREHOOK: type: QUERY +PREHOOK: Input: default@cbo_t1 +PREHOOK: Input: default@cbo_t1@dt=2014 +PREHOOK: Input: default@cbo_t2 +PREHOOK: Input: default@cbo_t2@dt=2014 +PREHOOK: Input: default@cbo_t3 +#### A masked pattern was here #### +POSTHOOK: query: select r2.key from (select key, c_int from (select key, c_int from cbo_t1 union all select key, c_int from cbo_t3 )r1 union all select key, c_int from cbo_t3)r2 join (select key, c_int from (select * from cbo_t1 union all select * from cbo_t2 where cbo_t2.key >=0)r1 union all select key, c_int from cbo_t3)r3 on r2.key=r3.key where r3.key >=0 order by r2.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cbo_t1 +POSTHOOK: Input: default@cbo_t1@dt=2014 +POSTHOOK: Input: default@cbo_t2 +POSTHOOK: Input: default@cbo_t2@dt=2014 +POSTHOOK: Input: default@cbo_t3 +#### 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 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 Index: ql/src/test/results/clientpositive/spark/ctas.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ctas.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/ctas.q.out (working copy) @@ -32,8 +32,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -176,8 +176,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -320,8 +320,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -529,8 +529,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -716,8 +716,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/escape_clusterby1.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -65,7 +65,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/escape_sortby1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/escape_sortby1.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/escape_sortby1.q.out (working copy) @@ -14,7 +14,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -64,7 +64,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby10.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby10.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/groupby10.q.out (working copy) @@ -451,7 +451,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out (working copy) @@ -39,7 +39,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out (working copy) @@ -39,8 +39,8 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 31) - Reducer 6 <- Map 1 (SORT, 31) + Reducer 5 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) + Reducer 6 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) Reducer 3 <- Reducer 5 (SORT, 1) Reducer 4 <- Reducer 6 (SORT, 1) #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/spark/groupby8_map.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby8_map.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/groupby8_map.q.out (working copy) @@ -39,7 +39,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out (working copy) @@ -39,7 +39,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 31) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 31) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out (working copy) @@ -55,7 +55,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -227,7 +227,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -399,7 +399,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -571,7 +571,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/identity_project_remove_skip.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/identity_project_remove_skip.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/identity_project_remove_skip.q.out (working copy) @@ -25,7 +25,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -56,7 +56,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/input14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/input14.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/input14.q.out (working copy) @@ -33,7 +33,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/input17.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/input17.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/input17.q.out (working copy) @@ -33,7 +33,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/input18.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/input18.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/input18.q.out (working copy) @@ -33,7 +33,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join0.q.java1.7.out =================================================================== --- ql/src/test/results/clientpositive/spark/join0.q.java1.7.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/join0.q.java1.7.out (working copy) @@ -30,7 +30,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join0.q.java1.8.out =================================================================== --- ql/src/test/results/clientpositive/spark/join0.q.java1.8.out (revision 0) +++ ql/src/test/results/clientpositive/spark/join0.q.java1.8.out (working copy) @@ -0,0 +1,238 @@ +Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: -- JAVA_VERSION_SPECIFIC_OUTPUT +-- SORT_QUERY_RESULTS + +EXPLAIN +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +PREHOOK: type: QUERY +POSTHOOK: query: -- JAVA_VERSION_SPECIFIC_OUTPUT +-- SORT_QUERY_RESULTS + +EXPLAIN +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +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, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (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 + 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: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: src + 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: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + sort order: ++++ + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 182 Data size: 1939 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 + +Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: EXPLAIN FORMATTED +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN FORMATTED +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +POSTHOOK: type: QUERY +#### A masked pattern was here #### +Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Work 'Reducer 2' is a cross product +PREHOOK: query: SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### 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 +0 val_0 2 val_2 +0 val_0 2 val_2 +0 val_0 2 val_2 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 9 val_9 +0 val_0 9 val_9 +0 val_0 9 val_9 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 2 val_2 +2 val_2 4 val_4 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 8 val_8 +2 val_2 9 val_9 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 2 val_2 +4 val_4 4 val_4 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 8 val_8 +4 val_4 9 val_9 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 4 val_4 +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 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 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 9 val_9 +5 val_5 9 val_9 +5 val_5 9 val_9 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 2 val_2 +8 val_8 4 val_4 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 8 val_8 +8 val_8 9 val_9 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 2 val_2 +9 val_9 4 val_4 +9 val_9 5 val_5 +9 val_9 5 val_5 +9 val_9 5 val_5 +9 val_9 8 val_8 +9 val_9 9 val_9 Index: ql/src/test/results/clientpositive/spark/join15.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join15.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/join15.q.out (working copy) @@ -17,7 +17,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join20.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/join20.q.out (working copy) @@ -19,7 +19,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -680,7 +680,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join21.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join21.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/join21.q.out (working copy) @@ -17,7 +17,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join23.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join23.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/join23.q.out (working copy) @@ -18,7 +18,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/join40.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/join40.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/join40.q.out (working copy) @@ -1779,7 +1779,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2440,7 +2440,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out =================================================================== --- ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out (revision 0) +++ ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out (working copy) @@ -0,0 +1,665 @@ +PREHOOK: query: -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- SORT_QUERY_RESULTS +-- 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, value) on (('484','val_484'),('51','val_14'),('103','val_103')) + stored as DIRECTORIES + STORED AS RCFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@list_bucketing_static_part +POSTHOOK: query: -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- SORT_QUERY_RESULTS +-- 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, value) on (('484','val_484'),('51','val_14'),('103','val_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 srcpart where ds = '2008-04-08' +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 srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + srcpart + 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 + TOK_WHERE + = + TOK_TABLE_OR_COL + ds + '2008-04-08' + + +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: srcpart + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 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: 1000 Data size: 10624 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: hr=11 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 11 + 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.srcpart + numFiles 1 + numRows 500 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 5312 + serialization.ddl struct srcpart { 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: + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpart { 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.srcpart + name: default.srcpart +#### A masked pattern was here #### + Partition + base file name: hr=12 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + hr 12 + 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.srcpart + numFiles 1 + numRows 500 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 5312 + serialization.ddl struct srcpart { 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: + bucket_count -1 + columns key,value + columns.comments 'default','default' + columns.types string:string +#### A masked pattern was here #### + name default.srcpart + partition_columns ds/hr + partition_columns.types string:string + serialization.ddl struct srcpart { 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.srcpart + name: default.srcpart + Truncated Path -> Alias: + /srcpart/ds=2008-04-08/hr=11 [srcpart] + /srcpart/ds=2008-04-08/hr=12 [srcpart] + + 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 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@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 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@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 [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: list_bucketing_static_part PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.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 6 + numRows 1000 + rawDataSize 9624 + totalSize 10898 +#### 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, value] +Skewed Values: [[484, val_484], [51, val_14], [103, val_103]] +#### A masked pattern was here #### +Skewed Value to Truncated Path: {[484, val_484]=/list_bucketing_static_part/ds=2008-04-08/hr=11/key=484/value=val_484, [103, val_103]=/list_bucketing_static_part/ds=2008-04-08/hr=11/key=103/value=val_103} +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select count(1) 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 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) 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 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from list_bucketing_static_part +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from list_bucketing_static_part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: explain extended +select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + list_bucketing_static_part + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + TOK_WHERE + and + and + and + = + TOK_TABLE_OR_COL + ds + '2008-04-08' + = + TOK_TABLE_OR_COL + hr + '11' + = + TOK_TABLE_OR_COL + key + '484' + = + TOK_TABLE_OR_COL + value + 'val_484' + + +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + partition values: + ds 2008-04-08 + hr 11 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.list_bucketing_static_part + numFiles 6 + numRows 1000 + partition_columns ds/hr + partition_columns.types string:string + rawDataSize 9624 + serialization.ddl struct list_bucketing_static_part { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + totalSize 10898 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + properties: + bucket_count -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 + name: default.list_bucketing_static_part + Processor Tree: + TableScan + alias: list_bucketing_static_part + Statistics: Num rows: 1000 Data size: 9624 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: ((key = '484') and (value = 'val_484')) (type: boolean) + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '484' (type: string), 'val_484' (type: string), '2008-04-08' (type: string), '11' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE + ListSink + +PREHOOK: query: select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +484 val_484 2008-04-08 11 +484 val_484 2008-04-08 11 +PREHOOK: query: select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' +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 +#### A masked pattern was here #### +POSTHOOK: query: select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' +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 +#### A masked pattern was here #### +484 val_484 2008-04-08 11 +484 val_484 2008-04-08 12 +PREHOOK: query: -- 51 and val_51 in the table so skewed data for 51 and val_14 should be none +-- but query should succeed for 51 or 51 and val_14 +select * from srcpart where ds = '2008-04-08' and key = '51' +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 +#### A masked pattern was here #### +POSTHOOK: query: -- 51 and val_51 in the table so skewed data for 51 and val_14 should be none +-- but query should succeed for 51 or 51 and val_14 +select * from srcpart where ds = '2008-04-08' and key = '51' +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 +#### A masked pattern was here #### +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +PREHOOK: query: select * from list_bucketing_static_part where key = '51' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select * from list_bucketing_static_part where key = '51' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +PREHOOK: query: select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14' +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 +#### A masked pattern was here #### +POSTHOOK: query: select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14' +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 +#### A masked pattern was here #### +PREHOOK: query: select * from list_bucketing_static_part where key = '51' and value = 'val_14' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select * from list_bucketing_static_part where key = '51' and value = 'val_14' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +PREHOOK: query: -- queries with < <= > >= should work for skewed test although we don't benefit from pruning +select count(1) from srcpart where ds = '2008-04-08' and key < '51' +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 +#### A masked pattern was here #### +POSTHOOK: query: -- queries with < <= > >= should work for skewed test although we don't benefit from pruning +select count(1) from srcpart where ds = '2008-04-08' and key < '51' +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 +#### A masked pattern was here #### +910 +PREHOOK: query: select count(1) from list_bucketing_static_part where key < '51' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from list_bucketing_static_part where key < '51' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +910 +PREHOOK: query: select count(1) from srcpart where ds = '2008-04-08' and key <= '51' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from srcpart where ds = '2008-04-08' and key <= '51' +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 +#### A masked pattern was here #### +914 +PREHOOK: query: select count(1) from list_bucketing_static_part where key <= '51' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from list_bucketing_static_part where key <= '51' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +914 +PREHOOK: query: select count(1) from srcpart where ds = '2008-04-08' and key > '51' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from srcpart where ds = '2008-04-08' and key > '51' +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 +#### A masked pattern was here #### +86 +PREHOOK: query: select count(1) from list_bucketing_static_part where key > '51' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from list_bucketing_static_part where key > '51' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +86 +PREHOOK: query: select count(1) from srcpart where ds = '2008-04-08' and key >= '51' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from srcpart where ds = '2008-04-08' and key >= '51' +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 +#### A masked pattern was here #### +90 +PREHOOK: query: select count(1) from list_bucketing_static_part where key >= '51' +PREHOOK: type: QUERY +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(1) from list_bucketing_static_part where key >= '51' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Input: default@list_bucketing_static_part@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +90 +PREHOOK: query: -- clean up +drop table list_bucketing_static_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@list_bucketing_static_part +PREHOOK: Output: default@list_bucketing_static_part +POSTHOOK: query: -- clean up +drop table list_bucketing_static_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@list_bucketing_static_part +POSTHOOK: Output: default@list_bucketing_static_part Index: ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/mapjoin_filter_on_outerjoin.q.out (working copy) @@ -108,7 +108,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -269,7 +269,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 Index: ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out (working copy) @@ -285,7 +285,7 @@ Stage: Stage-1 Spark Edges: - Reducer 4 <- Map 3 (SORT, 2) + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 3 @@ -1115,7 +1115,7 @@ Stage: Stage-1 Spark Edges: - Reducer 3 <- Map 2 (SORT, 2) + Reducer 3 <- Map 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 2 Index: ql/src/test/results/clientpositive/spark/multi_insert.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/multi_insert.q.out (working copy) @@ -587,7 +587,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -733,7 +733,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -879,7 +879,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1025,7 +1025,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out (working copy) @@ -43,7 +43,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -230,7 +230,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out (working copy) @@ -50,7 +50,7 @@ Spark Edges: Reducer 2 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -172,7 +172,7 @@ Spark Edges: Reducer 2 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1750,7 +1750,7 @@ Spark Edges: Reducer 2 <- Map 1 (SORT, 1) - Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out (working copy) @@ -540,7 +540,7 @@ Spark Edges: Reducer 2 <- Map 4 (GROUP, 2) - Reducer 3 <- Map 5 (SORT, 2) + Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 4 Index: ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out (working copy) @@ -604,7 +604,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -754,7 +754,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -904,7 +904,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1054,7 +1054,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2305,10 +2305,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2432,10 +2432,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2559,10 +2559,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2686,10 +2686,10 @@ Stage: Stage-2 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2821,10 +2821,10 @@ Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3059,10 +3059,10 @@ Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3297,10 +3297,10 @@ Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3535,10 +3535,10 @@ Stage: Stage-4 Spark Edges: - Reducer 5 <- Map 1 (SORT, 2) - Reducer 6 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 5 (SORT, 2) - Reducer 4 <- Reducer 6 (SORT, 2) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 5 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 6 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/multigroupby_singlemr.q.out (working copy) @@ -461,7 +461,7 @@ Stage: Stage-2 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/parallel.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/parallel.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/parallel.q.out (working copy) @@ -40,7 +40,7 @@ Spark Edges: Reducer 2 <- Map 1 (GROUP, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/parallel_join0.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/parallel_join0.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/parallel_join0.q.out (working copy) @@ -28,7 +28,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (SORT, 4) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 4) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ppd_join4.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_join4.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/ppd_join4.q.out (working copy) @@ -52,7 +52,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/ppd_transform.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ppd_transform.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/ppd_transform.q.out (working copy) @@ -28,7 +28,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -208,7 +208,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/ptf.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ptf.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/ptf.q.out (working copy) @@ -3318,9 +3318,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -3588,9 +3588,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: @@ -3850,8 +3850,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: @@ -4092,10 +4092,10 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) - Reducer 5 <- Reducer 4 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -4373,9 +4373,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -4628,8 +4628,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/ptf_streaming.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/ptf_streaming.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/ptf_streaming.q.out (working copy) @@ -1854,9 +1854,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2124,9 +2124,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: @@ -2384,8 +2384,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out (working copy) @@ -389,7 +389,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/reduce_deduplicate_exclude_join.q.out (working copy) @@ -30,7 +30,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/runtime_skewjoin_mapjoin_spark.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/runtime_skewjoin_mapjoin_spark.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/runtime_skewjoin_mapjoin_spark.q.out (working copy) @@ -0,0 +1,314 @@ +PREHOOK: query: -- This is mainly intended for spark, to test runtime skew join together with map join + +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@T1 +POSTHOOK: query: -- This is mainly intended for spark, to test runtime skew join together with map join + +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE +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: EXPLAIN +SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-10, Stage-9 + Stage-10 + Stage-6 depends on stages: Stage-10 + Stage-9 depends on stages: Stage-6 + Stage-5 depends on stages: Stage-9 + Stage-4 depends on stages: Stage-5 , consists of Stage-8, Stage-2 + Stage-8 + Stage-3 depends on stages: Stage-8 + Stage-2 depends on stages: Stage-3 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (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 5 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + handleSkewJoin: true + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-10 + Spark +#### A masked pattern was here #### + Vertices: + Map 13 + Map Operator Tree: + TableScan + Spark HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-6 + Spark +#### A masked pattern was here #### + Vertices: + Map 12 + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-9 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-5 + Spark + Edges: + Reducer 3 <- Map 11 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 11 + Map Operator Tree: + TableScan + 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 + Map 6 + Map Operator Tree: + TableScan + alias: src + 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 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + input vertices: + 1 Map 7 + 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 + Local Work: + Map Reduce Local Work + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + handleSkewJoin: true + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-4 + Conditional Operator + + Stage: Stage-8 + Spark +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + Spark HashTable Sink Operator + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 9 + Map Operator Tree: + TableScan + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 reducesinkkey0 (type: string) + 1 reducesinkkey0 (type: string) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Spark + Edges: + Reducer 4 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 8 + Map Operator Tree: + TableScan + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 4 + 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 + 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(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT COUNT(*) FROM + (SELECT src1.key,src1.value FROM src src1 JOIN src src2 ON src1.key=src2.key) a +JOIN + (SELECT src.key,src.value FROM src JOIN T1 ON src.key=T1.key) b +ON a.key=b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +3 Index: ql/src/test/results/clientpositive/spark/semijoin.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/semijoin.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/semijoin.q.out (working copy) @@ -131,7 +131,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -233,7 +233,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -337,7 +337,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -433,7 +433,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -540,7 +540,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -639,7 +639,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -742,7 +742,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -842,7 +842,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -971,7 +971,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1060,7 +1060,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1160,7 +1160,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1293,7 +1293,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1455,7 +1455,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1544,7 +1544,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1673,7 +1673,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1805,7 +1805,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -1937,7 +1937,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2069,7 +2069,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2203,7 +2203,7 @@ Spark Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -2349,7 +2349,7 @@ Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/sort.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/sort.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/sort.q.out (working copy) @@ -16,7 +16,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.8.out =================================================================== --- ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.8.out (revision 0) +++ ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.java1.8.out (working copy) @@ -0,0 +1,890 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS +-- JAVA_VERSION_SPECIFIC_OUTPUT + +CREATE TABLE src_4( + key STRING, + value STRING +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_4 +POSTHOOK: query: -- SORT_QUERY_RESULTS +-- JAVA_VERSION_SPECIFIC_OUTPUT + +CREATE TABLE src_4( + key STRING, + value STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_4 +RUN: Stage-0:DDL +PREHOOK: query: CREATE TABLE src_5( + key STRING, + value STRING +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_5 +POSTHOOK: query: CREATE TABLE src_5( + key STRING, + value STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_5 +RUN: Stage-0:DDL +Warning: Shuffle Join JOIN[31][tables = [sq_2_notin_nullcheck]] in Work 'Reducer 2' is a cross product +PREHOOK: query: explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 2 <- Map 10 (PARTITION-LEVEL SORT, 1), Reducer 9 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Map 11 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 9 <- Map 8 (GROUP, 1) + Reducer 4 <- Reducer 3 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + value expressions: key (type: string), value (type: string) + Map 11 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string), value (type: string) + sort order: ++ + Map-reduce partition columns: key (type: string), value (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key > '9') and value is not null) (type: boolean) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Map 7 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '2') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 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: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Map 8 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key > '2') and key is null) (type: boolean) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + 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: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col5 + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col5 is null (type: boolean) + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 302 Data size: 3208 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_5 + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 key (type: string), value (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 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_4 + Reducer 9 + 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 + Filter Operator + predicate: (_col0 = 0) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + + Stage: Stage-1 + 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.src_5 + + Stage: Stage-3 + Stats-Aggr Operator + + 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.src_4 + + Stage: Stage-4 + Stats-Aggr Operator + +Warning: Shuffle Join JOIN[31][tables = [sq_2_notin_nullcheck]] in Work 'Reducer 2' is a cross product +PREHOOK: query: from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@src_4 +PREHOOK: Output: default@src_5 +POSTHOOK: query: from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@src_4 +POSTHOOK: Output: default@src_5 +POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ] +RUN: Stage-2:MAPRED +RUN: Stage-1:MOVE +RUN: Stage-0:MOVE +RUN: Stage-3:STATS +RUN: Stage-4:STATS +PREHOOK: query: select * from src_4 +PREHOOK: type: QUERY +PREHOOK: Input: default@src_4 +#### A masked pattern was here #### +POSTHOOK: query: select * from src_4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_4 +#### A masked pattern was here #### +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +PREHOOK: query: select * from src_5 +PREHOOK: type: QUERY +PREHOOK: Input: default@src_5 +#### A masked pattern was here #### +POSTHOOK: query: select * from src_5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_5 +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +150 val_150 +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 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +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 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +Warning: Map Join MAPJOIN[46][bigTable=b] in task 'Stage-2:MAPRED' is a cross product +PREHOOK: query: explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +PREHOOK: type: QUERY +POSTHOOK: query: explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-5 is a root stage + Stage-2 depends on stages: Stage-5 + Stage-1 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-5 + Spark + Edges: + Reducer 6 <- Map 5 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key > '9') and value is not null) (type: boolean) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: string), value (type: string) + 1 _col0 (type: string), _col1 (type: string) + Local Work: + Map Reduce Local Work + Map 4 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key > '2') (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + Map 5 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key > '2') and key is null) (type: boolean) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + 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 6 + Local Work: + Map Reduce Local Work + 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 + Filter Operator + predicate: (_col0 = 0) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: 0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 + 1 + + Stage: Stage-2 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) +#### 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 + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1 + input vertices: + 1 Reducer 6 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col5 + input vertices: + 1 Map 4 + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col5 is null (type: boolean) + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 key (type: string), value (type: string) + 1 _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + input vertices: + 1 Map 3 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 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_4 + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 302 Data size: 3208 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_5 + + Stage: Stage-1 + 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.src_5 + + Stage: Stage-3 + Stats-Aggr Operator + + 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.src_4 + + Stage: Stage-4 + Stats-Aggr Operator + +Warning: Map Join MAPJOIN[46][bigTable=b] in task 'Stage-2:MAPRED' is a cross product +PREHOOK: query: from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@src_4 +PREHOOK: Output: default@src_5 +POSTHOOK: query: from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@src_4 +POSTHOOK: Output: default@src_5 +POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ] +RUN: Stage-5:MAPRED +RUN: Stage-2:MAPRED +RUN: Stage-1:MOVE +RUN: Stage-0:MOVE +RUN: Stage-3:STATS +RUN: Stage-4:STATS +PREHOOK: query: select * from src_4 +PREHOOK: type: QUERY +PREHOOK: Input: default@src_4 +#### A masked pattern was here #### +POSTHOOK: query: select * from src_4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_4 +#### A masked pattern was here #### +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +PREHOOK: query: select * from src_5 +PREHOOK: type: QUERY +PREHOOK: Input: default@src_5 +#### A masked pattern was here #### +POSTHOOK: query: select * from src_5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_5 +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +150 val_150 +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 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +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 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +2 val_2 Index: ql/src/test/results/clientpositive/spark/transform_ppr1.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/transform_ppr1.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/transform_ppr1.q.out (working copy) @@ -103,7 +103,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/transform_ppr2.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/transform_ppr2.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/transform_ppr2.q.out (working copy) @@ -105,7 +105,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union12.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union12.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union12.q.out (working copy) @@ -0,0 +1,200 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink + +create table tmptable(key string, value int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tmptable +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink + +create table tmptable(key string, value int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tmptable +PREHOOK: query: explain +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src1 s2 + UNION ALL + select 'tst3' as key, count(1) as value from srcbucket s3) unionsrc +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src1 s2 + UNION ALL + select 'tst3' as key, count(1) as value from srcbucket s3) unionsrc +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, 1) + Reducer 4 <- Map 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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) + Map 3 + Map Operator Tree: + TableScan + alias: s2 + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE 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) + Map 5 + Map Operator Tree: + TableScan + alias: s3 + Statistics: Num rows: 1000 Data size: 10603 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1000 Data size: 10603 Basic stats: COMPLETE 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 + Select Operator + expressions: 'tst1' (type: string), _col0 (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + 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 + name: default.tmptable + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: 'tst2' (type: string), _col0 (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + 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 + name: default.tmptable + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: 'tst3' (type: string), _col0 (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + 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 + name: default.tmptable + + 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.tmptable + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src1 s2 + UNION ALL + select 'tst3' as key, count(1) as value from srcbucket s3) unionsrc +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +PREHOOK: Input: default@srcbucket +PREHOOK: Output: default@tmptable +POSTHOOK: query: insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src1 s2 + UNION ALL + select 'tst3' as key, count(1) as value from srcbucket s3) unionsrc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +POSTHOOK: Input: default@srcbucket +POSTHOOK: Output: default@tmptable +POSTHOOK: Lineage: tmptable.key EXPRESSION [] +POSTHOOK: Lineage: tmptable.value EXPRESSION [(src)s1.null, (src1)s2.null, (srcbucket)s3.null, ] +PREHOOK: query: select * from tmptable x sort by x.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tmptable +#### A masked pattern was here #### +POSTHOOK: query: select * from tmptable x sort by x.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmptable +#### A masked pattern was here #### +tst1 500 +tst2 25 +tst3 1000 Index: ql/src/test/results/clientpositive/spark/union17.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union17.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union17.q.out (working copy) @@ -0,0 +1,855 @@ +PREHOOK: query: CREATE TABLE DEST1(key STRING, value STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@DEST1 +POSTHOOK: query: CREATE TABLE DEST1(key STRING, value STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@DEST1 +PREHOOK: query: CREATE TABLE DEST2(key STRING, val1 STRING, val2 STRING) STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@DEST2 +POSTHOOK: query: CREATE TABLE DEST2(key STRING, val1 STRING, val2 STRING) STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@DEST2 +PREHOOK: query: -- SORT_QUERY_RESULTS +-- union case:map-reduce sub-queries followed by multi-table insert + +explain +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value +PREHOOK: type: QUERY +POSTHOOK: query: -- SORT_QUERY_RESULTS +-- union case:map-reduce sub-queries followed by multi-table insert + +explain +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value +POSTHOOK: type: QUERY +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 8 <- Map 1 (GROUP, 1) + Reducer 9 <- Map 1 (GROUP, 1) + Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 2), Reducer 8 (GROUP PARTITION-LEVEL SORT, 2) + Reducer 4 <- Map 7 (GROUP PARTITION-LEVEL SORT, 2), Reducer 9 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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) + Map 6 + Map Operator Tree: + TableScan + alias: s2 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(DISTINCT substr(_col1, 5)) + keys: _col0 (type: string), substr(_col1, 5) (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Map 7 + Map Operator Tree: + TableScan + alias: s2 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(DISTINCT substr(_col1, 5)) + keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL + 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.dest1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col2:0._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL + 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.dest2 + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: 'tst1' (type: string), UDFToString(_col0) (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(DISTINCT substr(_col1, 5)) + keys: _col0 (type: string), substr(_col1, 5) (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: 'tst1' (type: string), UDFToString(_col0) (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(DISTINCT substr(_col1, 5)) + keys: _col0 (type: string), _col1 (type: string), substr(_col1, 5) (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + + 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.dest1 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-1 + 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.dest2 + + Stage: Stage-4 + Stats-Aggr Operator + +PREHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dest1 +PREHOOK: Output: default@dest2 +POSTHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dest1 +POSTHOOK: Output: default@dest2 +POSTHOOK: Lineage: dest1.key EXPRESSION [(src)s2.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.value EXPRESSION [(src)s1.null, (src)s2.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest2.key EXPRESSION [(src)s2.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest2.val1 EXPRESSION [(src)s1.null, (src)s2.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dest2.val2 EXPRESSION [(src)s1.null, (src)s2.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 #### +0 1 +10 1 +100 1 +103 1 +104 1 +105 1 +11 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +12 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +15 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +17 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +18 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +19 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +2 1 +20 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +24 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +26 1 +260 1 +262 1 +263 1 +265 1 +266 1 +27 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +28 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +30 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +33 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +34 1 +341 1 +342 1 +344 1 +345 1 +348 1 +35 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +37 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +4 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +41 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +42 1 +421 1 +424 1 +427 1 +429 1 +43 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +44 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +47 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 +5 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +8 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +9 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +tst1 1 +PREHOOK: query: SELECT DEST2.* FROM DEST2 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT DEST2.* FROM DEST2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest2 +#### A masked pattern was here #### +0 val_0 1 +10 val_10 1 +100 val_100 1 +103 val_103 1 +104 val_104 1 +105 val_105 1 +11 val_11 1 +111 val_111 1 +113 val_113 1 +114 val_114 1 +116 val_116 1 +118 val_118 1 +119 val_119 1 +12 val_12 1 +120 val_120 1 +125 val_125 1 +126 val_126 1 +128 val_128 1 +129 val_129 1 +131 val_131 1 +133 val_133 1 +134 val_134 1 +136 val_136 1 +137 val_137 1 +138 val_138 1 +143 val_143 1 +145 val_145 1 +146 val_146 1 +149 val_149 1 +15 val_15 1 +150 val_150 1 +152 val_152 1 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 1 +165 val_165 1 +166 val_166 1 +167 val_167 1 +168 val_168 1 +169 val_169 1 +17 val_17 1 +170 val_170 1 +172 val_172 1 +174 val_174 1 +175 val_175 1 +176 val_176 1 +177 val_177 1 +178 val_178 1 +179 val_179 1 +18 val_18 1 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 1 +189 val_189 1 +19 val_19 1 +190 val_190 1 +191 val_191 1 +192 val_192 1 +193 val_193 1 +194 val_194 1 +195 val_195 1 +196 val_196 1 +197 val_197 1 +199 val_199 1 +2 val_2 1 +20 val_20 1 +200 val_200 1 +201 val_201 1 +202 val_202 1 +203 val_203 1 +205 val_205 1 +207 val_207 1 +208 val_208 1 +209 val_209 1 +213 val_213 1 +214 val_214 1 +216 val_216 1 +217 val_217 1 +218 val_218 1 +219 val_219 1 +221 val_221 1 +222 val_222 1 +223 val_223 1 +224 val_224 1 +226 val_226 1 +228 val_228 1 +229 val_229 1 +230 val_230 1 +233 val_233 1 +235 val_235 1 +237 val_237 1 +238 val_238 1 +239 val_239 1 +24 val_24 1 +241 val_241 1 +242 val_242 1 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 1 +256 val_256 1 +257 val_257 1 +258 val_258 1 +26 val_26 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 1 +266 val_266 1 +27 val_27 1 +272 val_272 1 +273 val_273 1 +274 val_274 1 +275 val_275 1 +277 val_277 1 +278 val_278 1 +28 val_28 1 +280 val_280 1 +281 val_281 1 +282 val_282 1 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 1 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 1 +30 val_30 1 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 1 +308 val_308 1 +309 val_309 1 +310 val_310 1 +311 val_311 1 +315 val_315 1 +316 val_316 1 +317 val_317 1 +318 val_318 1 +321 val_321 1 +322 val_322 1 +323 val_323 1 +325 val_325 1 +327 val_327 1 +33 val_33 1 +331 val_331 1 +332 val_332 1 +333 val_333 1 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +34 val_34 1 +341 val_341 1 +342 val_342 1 +344 val_344 1 +345 val_345 1 +348 val_348 1 +35 val_35 1 +351 val_351 1 +353 val_353 1 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 1 +368 val_368 1 +369 val_369 1 +37 val_37 1 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 1 +384 val_384 1 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 1 +396 val_396 1 +397 val_397 1 +399 val_399 1 +4 val_4 1 +400 val_400 1 +401 val_401 1 +402 val_402 1 +403 val_403 1 +404 val_404 1 +406 val_406 1 +407 val_407 1 +409 val_409 1 +41 val_41 1 +411 val_411 1 +413 val_413 1 +414 val_414 1 +417 val_417 1 +418 val_418 1 +419 val_419 1 +42 val_42 1 +421 val_421 1 +424 val_424 1 +427 val_427 1 +429 val_429 1 +43 val_43 1 +430 val_430 1 +431 val_431 1 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 1 +439 val_439 1 +44 val_44 1 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 1 +455 val_455 1 +457 val_457 1 +458 val_458 1 +459 val_459 1 +460 val_460 1 +462 val_462 1 +463 val_463 1 +466 val_466 1 +467 val_467 1 +468 val_468 1 +469 val_469 1 +47 val_47 1 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 1 +479 val_479 1 +480 val_480 1 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 1 +490 val_490 1 +491 val_491 1 +492 val_492 1 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 1 +5 val_5 1 +51 val_51 1 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 1 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 1 +69 val_69 1 +70 val_70 1 +72 val_72 1 +74 val_74 1 +76 val_76 1 +77 val_77 1 +78 val_78 1 +8 val_8 1 +80 val_80 1 +82 val_82 1 +83 val_83 1 +84 val_84 1 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 1 +92 val_92 1 +95 val_95 1 +96 val_96 1 +97 val_97 1 +98 val_98 1 +tst1 500 1 Index: ql/src/test/results/clientpositive/spark/union20.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union20.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union20.q.out (working copy) @@ -0,0 +1,200 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS +-- union :map-reduce sub-queries followed by join + +explain +SELECT unionsrc1.key, unionsrc1.value, unionsrc2.key, unionsrc2.value +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- SORT_QUERY_RESULTS +-- union :map-reduce sub-queries followed by join + +explain +SELECT unionsrc1.key, unionsrc1.value, unionsrc2.key, unionsrc2.value +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.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, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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) + Map 4 + Map Operator Tree: + TableScan + alias: s2 + Filter Operator + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: s3 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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) + Map 7 + Map Operator Tree: + TableScan + alias: s4 + Filter Operator + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: 'tst1' (type: string), UDFToString(_col0) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 13944 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL + File Output Operator + compressed: false + Statistics: Num rows: 13944 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL + 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 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: 'tst1' (type: string), UDFToString(_col0) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: string) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT unionsrc1.key, unionsrc1.value, unionsrc2.key, unionsrc2.value +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT unionsrc1.key, unionsrc1.value, unionsrc2.key, unionsrc2.value +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### 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 +2 val_2 2 val_2 +4 val_4 4 val_4 +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 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 +8 val_8 8 val_8 +9 val_9 9 val_9 +tst1 500 tst1 500 Index: ql/src/test/results/clientpositive/spark/union21.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union21.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union21.q.out (working copy) @@ -0,0 +1,724 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS +-- union of constants, udf outputs, and columns from text table and thrift table + +explain +SELECT key, count(1) +FROM ( + SELECT '1' as key from src + UNION ALL + SELECT reverse(key) as key from src + UNION ALL + SELECT key as key from src + UNION ALL + SELECT astring as key from src_thrift + UNION ALL + SELECT lstring[0] as key from src_thrift +) union_output +GROUP BY key +PREHOOK: type: QUERY +POSTHOOK: query: -- SORT_QUERY_RESULTS +-- union of constants, udf outputs, and columns from text table and thrift table + +explain +SELECT key, count(1) +FROM ( + SELECT '1' as key from src + UNION ALL + SELECT reverse(key) as key from src + UNION ALL + SELECT key as key from src + UNION ALL + SELECT astring as key from src_thrift + UNION ALL + SELECT lstring[0] as key from src_thrift +) union_output +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), Map 3 (GROUP, 2), Map 4 (GROUP, 2), Map 5 (GROUP, 2), Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Select Operator + expressions: '1' (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: src + Select Operator + expressions: reverse(key) (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: src + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 5 + Map Operator Tree: + TableScan + alias: src_thrift + Select Operator + expressions: astring (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: src_thrift + Select Operator + expressions: lstring[0] (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL + 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(1) +FROM ( + SELECT '1' as key from src + UNION ALL + SELECT reverse(key) as key from src + UNION ALL + SELECT key as key from src + UNION ALL + SELECT astring as key from src_thrift + UNION ALL + SELECT lstring[0] as key from src_thrift +) union_output +GROUP BY key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src_thrift +#### A masked pattern was here #### +POSTHOOK: query: SELECT key, count(1) +FROM ( + SELECT '1' as key from src + UNION ALL + SELECT reverse(key) as key from src + UNION ALL + SELECT key as key from src + UNION ALL + SELECT astring as key from src_thrift + UNION ALL + SELECT lstring[0] as key from src_thrift +) union_output +GROUP BY key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src_thrift +#### A masked pattern was here #### +0 7 +001 2 +002 2 +004 1 +01 1 +013 1 +02 1 +021 2 +03 1 +032 5 +034 3 +051 1 +061 1 +062 1 +063 1 +064 1 +07 3 +071 1 +074 1 +08 1 +081 1 +082 2 +084 3 +09 3 +091 1 +094 1 +1 500 +10 2 +100 2 +102 1 +103 2 +104 7 +105 1 +11 2 +111 2 +113 5 +114 2 +116 1 +118 2 +119 3 +12 2 +120 2 +122 2 +123 2 +124 1 +125 2 +126 1 +128 3 +129 2 +131 2 +133 3 +134 5 +136 1 +137 2 +138 4 +14 1 +142 1 +143 2 +145 1 +146 2 +149 2 +15 4 +150 1 +152 2 +153 2 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 2 +165 2 +166 1 +167 3 +168 1 +169 4 +17 1 +170 1 +172 2 +174 2 +175 2 +176 2 +177 1 +178 1 +179 2 +18 2 +180 1 +181 2 +182 2 +183 1 +184 1 +186 1 +187 3 +189 1 +19 1 +190 1 +191 4 +192 2 +193 3 +194 2 +195 2 +196 1 +197 2 +199 3 +2 2 +20 2 +200 2 +201 1 +202 2 +203 3 +204 1 +205 2 +207 2 +208 3 +209 2 +21 2 +213 2 +214 1 +216 2 +217 2 +218 1 +219 2 +221 2 +222 2 +223 4 +224 2 +226 1 +228 1 +229 2 +230 5 +233 3 +234 1 +235 1 +237 2 +238 2 +239 2 +24 4 +241 1 +242 4 +243 2 +244 1 +247 1 +248 1 +249 1 +251 2 +252 2 +254 1 +255 2 +256 2 +257 1 +258 1 +26 2 +260 1 +261 1 +262 2 +263 2 +264 2 +265 2 +266 1 +27 3 +271 2 +272 4 +273 3 +274 2 +275 1 +277 4 +278 2 +28 2 +280 2 +281 2 +282 4 +283 3 +284 2 +285 1 +286 1 +287 1 +288 2 +289 1 +29 1 +291 2 +292 2 +293 1 +294 2 +296 1 +298 3 +30 2 +301 2 +302 3 +304 3 +305 1 +306 1 +307 2 +308 1 +309 2 +310 1 +311 5 +312 2 +314 2 +315 1 +316 3 +317 2 +318 3 +321 2 +322 4 +323 2 +325 2 +327 3 +33 2 +331 3 +332 3 +333 4 +335 1 +336 1 +338 1 +339 1 +34 2 +341 2 +342 2 +344 3 +345 1 +348 5 +35 4 +351 2 +353 4 +354 1 +356 1 +360 1 +361 1 +362 2 +364 3 +365 1 +366 1 +367 2 +368 1 +369 3 +37 2 +372 3 +373 2 +374 1 +375 1 +377 1 +378 1 +379 1 +38 2 +381 1 +382 3 +384 4 +386 1 +389 1 +391 3 +392 1 +393 2 +394 2 +395 2 +396 3 +397 2 +399 2 +4 2 +40 1 +400 1 +401 7 +402 1 +403 3 +404 4 +406 4 +407 1 +409 3 +41 1 +411 2 +412 1 +413 2 +414 4 +417 3 +418 1 +419 1 +42 4 +421 1 +422 2 +424 4 +427 1 +429 2 +43 2 +430 3 +431 5 +432 1 +435 1 +436 1 +437 1 +438 3 +439 2 +44 2 +442 1 +443 3 +444 2 +446 1 +448 1 +449 1 +45 1 +452 1 +453 1 +454 6 +455 1 +457 1 +458 2 +459 2 +46 1 +460 1 +461 2 +462 2 +463 3 +466 3 +467 1 +468 4 +469 5 +47 2 +470 1 +471 2 +472 2 +473 1 +475 1 +477 1 +478 2 +479 1 +48 2 +480 3 +481 1 +482 2 +483 4 +484 2 +485 1 +487 1 +489 4 +490 1 +491 2 +492 2 +493 2 +494 2 +495 1 +496 1 +497 1 +498 3 +5 6 +50 1 +501 1 +502 2 +503 1 +51 4 +513 1 +521 2 +523 2 +53 4 +532 1 +533 1 +534 1 +54 1 +541 1 +543 1 +551 1 +552 2 +554 1 +56 1 +561 2 +562 2 +563 1 +57 1 +571 2 +572 1 +573 1 +574 1 +58 3 +582 1 +584 1 +59 2 +591 2 +593 2 +594 1 +60 1 +603 1 +604 4 +611 1 +612 2 +613 3 +62 2 +621 1 +622 1 +631 1 +633 1 +634 1 +64 1 +641 2 +644 1 +65 1 +651 1 +652 2 +653 1 +66 2 +661 1 +662 1 +663 1 +664 3 +67 4 +671 2 +68 1 +681 1 +682 1 +683 1 +69 2 +691 1 +692 1 +693 3 +694 1 +70 4 +702 2 +703 2 +704 1 +71 1 +712 2 +713 2 +714 3 +72 3 +723 3 +724 1 +73 2 +731 2 +732 2 +734 1 +74 2 +742 1 +75 1 +751 1 +752 1 +754 1 +76 4 +761 3 +763 2 +764 1 +77 2 +771 1 +772 4 +773 1 +774 1 +78 2 +781 3 +782 1 +784 1 +79 2 +791 2 +793 2 +794 1 +8 2 +80 2 +802 3 +803 1 +81 2 +811 2 +812 1 +813 3 +814 1 +82 2 +821 3 +822 1 +83 2 +831 4 +832 2 +833 1 +834 3 +84 2 +842 1 +843 5 +844 1 +85 3 +851 1 +852 1 +854 2 +86 1 +861 1 +863 1 +864 4 +87 2 +871 1 +872 2 +873 1 +874 2 +882 2 +89 2 +892 3 +894 3 +9 2 +90 4 +902 2 +903 2 +904 3 +91 1 +911 3 +912 2 +914 1 +92 1 +921 2 +922 2 +924 2 +932 2 +933 1 +934 2 +941 2 +942 1 +944 1 +95 2 +954 2 +96 2 +961 4 +963 3 +964 5 +97 2 +971 2 +973 1 +974 1 +98 2 +981 1 +982 1 +983 1 +984 4 +991 3 +993 2 +NULL 2 +record_0 1 +record_1 1 +record_2 1 +record_3 1 +record_4 1 +record_5 1 +record_6 1 +record_7 1 +record_8 1 +record_9 1 Index: ql/src/test/results/clientpositive/spark/union22.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union22.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union22.q.out (working copy) @@ -0,0 +1,1602 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +create table dst_union22(k1 string, k2 string, k3 string, k4 string) partitioned by (ds string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dst_union22 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +create table dst_union22(k1 string, k2 string, k3 string, k4 string) partitioned by (ds string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dst_union22 +PREHOOK: query: create table dst_union22_delta(k0 string, k1 string, k2 string, k3 string, k4 string, k5 string) partitioned by (ds string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dst_union22_delta +POSTHOOK: query: create table dst_union22_delta(k0 string, k1 string, k2 string, k3 string, k4 string, k5 string) partitioned by (ds string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dst_union22_delta +PREHOOK: query: insert overwrite table dst_union22 partition (ds='1') +select key, value, key , value from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dst_union22@ds=1 +POSTHOOK: query: insert overwrite table dst_union22 partition (ds='1') +select key, value, key , value from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dst_union22@ds=1 +POSTHOOK: Lineage: dst_union22 PARTITION(ds=1).k1 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22 PARTITION(ds=1).k2 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22 PARTITION(ds=1).k3 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22 PARTITION(ds=1).k4 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert overwrite table dst_union22_delta partition (ds='1') +select key, key, value, key, value, value from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@dst_union22_delta@ds=1 +POSTHOOK: query: insert overwrite table dst_union22_delta partition (ds='1') +select key, key, value, key, value, value from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@dst_union22_delta@ds=1 +POSTHOOK: Lineage: dst_union22_delta PARTITION(ds=1).k0 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22_delta PARTITION(ds=1).k1 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22_delta PARTITION(ds=1).k2 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22_delta PARTITION(ds=1).k3 SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22_delta PARTITION(ds=1).k4 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: dst_union22_delta PARTITION(ds=1).k5 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- Since the inputs are small, it should be automatically converted to mapjoin + +explain extended +insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq +PREHOOK: type: QUERY +POSTHOOK: query: -- Since the inputs are small, it should be automatically converted to mapjoin + +explain extended +insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_SUBQUERY + TOK_UNIONALL + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + dst_union22_delta + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + k1 + k1 + TOK_SELEXPR + TOK_TABLE_OR_COL + k2 + k2 + TOK_SELEXPR + TOK_TABLE_OR_COL + k3 + k3 + TOK_SELEXPR + TOK_TABLE_OR_COL + k4 + k4 + TOK_WHERE + and + = + TOK_TABLE_OR_COL + ds + '1' + <= + TOK_TABLE_OR_COL + k0 + 50 + TOK_QUERY + TOK_FROM + TOK_LEFTOUTERJOIN + TOK_TABREF + TOK_TABNAME + dst_union22 + a + TOK_SUBQUERY + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + dst_union22_delta + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + TOK_WHERE + and + = + TOK_TABLE_OR_COL + ds + '1' + > + TOK_TABLE_OR_COL + k0 + 50 + b + and + = + . + TOK_TABLE_OR_COL + a + k1 + . + TOK_TABLE_OR_COL + b + k1 + = + . + TOK_TABLE_OR_COL + a + ds + '1' + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + . + TOK_TABLE_OR_COL + a + k1 + k1 + TOK_SELEXPR + . + TOK_TABLE_OR_COL + a + k2 + k2 + TOK_SELEXPR + . + TOK_TABLE_OR_COL + b + k3 + k3 + TOK_SELEXPR + . + TOK_TABLE_OR_COL + b + k4 + k4 + TOK_WHERE + > + . + TOK_TABLE_OR_COL + a + k1 + 20 + subq + TOK_INSERT + TOK_DESTINATION + TOK_TAB + TOK_TABNAME + dst_union22 + TOK_PARTSPEC + TOK_PARTVAL + ds + '2' + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-1 depends on stages: Stage-3 + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: dst_union22_delta + Statistics: Num rows: 500 Data size: 16936 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: ((k0 > 50) and (k1 > 20)) (type: boolean) + Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: k1 (type: string), k3 (type: string), k4 (type: string) + outputColumnNames: _col1, _col3, _col4 + Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + filter mappings: + 0 [1, 1] + filter predicates: + 0 {(ds = '1')} + 1 + keys: + 0 k1 (type: string) + 1 _col1 (type: string) + Position of Big Table: 0 + Local Work: + Map Reduce Local Work + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=1 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns k0,k1,k2,k3,k4,k5 + columns.comments + columns.types string:string:string:string:string:string +#### A masked pattern was here #### + name default.dst_union22_delta + numFiles 1 + numRows 500 + partition_columns ds + partition_columns.types string + rawDataSize 16936 + serialization.ddl struct dst_union22_delta { string k0, string k1, string k2, string k3, string k4, string k5} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 17436 +#### 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 -1 + columns k0,k1,k2,k3,k4,k5 + columns.comments + columns.types string:string:string:string:string:string +#### A masked pattern was here #### + name default.dst_union22_delta + partition_columns ds + partition_columns.types string + serialization.ddl struct dst_union22_delta { string k0, string k1, string k2, string k3, string k4, string k5} + 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.dst_union22_delta + name: default.dst_union22_delta + Truncated Path -> Alias: + /dst_union22_delta/ds=1 [dst_union22_delta] + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: dst_union22_delta + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (k0 <= 50) (type: boolean) + Select Operator + expressions: k1 (type: string), k2 (type: string), k3 (type: string), k4 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Static Partition Specification: ds=2/ +#### 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 k1,k2,k3,k4 + columns.comments + columns.types string:string:string:string +#### A masked pattern was here #### + name default.dst_union22 + partition_columns ds + partition_columns.types string + serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} + 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.dst_union22 + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=1 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns k0,k1,k2,k3,k4,k5 + columns.comments + columns.types string:string:string:string:string:string +#### A masked pattern was here #### + name default.dst_union22_delta + numFiles 1 + numRows 500 + partition_columns ds + partition_columns.types string + rawDataSize 16936 + serialization.ddl struct dst_union22_delta { string k0, string k1, string k2, string k3, string k4, string k5} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 17436 +#### 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 -1 + columns k0,k1,k2,k3,k4,k5 + columns.comments + columns.types string:string:string:string:string:string +#### A masked pattern was here #### + name default.dst_union22_delta + partition_columns ds + partition_columns.types string + serialization.ddl struct dst_union22_delta { string k0, string k1, string k2, string k3, string k4, string k5} + 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.dst_union22_delta + name: default.dst_union22_delta + Truncated Path -> Alias: + /dst_union22_delta/ds=1 [dst_union22_delta] + Map 3 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (k1 > 20) (type: boolean) + Map Join Operator + condition map: + Left Outer Join0 to 1 + filter mappings: + 0 [1, 1] + filter predicates: + 0 {(ds = '1')} + 1 + keys: + 0 k1 (type: string) + 1 _col1 (type: string) + outputColumnNames: _col0, _col1, _col11, _col12 + input vertices: + 1 Map 2 + Position of Big Table: 0 + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col11 (type: string), _col12 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 1 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Static Partition Specification: ds=2/ +#### 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 k1,k2,k3,k4 + columns.comments + columns.types string:string:string:string +#### A masked pattern was here #### + name default.dst_union22 + partition_columns ds + partition_columns.types string + serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} + 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.dst_union22 + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false + Local Work: + Map Reduce Local Work + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=1 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns k1,k2,k3,k4 + columns.comments + columns.types string:string:string:string +#### A masked pattern was here #### + name default.dst_union22 + numFiles 1 + numRows 500 + partition_columns ds + partition_columns.types string + rawDataSize 11124 + serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 11624 +#### 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 -1 + columns k1,k2,k3,k4 + columns.comments + columns.types string:string:string:string +#### A masked pattern was here #### + name default.dst_union22 + partition_columns ds + partition_columns.types string + serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} + 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.dst_union22 + name: default.dst_union22 + Truncated Path -> Alias: + /dst_union22/ds=1 [a] + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2 + 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 k1,k2,k3,k4 + columns.comments + columns.types string:string:string:string +#### A masked pattern was here #### + name default.dst_union22 + partition_columns ds + partition_columns.types string + serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} + 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.dst_union22 + + Stage: Stage-2 + Stats-Aggr Operator +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq +PREHOOK: type: QUERY +PREHOOK: Input: default@dst_union22 +PREHOOK: Input: default@dst_union22@ds=1 +PREHOOK: Input: default@dst_union22_delta +PREHOOK: Input: default@dst_union22_delta@ds=1 +PREHOOK: Output: default@dst_union22@ds=2 +POSTHOOK: query: insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dst_union22 +POSTHOOK: Input: default@dst_union22@ds=1 +POSTHOOK: Input: default@dst_union22_delta +POSTHOOK: Input: default@dst_union22_delta@ds=1 +POSTHOOK: Output: default@dst_union22@ds=2 +POSTHOOK: Lineage: dst_union22 PARTITION(ds=2).k1 EXPRESSION [(dst_union22_delta)dst_union22_delta.FieldSchema(name:k1, type:string, comment:null), (dst_union22)a.FieldSchema(name:k1, type:string, comment:null), ] +POSTHOOK: Lineage: dst_union22 PARTITION(ds=2).k2 EXPRESSION [(dst_union22_delta)dst_union22_delta.FieldSchema(name:k2, type:string, comment:null), (dst_union22)a.FieldSchema(name:k2, type:string, comment:null), ] +POSTHOOK: Lineage: dst_union22 PARTITION(ds=2).k3 EXPRESSION [(dst_union22_delta)dst_union22_delta.FieldSchema(name:k3, type:string, comment:null), (dst_union22_delta)dst_union22_delta.FieldSchema(name:k3, type:string, comment:null), ] +POSTHOOK: Lineage: dst_union22 PARTITION(ds=2).k4 EXPRESSION [(dst_union22_delta)dst_union22_delta.FieldSchema(name:k4, type:string, comment:null), (dst_union22_delta)dst_union22_delta.FieldSchema(name:k4, type:string, comment:null), ] +PREHOOK: query: select * from dst_union22 where ds = '2' +PREHOOK: type: QUERY +PREHOOK: Input: default@dst_union22 +PREHOOK: Input: default@dst_union22@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from dst_union22 where ds = '2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dst_union22 +POSTHOOK: Input: default@dst_union22@ds=2 +#### A masked pattern was here #### +0 val_0 0 val_0 2 +0 val_0 0 val_0 2 +0 val_0 0 val_0 2 +10 val_10 10 val_10 2 +100 val_100 100 val_100 2 +100 val_100 100 val_100 2 +100 val_100 100 val_100 2 +100 val_100 100 val_100 2 +103 val_103 103 val_103 2 +103 val_103 103 val_103 2 +103 val_103 103 val_103 2 +103 val_103 103 val_103 2 +104 val_104 104 val_104 2 +104 val_104 104 val_104 2 +104 val_104 104 val_104 2 +104 val_104 104 val_104 2 +105 val_105 105 val_105 2 +11 val_11 11 val_11 2 +111 val_111 111 val_111 2 +113 val_113 113 val_113 2 +113 val_113 113 val_113 2 +113 val_113 113 val_113 2 +113 val_113 113 val_113 2 +114 val_114 114 val_114 2 +116 val_116 116 val_116 2 +118 val_118 118 val_118 2 +118 val_118 118 val_118 2 +118 val_118 118 val_118 2 +118 val_118 118 val_118 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +12 val_12 12 val_12 2 +12 val_12 12 val_12 2 +120 val_120 120 val_120 2 +120 val_120 120 val_120 2 +120 val_120 120 val_120 2 +120 val_120 120 val_120 2 +125 val_125 125 val_125 2 +125 val_125 125 val_125 2 +125 val_125 125 val_125 2 +125 val_125 125 val_125 2 +126 val_126 126 val_126 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +129 val_129 129 val_129 2 +129 val_129 129 val_129 2 +129 val_129 129 val_129 2 +129 val_129 129 val_129 2 +131 val_131 131 val_131 2 +133 val_133 133 val_133 2 +134 val_134 134 val_134 2 +134 val_134 134 val_134 2 +134 val_134 134 val_134 2 +134 val_134 134 val_134 2 +136 val_136 136 val_136 2 +137 val_137 137 val_137 2 +137 val_137 137 val_137 2 +137 val_137 137 val_137 2 +137 val_137 137 val_137 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +143 val_143 143 val_143 2 +145 val_145 145 val_145 2 +146 val_146 146 val_146 2 +146 val_146 146 val_146 2 +146 val_146 146 val_146 2 +146 val_146 146 val_146 2 +149 val_149 149 val_149 2 +149 val_149 149 val_149 2 +149 val_149 149 val_149 2 +149 val_149 149 val_149 2 +15 val_15 15 val_15 2 +15 val_15 15 val_15 2 +150 val_150 150 val_150 2 +152 val_152 152 val_152 2 +152 val_152 152 val_152 2 +152 val_152 152 val_152 2 +152 val_152 152 val_152 2 +153 val_153 153 val_153 2 +155 val_155 155 val_155 2 +156 val_156 156 val_156 2 +157 val_157 157 val_157 2 +158 val_158 158 val_158 2 +160 val_160 160 val_160 2 +162 val_162 162 val_162 2 +163 val_163 163 val_163 2 +164 val_164 164 val_164 2 +164 val_164 164 val_164 2 +164 val_164 164 val_164 2 +164 val_164 164 val_164 2 +165 val_165 165 val_165 2 +165 val_165 165 val_165 2 +165 val_165 165 val_165 2 +165 val_165 165 val_165 2 +166 val_166 166 val_166 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +168 val_168 168 val_168 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +17 val_17 17 val_17 2 +170 val_170 170 val_170 2 +172 val_172 172 val_172 2 +172 val_172 172 val_172 2 +172 val_172 172 val_172 2 +172 val_172 172 val_172 2 +174 val_174 174 val_174 2 +174 val_174 174 val_174 2 +174 val_174 174 val_174 2 +174 val_174 174 val_174 2 +175 val_175 175 val_175 2 +175 val_175 175 val_175 2 +175 val_175 175 val_175 2 +175 val_175 175 val_175 2 +176 val_176 176 val_176 2 +176 val_176 176 val_176 2 +176 val_176 176 val_176 2 +176 val_176 176 val_176 2 +177 val_177 177 val_177 2 +178 val_178 178 val_178 2 +179 val_179 179 val_179 2 +179 val_179 179 val_179 2 +179 val_179 179 val_179 2 +179 val_179 179 val_179 2 +18 val_18 18 val_18 2 +18 val_18 18 val_18 2 +180 val_180 180 val_180 2 +181 val_181 181 val_181 2 +183 val_183 183 val_183 2 +186 val_186 186 val_186 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +189 val_189 189 val_189 2 +19 val_19 19 val_19 2 +190 val_190 190 val_190 2 +191 val_191 191 val_191 2 +191 val_191 191 val_191 2 +191 val_191 191 val_191 2 +191 val_191 191 val_191 2 +192 val_192 192 val_192 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +194 val_194 194 val_194 2 +195 val_195 195 val_195 2 +195 val_195 195 val_195 2 +195 val_195 195 val_195 2 +195 val_195 195 val_195 2 +196 val_196 196 val_196 2 +197 val_197 197 val_197 2 +197 val_197 197 val_197 2 +197 val_197 197 val_197 2 +197 val_197 197 val_197 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +2 val_2 2 val_2 2 +20 val_20 20 val_20 2 +200 val_200 200 val_200 2 +200 val_200 200 val_200 2 +200 val_200 200 val_200 2 +200 val_200 200 val_200 2 +201 val_201 201 val_201 2 +202 val_202 202 val_202 2 +203 val_203 203 val_203 2 +203 val_203 203 val_203 2 +203 val_203 203 val_203 2 +203 val_203 203 val_203 2 +205 val_205 205 val_205 2 +205 val_205 205 val_205 2 +205 val_205 205 val_205 2 +205 val_205 205 val_205 2 +207 val_207 207 val_207 2 +207 val_207 207 val_207 2 +207 val_207 207 val_207 2 +207 val_207 207 val_207 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +209 val_209 209 val_209 2 +209 val_209 209 val_209 2 +209 val_209 209 val_209 2 +209 val_209 209 val_209 2 +213 val_213 213 val_213 2 +213 val_213 213 val_213 2 +213 val_213 213 val_213 2 +213 val_213 213 val_213 2 +214 val_214 214 val_214 2 +216 val_216 216 val_216 2 +216 val_216 216 val_216 2 +216 val_216 216 val_216 2 +216 val_216 216 val_216 2 +217 val_217 217 val_217 2 +217 val_217 217 val_217 2 +217 val_217 217 val_217 2 +217 val_217 217 val_217 2 +218 val_218 218 val_218 2 +219 val_219 219 val_219 2 +219 val_219 219 val_219 2 +219 val_219 219 val_219 2 +219 val_219 219 val_219 2 +221 val_221 221 val_221 2 +221 val_221 221 val_221 2 +221 val_221 221 val_221 2 +221 val_221 221 val_221 2 +222 val_222 222 val_222 2 +223 val_223 223 val_223 2 +223 val_223 223 val_223 2 +223 val_223 223 val_223 2 +223 val_223 223 val_223 2 +224 val_224 224 val_224 2 +224 val_224 224 val_224 2 +224 val_224 224 val_224 2 +224 val_224 224 val_224 2 +226 val_226 226 val_226 2 +228 val_228 228 val_228 2 +229 val_229 229 val_229 2 +229 val_229 229 val_229 2 +229 val_229 229 val_229 2 +229 val_229 229 val_229 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +233 val_233 233 val_233 2 +233 val_233 233 val_233 2 +233 val_233 233 val_233 2 +233 val_233 233 val_233 2 +235 val_235 235 val_235 2 +237 val_237 237 val_237 2 +237 val_237 237 val_237 2 +237 val_237 237 val_237 2 +237 val_237 237 val_237 2 +238 val_238 238 val_238 2 +238 val_238 238 val_238 2 +238 val_238 238 val_238 2 +238 val_238 238 val_238 2 +239 val_239 239 val_239 2 +239 val_239 239 val_239 2 +239 val_239 239 val_239 2 +239 val_239 239 val_239 2 +24 val_24 24 val_24 2 +24 val_24 24 val_24 2 +24 val_24 NULL NULL 2 +24 val_24 NULL NULL 2 +241 val_241 241 val_241 2 +242 val_242 242 val_242 2 +242 val_242 242 val_242 2 +242 val_242 242 val_242 2 +242 val_242 242 val_242 2 +244 val_244 244 val_244 2 +247 val_247 247 val_247 2 +248 val_248 248 val_248 2 +249 val_249 249 val_249 2 +252 val_252 252 val_252 2 +255 val_255 255 val_255 2 +255 val_255 255 val_255 2 +255 val_255 255 val_255 2 +255 val_255 255 val_255 2 +256 val_256 256 val_256 2 +256 val_256 256 val_256 2 +256 val_256 256 val_256 2 +256 val_256 256 val_256 2 +257 val_257 257 val_257 2 +258 val_258 258 val_258 2 +26 val_26 26 val_26 2 +26 val_26 26 val_26 2 +26 val_26 NULL NULL 2 +26 val_26 NULL NULL 2 +260 val_260 260 val_260 2 +262 val_262 262 val_262 2 +263 val_263 263 val_263 2 +265 val_265 265 val_265 2 +265 val_265 265 val_265 2 +265 val_265 265 val_265 2 +265 val_265 265 val_265 2 +266 val_266 266 val_266 2 +27 val_27 27 val_27 2 +27 val_27 NULL NULL 2 +272 val_272 272 val_272 2 +272 val_272 272 val_272 2 +272 val_272 272 val_272 2 +272 val_272 272 val_272 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +274 val_274 274 val_274 2 +275 val_275 275 val_275 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +278 val_278 278 val_278 2 +278 val_278 278 val_278 2 +278 val_278 278 val_278 2 +278 val_278 278 val_278 2 +28 val_28 28 val_28 2 +28 val_28 NULL NULL 2 +280 val_280 280 val_280 2 +280 val_280 280 val_280 2 +280 val_280 280 val_280 2 +280 val_280 280 val_280 2 +281 val_281 281 val_281 2 +281 val_281 281 val_281 2 +281 val_281 281 val_281 2 +281 val_281 281 val_281 2 +282 val_282 282 val_282 2 +282 val_282 282 val_282 2 +282 val_282 282 val_282 2 +282 val_282 282 val_282 2 +283 val_283 283 val_283 2 +284 val_284 284 val_284 2 +285 val_285 285 val_285 2 +286 val_286 286 val_286 2 +287 val_287 287 val_287 2 +288 val_288 288 val_288 2 +288 val_288 288 val_288 2 +288 val_288 288 val_288 2 +288 val_288 288 val_288 2 +289 val_289 289 val_289 2 +291 val_291 291 val_291 2 +292 val_292 292 val_292 2 +296 val_296 296 val_296 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +30 val_30 30 val_30 2 +30 val_30 NULL NULL 2 +302 val_302 302 val_302 2 +305 val_305 305 val_305 2 +306 val_306 306 val_306 2 +307 val_307 307 val_307 2 +307 val_307 307 val_307 2 +307 val_307 307 val_307 2 +307 val_307 307 val_307 2 +308 val_308 308 val_308 2 +309 val_309 309 val_309 2 +309 val_309 309 val_309 2 +309 val_309 309 val_309 2 +309 val_309 309 val_309 2 +310 val_310 310 val_310 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +315 val_315 315 val_315 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +317 val_317 317 val_317 2 +317 val_317 317 val_317 2 +317 val_317 317 val_317 2 +317 val_317 317 val_317 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +321 val_321 321 val_321 2 +321 val_321 321 val_321 2 +321 val_321 321 val_321 2 +321 val_321 321 val_321 2 +322 val_322 322 val_322 2 +322 val_322 322 val_322 2 +322 val_322 322 val_322 2 +322 val_322 322 val_322 2 +323 val_323 323 val_323 2 +325 val_325 325 val_325 2 +325 val_325 325 val_325 2 +325 val_325 325 val_325 2 +325 val_325 325 val_325 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +33 val_33 33 val_33 2 +33 val_33 NULL NULL 2 +331 val_331 331 val_331 2 +331 val_331 331 val_331 2 +331 val_331 331 val_331 2 +331 val_331 331 val_331 2 +332 val_332 332 val_332 2 +333 val_333 333 val_333 2 +333 val_333 333 val_333 2 +333 val_333 333 val_333 2 +333 val_333 333 val_333 2 +335 val_335 335 val_335 2 +336 val_336 336 val_336 2 +338 val_338 338 val_338 2 +339 val_339 339 val_339 2 +34 val_34 34 val_34 2 +34 val_34 NULL NULL 2 +341 val_341 341 val_341 2 +342 val_342 342 val_342 2 +342 val_342 342 val_342 2 +342 val_342 342 val_342 2 +342 val_342 342 val_342 2 +344 val_344 344 val_344 2 +344 val_344 344 val_344 2 +344 val_344 344 val_344 2 +344 val_344 344 val_344 2 +345 val_345 345 val_345 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +35 val_35 35 val_35 2 +35 val_35 35 val_35 2 +35 val_35 35 val_35 2 +35 val_35 NULL NULL 2 +35 val_35 NULL NULL 2 +35 val_35 NULL NULL 2 +351 val_351 351 val_351 2 +353 val_353 353 val_353 2 +353 val_353 353 val_353 2 +353 val_353 353 val_353 2 +353 val_353 353 val_353 2 +356 val_356 356 val_356 2 +360 val_360 360 val_360 2 +362 val_362 362 val_362 2 +364 val_364 364 val_364 2 +365 val_365 365 val_365 2 +366 val_366 366 val_366 2 +367 val_367 367 val_367 2 +367 val_367 367 val_367 2 +367 val_367 367 val_367 2 +367 val_367 367 val_367 2 +368 val_368 368 val_368 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +37 val_37 37 val_37 2 +37 val_37 37 val_37 2 +37 val_37 NULL NULL 2 +37 val_37 NULL NULL 2 +373 val_373 373 val_373 2 +374 val_374 374 val_374 2 +375 val_375 375 val_375 2 +377 val_377 377 val_377 2 +378 val_378 378 val_378 2 +379 val_379 379 val_379 2 +382 val_382 382 val_382 2 +382 val_382 382 val_382 2 +382 val_382 382 val_382 2 +382 val_382 382 val_382 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +386 val_386 386 val_386 2 +389 val_389 389 val_389 2 +392 val_392 392 val_392 2 +393 val_393 393 val_393 2 +394 val_394 394 val_394 2 +395 val_395 395 val_395 2 +395 val_395 395 val_395 2 +395 val_395 395 val_395 2 +395 val_395 395 val_395 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +397 val_397 397 val_397 2 +397 val_397 397 val_397 2 +397 val_397 397 val_397 2 +397 val_397 397 val_397 2 +399 val_399 399 val_399 2 +399 val_399 399 val_399 2 +399 val_399 399 val_399 2 +399 val_399 399 val_399 2 +4 val_4 4 val_4 2 +400 val_400 400 val_400 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +402 val_402 402 val_402 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +404 val_404 404 val_404 2 +404 val_404 404 val_404 2 +404 val_404 404 val_404 2 +404 val_404 404 val_404 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +407 val_407 407 val_407 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +41 val_41 41 val_41 2 +41 val_41 NULL NULL 2 +411 val_411 411 val_411 2 +413 val_413 413 val_413 2 +413 val_413 413 val_413 2 +413 val_413 413 val_413 2 +413 val_413 413 val_413 2 +414 val_414 414 val_414 2 +414 val_414 414 val_414 2 +414 val_414 414 val_414 2 +414 val_414 414 val_414 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +418 val_418 418 val_418 2 +419 val_419 419 val_419 2 +42 val_42 42 val_42 2 +42 val_42 42 val_42 2 +42 val_42 NULL NULL 2 +42 val_42 NULL NULL 2 +421 val_421 421 val_421 2 +424 val_424 424 val_424 2 +424 val_424 424 val_424 2 +424 val_424 424 val_424 2 +424 val_424 424 val_424 2 +427 val_427 427 val_427 2 +429 val_429 429 val_429 2 +429 val_429 429 val_429 2 +429 val_429 429 val_429 2 +429 val_429 429 val_429 2 +43 val_43 43 val_43 2 +43 val_43 NULL NULL 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +432 val_432 432 val_432 2 +435 val_435 435 val_435 2 +436 val_436 436 val_436 2 +437 val_437 437 val_437 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +439 val_439 439 val_439 2 +439 val_439 439 val_439 2 +439 val_439 439 val_439 2 +439 val_439 439 val_439 2 +44 val_44 44 val_44 2 +44 val_44 NULL NULL 2 +443 val_443 443 val_443 2 +444 val_444 444 val_444 2 +446 val_446 446 val_446 2 +448 val_448 448 val_448 2 +449 val_449 449 val_449 2 +452 val_452 452 val_452 2 +453 val_453 453 val_453 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +455 val_455 455 val_455 2 +457 val_457 457 val_457 2 +458 val_458 458 val_458 2 +458 val_458 458 val_458 2 +458 val_458 458 val_458 2 +458 val_458 458 val_458 2 +459 val_459 459 val_459 2 +459 val_459 459 val_459 2 +459 val_459 459 val_459 2 +459 val_459 459 val_459 2 +460 val_460 460 val_460 2 +462 val_462 462 val_462 2 +462 val_462 462 val_462 2 +462 val_462 462 val_462 2 +462 val_462 462 val_462 2 +463 val_463 463 val_463 2 +463 val_463 463 val_463 2 +463 val_463 463 val_463 2 +463 val_463 463 val_463 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +467 val_467 467 val_467 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +47 val_47 47 val_47 2 +47 val_47 NULL NULL 2 +470 val_470 470 val_470 2 +472 val_472 472 val_472 2 +475 val_475 475 val_475 2 +477 val_477 477 val_477 2 +478 val_478 478 val_478 2 +478 val_478 478 val_478 2 +478 val_478 478 val_478 2 +478 val_478 478 val_478 2 +479 val_479 479 val_479 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +481 val_481 481 val_481 2 +482 val_482 482 val_482 2 +483 val_483 483 val_483 2 +484 val_484 484 val_484 2 +485 val_485 485 val_485 2 +487 val_487 487 val_487 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +490 val_490 490 val_490 2 +491 val_491 491 val_491 2 +492 val_492 492 val_492 2 +492 val_492 492 val_492 2 +492 val_492 492 val_492 2 +492 val_492 492 val_492 2 +493 val_493 493 val_493 2 +494 val_494 494 val_494 2 +495 val_495 495 val_495 2 +496 val_496 496 val_496 2 +497 val_497 497 val_497 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +5 val_5 5 val_5 2 +5 val_5 5 val_5 2 +5 val_5 5 val_5 2 +51 val_51 51 val_51 2 +51 val_51 51 val_51 2 +51 val_51 51 val_51 2 +51 val_51 51 val_51 2 +53 val_53 53 val_53 2 +54 val_54 54 val_54 2 +57 val_57 57 val_57 2 +58 val_58 58 val_58 2 +58 val_58 58 val_58 2 +58 val_58 58 val_58 2 +58 val_58 58 val_58 2 +64 val_64 64 val_64 2 +65 val_65 65 val_65 2 +66 val_66 66 val_66 2 +67 val_67 67 val_67 2 +67 val_67 67 val_67 2 +67 val_67 67 val_67 2 +67 val_67 67 val_67 2 +69 val_69 69 val_69 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +72 val_72 72 val_72 2 +72 val_72 72 val_72 2 +72 val_72 72 val_72 2 +72 val_72 72 val_72 2 +74 val_74 74 val_74 2 +76 val_76 76 val_76 2 +76 val_76 76 val_76 2 +76 val_76 76 val_76 2 +76 val_76 76 val_76 2 +77 val_77 77 val_77 2 +78 val_78 78 val_78 2 +8 val_8 8 val_8 2 +80 val_80 80 val_80 2 +82 val_82 82 val_82 2 +83 val_83 83 val_83 2 +83 val_83 83 val_83 2 +83 val_83 83 val_83 2 +83 val_83 83 val_83 2 +84 val_84 84 val_84 2 +84 val_84 84 val_84 2 +84 val_84 84 val_84 2 +84 val_84 84 val_84 2 +85 val_85 85 val_85 2 +86 val_86 86 val_86 2 +87 val_87 87 val_87 2 +9 val_9 9 val_9 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +92 val_92 92 val_92 2 +95 val_95 95 val_95 2 +95 val_95 95 val_95 2 +95 val_95 95 val_95 2 +95 val_95 95 val_95 2 +96 val_96 96 val_96 2 +97 val_97 97 val_97 2 +97 val_97 97 val_97 2 +97 val_97 97 val_97 2 +97 val_97 97 val_97 2 +98 val_98 98 val_98 2 +98 val_98 98 val_98 2 +98 val_98 98 val_98 2 +98 val_98 98 val_98 2 Index: ql/src/test/results/clientpositive/spark/union24.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union24.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union24.q.out (working copy) @@ -0,0 +1,1649 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +create table src2 as select key, count(1) as count from src group by key +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@src2 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +create table src2 as select key, count(1) as count from src group by key +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src2 +PREHOOK: query: create table src3 as select * from src2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src2 +PREHOOK: Output: database:default +PREHOOK: Output: default@src3 +POSTHOOK: query: create table src3 as select * from src2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src2 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src3 +PREHOOK: query: create table src4 as select * from src2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src2 +PREHOOK: Output: database:default +PREHOOK: Output: default@src4 +POSTHOOK: query: create table src4 as select * from src2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src2 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src4 +PREHOOK: query: create table src5 as select * from src2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src2 +PREHOOK: Output: database:default +PREHOOK: Output: default@src5 +POSTHOOK: query: create table src5 as select * from src2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src2 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src5 +PREHOOK: query: explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select key, count from src4 where key < 10 + union all + select key, count(1) as count from src5 where key < 10 group by key +)s +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select key, count from src4 where key < 10 + union all + select key, count(1) as count from src5 where key < 10 group by key +)s +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_SUBQUERY + TOK_UNIONALL + TOK_UNIONALL + TOK_UNIONALL + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src2 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src3 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src4 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src5 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_FUNCTION + count + 1 + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_GROUPBY + TOK_TABLE_OR_COL + key + s + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + . + TOK_TABLE_OR_COL + s + key + TOK_SELEXPR + . + TOK_TABLE_OR_COL + s + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src2 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src2 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src2 + numFiles 2 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src2 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src2 + numFiles 2 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src2 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src2 + name: default.src2 + Truncated Path -> Alias: + /src2 [src2] + Map 2 + Map Operator Tree: + TableScan + alias: src3 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src3 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src3 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src3 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src3 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src3 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src3 + name: default.src3 + Truncated Path -> Alias: + /src3 [src3] + Map 3 + Map Operator Tree: + TableScan + alias: src4 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src4 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src4 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src4 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src4 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src4 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src4 + name: default.src4 + Truncated Path -> Alias: + /src4 [src4] + Map 4 + Map Operator Tree: + TableScan + alias: src5 + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 103 Data size: 494 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: 494 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src5 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src5 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src5 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src5 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src5 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src5 + name: default.src5 + Truncated Path -> Alias: + /src5 [src5] + Reducer 5 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select key, count from src4 where key < 10 + union all + select key, count(1) as count from src5 where key < 10 group by key +)s +PREHOOK: type: QUERY +PREHOOK: Input: default@src2 +PREHOOK: Input: default@src3 +PREHOOK: Input: default@src4 +PREHOOK: Input: default@src5 +#### A masked pattern was here #### +POSTHOOK: query: select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select key, count from src4 where key < 10 + union all + select key, count(1) as count from src5 where key < 10 group by key +)s +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src2 +POSTHOOK: Input: default@src3 +POSTHOOK: Input: default@src4 +POSTHOOK: Input: default@src5 +#### A masked pattern was here #### +0 1 +0 3 +0 3 +0 3 +2 1 +2 1 +2 1 +2 1 +4 1 +4 1 +4 1 +4 1 +5 1 +5 3 +5 3 +5 3 +8 1 +8 1 +8 1 +8 1 +9 1 +9 1 +9 1 +9 1 +PREHOOK: query: explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, b.count as count from src4 a join src5 b on a.key=b.key where a.key < 10 +)s +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, b.count as count from src4 a join src5 b on a.key=b.key where a.key < 10 +)s +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_SUBQUERY + TOK_UNIONALL + TOK_UNIONALL + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src2 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src3 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + src4 + a + TOK_TABREF + TOK_TABNAME + src5 + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + . + TOK_TABLE_OR_COL + a + key + key + TOK_SELEXPR + . + TOK_TABLE_OR_COL + b + count + count + TOK_WHERE + < + . + TOK_TABLE_OR_COL + a + key + 10 + s + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + . + TOK_TABLE_OR_COL + s + key + TOK_SELEXPR + . + TOK_TABLE_OR_COL + s + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src2 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src2 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src2 + numFiles 2 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src2 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src2 + numFiles 2 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src2 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src2 + name: default.src2 + Truncated Path -> Alias: + /src2 [src2] + Map 2 + Map Operator Tree: + TableScan + alias: src3 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src3 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src3 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src3 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src3 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src3 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src3 + name: default.src3 + Truncated Path -> Alias: + /src3 [src3] + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 103 Data size: 494 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: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE + tag: 0 + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src4 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src4 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src4 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src4 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src4 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src4 + name: default.src4 + Truncated Path -> Alias: + /src4 [a] + Map 5 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 103 Data size: 494 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: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE + tag: 1 + value expressions: count (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src5 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src5 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src5 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src5 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src5 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src5 + name: default.src5 + Truncated Path -> Alias: + /src5 [b] + Reducer 4 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0, _col6 + Select Operator + expressions: _col0 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, b.count as count from src4 a join src5 b on a.key=b.key where a.key < 10 +)s +PREHOOK: type: QUERY +PREHOOK: Input: default@src2 +PREHOOK: Input: default@src3 +PREHOOK: Input: default@src4 +PREHOOK: Input: default@src5 +#### A masked pattern was here #### +POSTHOOK: query: select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, b.count as count from src4 a join src5 b on a.key=b.key where a.key < 10 +)s +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src2 +POSTHOOK: Input: default@src3 +POSTHOOK: Input: default@src4 +POSTHOOK: Input: default@src5 +#### A masked pattern was here #### +0 3 +0 3 +0 3 +2 1 +2 1 +2 1 +4 1 +4 1 +4 1 +5 3 +5 3 +5 3 +8 1 +8 1 +8 1 +9 1 +9 1 +9 1 +PREHOOK: query: explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, count(1) as count from src4 a join src5 b on a.key=b.key where a.key < 10 group by a.key +)s +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, count(1) as count from src4 a join src5 b on a.key=b.key where a.key < 10 group by a.key +)s +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_SUBQUERY + TOK_UNIONALL + TOK_UNIONALL + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src2 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_TABREF + TOK_TABNAME + src3 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_TABLE_OR_COL + key + TOK_SELEXPR + TOK_TABLE_OR_COL + count + TOK_WHERE + < + TOK_TABLE_OR_COL + key + 10 + TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + src4 + a + TOK_TABREF + TOK_TABNAME + src5 + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + . + TOK_TABLE_OR_COL + a + key + key + TOK_SELEXPR + TOK_FUNCTION + count + 1 + count + TOK_WHERE + < + . + TOK_TABLE_OR_COL + a + key + 10 + TOK_GROUPBY + . + TOK_TABLE_OR_COL + a + key + s + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + . + TOK_TABLE_OR_COL + s + key + TOK_SELEXPR + . + TOK_TABLE_OR_COL + s + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src2 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src2 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src2 + numFiles 2 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src2 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src2 + numFiles 2 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src2 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src2 + name: default.src2 + Truncated Path -> Alias: + /src2 [src2] + Map 2 + Map Operator Tree: + TableScan + alias: src3 + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Select Operator + expressions: key (type: string), count (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src3 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src3 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src3 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src3 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src3 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src3 + name: default.src3 + Truncated Path -> Alias: + /src3 [src3] + Map 3 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 103 Data size: 494 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: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE + tag: 0 + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src4 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src4 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src4 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src4 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src4 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src4 + name: default.src4 + Truncated Path -> Alias: + /src4 [a] + Map 6 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 103 Data size: 494 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: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE + tag: 1 + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: src5 + 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src5 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src5 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### 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,count + columns.comments + columns.types string:bigint +#### A masked pattern was here #### + name default.src5 + numFiles 1 + numRows 309 + rawDataSize 1482 + serialization.ddl struct src5 { string key, i64 count} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 1791 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src5 + name: default.src5 + Truncated Path -> Alias: + /src5 [b] + Reducer 4 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 113 Data size: 543 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: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: bigint) + auto parallelism: false + Reducer 5 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + hive.serialization.extend.additional.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 s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, count(1) as count from src4 a join src5 b on a.key=b.key where a.key < 10 group by a.key +)s +PREHOOK: type: QUERY +PREHOOK: Input: default@src2 +PREHOOK: Input: default@src3 +PREHOOK: Input: default@src4 +PREHOOK: Input: default@src5 +#### A masked pattern was here #### +POSTHOOK: query: select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, count(1) as count from src4 a join src5 b on a.key=b.key where a.key < 10 group by a.key +)s +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src2 +POSTHOOK: Input: default@src3 +POSTHOOK: Input: default@src4 +POSTHOOK: Input: default@src5 +#### A masked pattern was here #### +0 1 +0 3 +0 3 +2 1 +2 1 +2 1 +4 1 +4 1 +4 1 +5 1 +5 3 +5 3 +8 1 +8 1 +8 1 +9 1 +9 1 +9 1 Index: ql/src/test/results/clientpositive/spark/union26.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union26.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union26.q.out (working copy) @@ -0,0 +1,1249 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +EXPLAIN +SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +PREHOOK: type: QUERY +POSTHOOK: query: -- SORT_QUERY_RESULTS + +EXPLAIN +SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (GROUP, 2), Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + 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) + Map 4 + 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 + 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 5 + Map Operator Tree: + TableScan + alias: srcpart + Lateral View Forward + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: key, value + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + value expressions: _col2 (type: bigint) + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string), _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + value expressions: _col2 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 637 Data size: 6767 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: bigint), _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 637 Data size: 6767 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 637 Data size: 6767 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(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +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 +#### A masked pattern was here #### +POSTHOOK: query: SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +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 +#### A masked pattern was here #### +10 100 val_100 +10 103 val_103 +10 104 val_104 +10 113 val_113 +10 118 val_118 +10 12 val_12 +10 120 val_120 +10 125 val_125 +10 129 val_129 +10 134 val_134 +10 137 val_137 +10 146 val_146 +10 149 val_149 +10 15 val_15 +10 152 val_152 +10 164 val_164 +10 165 val_165 +10 172 val_172 +10 174 val_174 +10 175 val_175 +10 176 val_176 +10 179 val_179 +10 18 val_18 +10 191 val_191 +10 195 val_195 +10 197 val_197 +10 200 val_200 +10 203 val_203 +10 205 val_205 +10 207 val_207 +10 209 val_209 +10 213 val_213 +10 216 val_216 +10 217 val_217 +10 219 val_219 +10 221 val_221 +10 223 val_223 +10 224 val_224 +10 229 val_229 +10 233 val_233 +10 237 val_237 +10 238 val_238 +10 239 val_239 +10 24 val_24 +10 242 val_242 +10 255 val_255 +10 256 val_256 +10 26 val_26 +10 265 val_265 +10 272 val_272 +10 278 val_278 +10 280 val_280 +10 281 val_281 +10 282 val_282 +10 288 val_288 +10 307 val_307 +10 309 val_309 +10 317 val_317 +10 321 val_321 +10 322 val_322 +10 325 val_325 +10 331 val_331 +10 333 val_333 +10 342 val_342 +10 344 val_344 +10 353 val_353 +10 367 val_367 +10 37 val_37 +10 382 val_382 +10 395 val_395 +10 397 val_397 +10 399 val_399 +10 404 val_404 +10 413 val_413 +10 414 val_414 +10 42 val_42 +10 424 val_424 +10 429 val_429 +10 439 val_439 +10 458 val_458 +10 459 val_459 +10 462 val_462 +10 463 val_463 +10 478 val_478 +10 492 val_492 +10 51 val_51 +10 58 val_58 +10 67 val_67 +10 72 val_72 +10 76 val_76 +10 83 val_83 +10 84 val_84 +10 95 val_95 +10 97 val_97 +10 98 val_98 +18 0 val_0 +18 119 val_119 +18 128 val_128 +18 167 val_167 +18 187 val_187 +18 193 val_193 +18 199 val_199 +18 208 val_208 +18 273 val_273 +18 298 val_298 +18 311 val_311 +18 316 val_316 +18 318 val_318 +18 327 val_327 +18 35 val_35 +18 369 val_369 +18 384 val_384 +18 396 val_396 +18 403 val_403 +18 409 val_409 +18 417 val_417 +18 430 val_430 +18 431 val_431 +18 438 val_438 +18 454 val_454 +18 466 val_466 +18 480 val_480 +18 498 val_498 +18 5 val_5 +18 70 val_70 +18 90 val_90 +28 138 val_138 +28 169 val_169 +28 277 val_277 +28 406 val_406 +28 468 val_468 +28 489 val_489 +4 10 val_10 +4 105 val_105 +4 11 val_11 +4 111 val_111 +4 114 val_114 +4 116 val_116 +4 126 val_126 +4 131 val_131 +4 133 val_133 +4 136 val_136 +4 143 val_143 +4 145 val_145 +4 150 val_150 +4 153 val_153 +4 155 val_155 +4 156 val_156 +4 157 val_157 +4 158 val_158 +4 160 val_160 +4 162 val_162 +4 163 val_163 +4 166 val_166 +4 168 val_168 +4 17 val_17 +4 170 val_170 +4 177 val_177 +4 178 val_178 +4 180 val_180 +4 181 val_181 +4 183 val_183 +4 186 val_186 +4 189 val_189 +4 19 val_19 +4 190 val_190 +4 192 val_192 +4 194 val_194 +4 196 val_196 +4 2 val_2 +4 20 val_20 +4 201 val_201 +4 202 val_202 +4 214 val_214 +4 218 val_218 +4 222 val_222 +4 226 val_226 +4 228 val_228 +4 235 val_235 +4 241 val_241 +4 244 val_244 +4 247 val_247 +4 248 val_248 +4 249 val_249 +4 252 val_252 +4 257 val_257 +4 258 val_258 +4 260 val_260 +4 262 val_262 +4 263 val_263 +4 266 val_266 +4 27 val_27 +4 274 val_274 +4 275 val_275 +4 28 val_28 +4 283 val_283 +4 284 val_284 +4 285 val_285 +4 286 val_286 +4 287 val_287 +4 289 val_289 +4 291 val_291 +4 292 val_292 +4 296 val_296 +4 30 val_30 +4 302 val_302 +4 305 val_305 +4 306 val_306 +4 308 val_308 +4 310 val_310 +4 315 val_315 +4 323 val_323 +4 33 val_33 +4 332 val_332 +4 335 val_335 +4 336 val_336 +4 338 val_338 +4 339 val_339 +4 34 val_34 +4 341 val_341 +4 345 val_345 +4 351 val_351 +4 356 val_356 +4 360 val_360 +4 362 val_362 +4 364 val_364 +4 365 val_365 +4 366 val_366 +4 368 val_368 +4 373 val_373 +4 374 val_374 +4 375 val_375 +4 377 val_377 +4 378 val_378 +4 379 val_379 +4 386 val_386 +4 389 val_389 +4 392 val_392 +4 393 val_393 +4 394 val_394 +4 4 val_4 +4 400 val_400 +4 402 val_402 +4 407 val_407 +4 41 val_41 +4 411 val_411 +4 418 val_418 +4 419 val_419 +4 421 val_421 +4 427 val_427 +4 43 val_43 +4 432 val_432 +4 435 val_435 +4 436 val_436 +4 437 val_437 +4 44 val_44 +4 443 val_443 +4 444 val_444 +4 446 val_446 +4 448 val_448 +4 449 val_449 +4 452 val_452 +4 453 val_453 +4 455 val_455 +4 457 val_457 +4 460 val_460 +4 467 val_467 +4 47 val_47 +4 470 val_470 +4 472 val_472 +4 475 val_475 +4 477 val_477 +4 479 val_479 +4 481 val_481 +4 482 val_482 +4 483 val_483 +4 484 val_484 +4 485 val_485 +4 487 val_487 +4 490 val_490 +4 491 val_491 +4 493 val_493 +4 494 val_494 +4 495 val_495 +4 496 val_496 +4 497 val_497 +4 53 val_53 +4 54 val_54 +4 57 val_57 +4 64 val_64 +4 65 val_65 +4 66 val_66 +4 69 val_69 +4 74 val_74 +4 77 val_77 +4 78 val_78 +4 8 val_8 +4 80 val_80 +4 82 val_82 +4 85 val_85 +4 86 val_86 +4 87 val_87 +4 9 val_9 +4 92 val_92 +4 96 val_96 +40 230 val_230 +40 348 val_348 +40 401 val_401 +40 469 val_469 +PREHOOK: query: SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +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 +#### A masked pattern was here #### +POSTHOOK: query: SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +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 +#### A masked pattern was here #### +10 100 val_100 +10 103 val_103 +10 104 val_104 +10 113 val_113 +10 118 val_118 +10 12 val_12 +10 120 val_120 +10 125 val_125 +10 129 val_129 +10 134 val_134 +10 137 val_137 +10 146 val_146 +10 149 val_149 +10 15 val_15 +10 152 val_152 +10 164 val_164 +10 165 val_165 +10 172 val_172 +10 174 val_174 +10 175 val_175 +10 176 val_176 +10 179 val_179 +10 18 val_18 +10 191 val_191 +10 195 val_195 +10 197 val_197 +10 200 val_200 +10 203 val_203 +10 205 val_205 +10 207 val_207 +10 209 val_209 +10 213 val_213 +10 216 val_216 +10 217 val_217 +10 219 val_219 +10 221 val_221 +10 223 val_223 +10 224 val_224 +10 229 val_229 +10 233 val_233 +10 237 val_237 +10 238 val_238 +10 239 val_239 +10 24 val_24 +10 242 val_242 +10 255 val_255 +10 256 val_256 +10 26 val_26 +10 265 val_265 +10 272 val_272 +10 278 val_278 +10 280 val_280 +10 281 val_281 +10 282 val_282 +10 288 val_288 +10 307 val_307 +10 309 val_309 +10 317 val_317 +10 321 val_321 +10 322 val_322 +10 325 val_325 +10 331 val_331 +10 333 val_333 +10 342 val_342 +10 344 val_344 +10 353 val_353 +10 367 val_367 +10 37 val_37 +10 382 val_382 +10 395 val_395 +10 397 val_397 +10 399 val_399 +10 404 val_404 +10 413 val_413 +10 414 val_414 +10 42 val_42 +10 424 val_424 +10 429 val_429 +10 439 val_439 +10 458 val_458 +10 459 val_459 +10 462 val_462 +10 463 val_463 +10 478 val_478 +10 492 val_492 +10 51 val_51 +10 58 val_58 +10 67 val_67 +10 72 val_72 +10 76 val_76 +10 83 val_83 +10 84 val_84 +10 95 val_95 +10 97 val_97 +10 98 val_98 +18 0 val_0 +18 119 val_119 +18 128 val_128 +18 167 val_167 +18 187 val_187 +18 193 val_193 +18 199 val_199 +18 208 val_208 +18 273 val_273 +18 298 val_298 +18 311 val_311 +18 316 val_316 +18 318 val_318 +18 327 val_327 +18 35 val_35 +18 369 val_369 +18 384 val_384 +18 396 val_396 +18 403 val_403 +18 409 val_409 +18 417 val_417 +18 430 val_430 +18 431 val_431 +18 438 val_438 +18 454 val_454 +18 466 val_466 +18 480 val_480 +18 498 val_498 +18 5 val_5 +18 70 val_70 +18 90 val_90 +28 138 val_138 +28 169 val_169 +28 277 val_277 +28 406 val_406 +28 468 val_468 +28 489 val_489 +4 10 val_10 +4 105 val_105 +4 11 val_11 +4 111 val_111 +4 114 val_114 +4 116 val_116 +4 126 val_126 +4 131 val_131 +4 133 val_133 +4 136 val_136 +4 143 val_143 +4 145 val_145 +4 150 val_150 +4 153 val_153 +4 155 val_155 +4 156 val_156 +4 157 val_157 +4 158 val_158 +4 160 val_160 +4 162 val_162 +4 163 val_163 +4 166 val_166 +4 168 val_168 +4 17 val_17 +4 170 val_170 +4 177 val_177 +4 178 val_178 +4 180 val_180 +4 181 val_181 +4 183 val_183 +4 186 val_186 +4 189 val_189 +4 19 val_19 +4 190 val_190 +4 192 val_192 +4 194 val_194 +4 196 val_196 +4 2 val_2 +4 20 val_20 +4 201 val_201 +4 202 val_202 +4 214 val_214 +4 218 val_218 +4 222 val_222 +4 226 val_226 +4 228 val_228 +4 235 val_235 +4 241 val_241 +4 244 val_244 +4 247 val_247 +4 248 val_248 +4 249 val_249 +4 252 val_252 +4 257 val_257 +4 258 val_258 +4 260 val_260 +4 262 val_262 +4 263 val_263 +4 266 val_266 +4 27 val_27 +4 274 val_274 +4 275 val_275 +4 28 val_28 +4 283 val_283 +4 284 val_284 +4 285 val_285 +4 286 val_286 +4 287 val_287 +4 289 val_289 +4 291 val_291 +4 292 val_292 +4 296 val_296 +4 30 val_30 +4 302 val_302 +4 305 val_305 +4 306 val_306 +4 308 val_308 +4 310 val_310 +4 315 val_315 +4 323 val_323 +4 33 val_33 +4 332 val_332 +4 335 val_335 +4 336 val_336 +4 338 val_338 +4 339 val_339 +4 34 val_34 +4 341 val_341 +4 345 val_345 +4 351 val_351 +4 356 val_356 +4 360 val_360 +4 362 val_362 +4 364 val_364 +4 365 val_365 +4 366 val_366 +4 368 val_368 +4 373 val_373 +4 374 val_374 +4 375 val_375 +4 377 val_377 +4 378 val_378 +4 379 val_379 +4 386 val_386 +4 389 val_389 +4 392 val_392 +4 393 val_393 +4 394 val_394 +4 4 val_4 +4 400 val_400 +4 402 val_402 +4 407 val_407 +4 41 val_41 +4 411 val_411 +4 418 val_418 +4 419 val_419 +4 421 val_421 +4 427 val_427 +4 43 val_43 +4 432 val_432 +4 435 val_435 +4 436 val_436 +4 437 val_437 +4 44 val_44 +4 443 val_443 +4 444 val_444 +4 446 val_446 +4 448 val_448 +4 449 val_449 +4 452 val_452 +4 453 val_453 +4 455 val_455 +4 457 val_457 +4 460 val_460 +4 467 val_467 +4 47 val_47 +4 470 val_470 +4 472 val_472 +4 475 val_475 +4 477 val_477 +4 479 val_479 +4 481 val_481 +4 482 val_482 +4 483 val_483 +4 484 val_484 +4 485 val_485 +4 487 val_487 +4 490 val_490 +4 491 val_491 +4 493 val_493 +4 494 val_494 +4 495 val_495 +4 496 val_496 +4 497 val_497 +4 53 val_53 +4 54 val_54 +4 57 val_57 +4 64 val_64 +4 65 val_65 +4 66 val_66 +4 69 val_69 +4 74 val_74 +4 77 val_77 +4 78 val_78 +4 8 val_8 +4 80 val_80 +4 82 val_82 +4 85 val_85 +4 86 val_86 +4 87 val_87 +4 9 val_9 +4 92 val_92 +4 96 val_96 +40 230 val_230 +40 348 val_348 +40 401 val_401 +40 469 val_469 +PREHOOK: query: SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +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 +#### A masked pattern was here #### +POSTHOOK: query: SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +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 +#### A masked pattern was here #### +10 100 val_100 +10 103 val_103 +10 104 val_104 +10 113 val_113 +10 118 val_118 +10 12 val_12 +10 120 val_120 +10 125 val_125 +10 129 val_129 +10 134 val_134 +10 137 val_137 +10 146 val_146 +10 149 val_149 +10 15 val_15 +10 152 val_152 +10 164 val_164 +10 165 val_165 +10 172 val_172 +10 174 val_174 +10 175 val_175 +10 176 val_176 +10 179 val_179 +10 18 val_18 +10 191 val_191 +10 195 val_195 +10 197 val_197 +10 200 val_200 +10 203 val_203 +10 205 val_205 +10 207 val_207 +10 209 val_209 +10 213 val_213 +10 216 val_216 +10 217 val_217 +10 219 val_219 +10 221 val_221 +10 223 val_223 +10 224 val_224 +10 229 val_229 +10 233 val_233 +10 237 val_237 +10 238 val_238 +10 239 val_239 +10 24 val_24 +10 242 val_242 +10 255 val_255 +10 256 val_256 +10 26 val_26 +10 265 val_265 +10 272 val_272 +10 278 val_278 +10 280 val_280 +10 281 val_281 +10 282 val_282 +10 288 val_288 +10 307 val_307 +10 309 val_309 +10 317 val_317 +10 321 val_321 +10 322 val_322 +10 325 val_325 +10 331 val_331 +10 333 val_333 +10 342 val_342 +10 344 val_344 +10 353 val_353 +10 367 val_367 +10 37 val_37 +10 382 val_382 +10 395 val_395 +10 397 val_397 +10 399 val_399 +10 404 val_404 +10 413 val_413 +10 414 val_414 +10 42 val_42 +10 424 val_424 +10 429 val_429 +10 439 val_439 +10 458 val_458 +10 459 val_459 +10 462 val_462 +10 463 val_463 +10 478 val_478 +10 492 val_492 +10 51 val_51 +10 58 val_58 +10 67 val_67 +10 72 val_72 +10 76 val_76 +10 83 val_83 +10 84 val_84 +10 95 val_95 +10 97 val_97 +10 98 val_98 +18 0 val_0 +18 119 val_119 +18 128 val_128 +18 167 val_167 +18 187 val_187 +18 193 val_193 +18 199 val_199 +18 208 val_208 +18 273 val_273 +18 298 val_298 +18 311 val_311 +18 316 val_316 +18 318 val_318 +18 327 val_327 +18 35 val_35 +18 369 val_369 +18 384 val_384 +18 396 val_396 +18 403 val_403 +18 409 val_409 +18 417 val_417 +18 430 val_430 +18 431 val_431 +18 438 val_438 +18 454 val_454 +18 466 val_466 +18 480 val_480 +18 498 val_498 +18 5 val_5 +18 70 val_70 +18 90 val_90 +28 138 val_138 +28 169 val_169 +28 277 val_277 +28 406 val_406 +28 468 val_468 +28 489 val_489 +4 10 val_10 +4 105 val_105 +4 11 val_11 +4 111 val_111 +4 114 val_114 +4 116 val_116 +4 126 val_126 +4 131 val_131 +4 133 val_133 +4 136 val_136 +4 143 val_143 +4 145 val_145 +4 150 val_150 +4 153 val_153 +4 155 val_155 +4 156 val_156 +4 157 val_157 +4 158 val_158 +4 160 val_160 +4 162 val_162 +4 163 val_163 +4 166 val_166 +4 168 val_168 +4 17 val_17 +4 170 val_170 +4 177 val_177 +4 178 val_178 +4 180 val_180 +4 181 val_181 +4 183 val_183 +4 186 val_186 +4 189 val_189 +4 19 val_19 +4 190 val_190 +4 192 val_192 +4 194 val_194 +4 196 val_196 +4 2 val_2 +4 20 val_20 +4 201 val_201 +4 202 val_202 +4 214 val_214 +4 218 val_218 +4 222 val_222 +4 226 val_226 +4 228 val_228 +4 235 val_235 +4 241 val_241 +4 244 val_244 +4 247 val_247 +4 248 val_248 +4 249 val_249 +4 252 val_252 +4 257 val_257 +4 258 val_258 +4 260 val_260 +4 262 val_262 +4 263 val_263 +4 266 val_266 +4 27 val_27 +4 274 val_274 +4 275 val_275 +4 28 val_28 +4 283 val_283 +4 284 val_284 +4 285 val_285 +4 286 val_286 +4 287 val_287 +4 289 val_289 +4 291 val_291 +4 292 val_292 +4 296 val_296 +4 30 val_30 +4 302 val_302 +4 305 val_305 +4 306 val_306 +4 308 val_308 +4 310 val_310 +4 315 val_315 +4 323 val_323 +4 33 val_33 +4 332 val_332 +4 335 val_335 +4 336 val_336 +4 338 val_338 +4 339 val_339 +4 34 val_34 +4 341 val_341 +4 345 val_345 +4 351 val_351 +4 356 val_356 +4 360 val_360 +4 362 val_362 +4 364 val_364 +4 365 val_365 +4 366 val_366 +4 368 val_368 +4 373 val_373 +4 374 val_374 +4 375 val_375 +4 377 val_377 +4 378 val_378 +4 379 val_379 +4 386 val_386 +4 389 val_389 +4 392 val_392 +4 393 val_393 +4 394 val_394 +4 4 val_4 +4 400 val_400 +4 402 val_402 +4 407 val_407 +4 41 val_41 +4 411 val_411 +4 418 val_418 +4 419 val_419 +4 421 val_421 +4 427 val_427 +4 43 val_43 +4 432 val_432 +4 435 val_435 +4 436 val_436 +4 437 val_437 +4 44 val_44 +4 443 val_443 +4 444 val_444 +4 446 val_446 +4 448 val_448 +4 449 val_449 +4 452 val_452 +4 453 val_453 +4 455 val_455 +4 457 val_457 +4 460 val_460 +4 467 val_467 +4 47 val_47 +4 470 val_470 +4 472 val_472 +4 475 val_475 +4 477 val_477 +4 479 val_479 +4 481 val_481 +4 482 val_482 +4 483 val_483 +4 484 val_484 +4 485 val_485 +4 487 val_487 +4 490 val_490 +4 491 val_491 +4 493 val_493 +4 494 val_494 +4 495 val_495 +4 496 val_496 +4 497 val_497 +4 53 val_53 +4 54 val_54 +4 57 val_57 +4 64 val_64 +4 65 val_65 +4 66 val_66 +4 69 val_69 +4 74 val_74 +4 77 val_77 +4 78 val_78 +4 8 val_8 +4 80 val_80 +4 82 val_82 +4 85 val_85 +4 86 val_86 +4 87 val_87 +4 9 val_9 +4 92 val_92 +4 96 val_96 +40 230 val_230 +40 348 val_348 +40 401 val_401 +40 469 val_469 Index: ql/src/test/results/clientpositive/spark/union27.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union27.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union27.q.out (working copy) @@ -0,0 +1,136 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS +create table jackson_sev_same as select * from src +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@jackson_sev_same +POSTHOOK: query: -- SORT_QUERY_RESULTS +create table jackson_sev_same as select * from src +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@jackson_sev_same +PREHOOK: query: create table dim_pho as select * from src +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@dim_pho +POSTHOOK: query: create table dim_pho as select * from src +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dim_pho +PREHOOK: query: create table jackson_sev_add as select * from src +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@jackson_sev_add +POSTHOOK: query: create table jackson_sev_add as select * from src +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@jackson_sev_add +PREHOOK: query: explain select b.* from jackson_sev_same a join (select * from dim_pho union all select * from jackson_sev_add)b on a.key=b.key and b.key=97 +PREHOOK: type: QUERY +POSTHOOK: query: explain select b.* from jackson_sev_same a join (select * from dim_pho union all select * from jackson_sev_add)b on a.key=b.key and b.key=97 +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 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: dim_pho + Filter Operator + predicate: (key = 97) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: '97' (type: string) + sort order: + + Map-reduce partition columns: '97' (type: string) + value expressions: _col1 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: jackson_sev_add + Filter Operator + predicate: (key = 97) (type: boolean) + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: '97' (type: string) + sort order: + + Map-reduce partition columns: '97' (type: string) + value expressions: _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 97) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '97' (type: string) + sort order: + + Map-reduce partition columns: '97' (type: string) + 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 key (type: string) + 1 _col0 (type: string) + outputColumnNames: _col6 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '97' (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 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 b.* from jackson_sev_same a join (select * from dim_pho union all select * from jackson_sev_add)b on a.key=b.key and b.key=97 +PREHOOK: type: QUERY +PREHOOK: Input: default@dim_pho +PREHOOK: Input: default@jackson_sev_add +PREHOOK: Input: default@jackson_sev_same +#### A masked pattern was here #### +POSTHOOK: query: select b.* from jackson_sev_same a join (select * from dim_pho union all select * from jackson_sev_add)b on a.key=b.key and b.key=97 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dim_pho +POSTHOOK: Input: default@jackson_sev_add +POSTHOOK: Input: default@jackson_sev_same +#### A masked pattern was here #### +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 Index: ql/src/test/results/clientpositive/spark/union3.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union3.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/union3.q.out (working copy) @@ -48,7 +48,7 @@ Reducer 5 <- Map 4 (GROUP, 1) Reducer 7 <- Map 6 (GROUP, 1) Reducer 9 <- Map 8 (GROUP, 1) - Reducer 3 <- Reducer 2 (SORT, 2), Reducer 5 (SORT, 2), Reducer 7 (SORT, 2), Reducer 9 (SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union31.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union31.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union31.q.out (working copy) @@ -0,0 +1,815 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +drop table t1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: -- SORT_QUERY_RESULTS + +drop table t1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table t2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table t2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table t1 as select * from src where key < 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1 as select * from src where key < 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: create table t2 as select * from src where key < 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@t2 +POSTHOOK: query: create table t2 as select * from src where key < 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t2 +PREHOOK: query: create table t3(key string, cnt int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t3 +POSTHOOK: query: create table t3(key string, cnt int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t3 +PREHOOK: query: create table t4(value string, cnt int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t4 +POSTHOOK: query: create table t4(value string, cnt int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t4 +PREHOOK: query: explain +from +(select * from t1 + union all + select * from t2 +) x +insert overwrite table t3 + select key, count(1) group by key +insert overwrite table t4 + select value, count(1) group by value +PREHOOK: type: QUERY +POSTHOOK: query: explain +from +(select * from t1 + union all + select * from t2 +) x +insert overwrite table t3 + select key, count(1) group by key +insert overwrite table t4 + select value, count(1) group by value +POSTHOOK: type: QUERY +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 5 (GROUP, 2), Map 7 (GROUP, 2) + Reducer 3 <- Map 6 (GROUP, 2), Map 8 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: t1 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 6 + Map Operator Tree: + TableScan + alias: t1 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 + Group By Operator + aggregations: count(1) + keys: _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 7 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + aggregations: count(1) + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Map 8 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 + Group By Operator + aggregations: count(1) + keys: _col1 (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + 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: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 70 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.t3 + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 70 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.t4 + + 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.t3 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-1 + 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.t4 + + Stage: Stage-4 + Stats-Aggr Operator + +PREHOOK: query: from +(select * from t1 + union all + select * from t2 +) x +insert overwrite table t3 + select key, count(1) group by key +insert overwrite table t4 + select value, count(1) group by value +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t3 +PREHOOK: Output: default@t4 +POSTHOOK: query: from +(select * from t1 + union all + select * from t2 +) x +insert overwrite table t3 + select key, count(1) group by key +insert overwrite table t4 + select value, count(1) group by value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t3 +POSTHOOK: Output: default@t4 +POSTHOOK: Lineage: t3.cnt EXPRESSION [(t1)t1.null, (t2)t2.null, ] +POSTHOOK: Lineage: t3.key EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), (t2)t2.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t4.cnt EXPRESSION [(t1)t1.null, (t2)t2.null, ] +POSTHOOK: Lineage: t4.value EXPRESSION [(t1)t1.FieldSchema(name:value, type:string, comment:null), (t2)t2.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select * from t3 +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: select * from t3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +0 6 +2 2 +4 2 +5 6 +8 2 +9 2 +PREHOOK: query: select * from t4 +PREHOOK: type: QUERY +PREHOOK: Input: default@t4 +#### A masked pattern was here #### +POSTHOOK: query: select * from t4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t4 +#### A masked pattern was here #### +val_0 6 +val_2 2 +val_4 2 +val_5 6 +val_8 2 +val_9 2 +PREHOOK: query: create table t5(c1 string, cnt int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t5 +POSTHOOK: query: create table t5(c1 string, cnt int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t5 +PREHOOK: query: create table t6(c1 string, cnt int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t6 +POSTHOOK: query: create table t6(c1 string, cnt int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t6 +PREHOOK: query: explain +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, count(1) as cnt from t2 group by key +) x +insert overwrite table t5 + select c1, sum(cnt) group by c1 +insert overwrite table t6 + select c1, sum(cnt) group by c1 +PREHOOK: type: QUERY +POSTHOOK: query: explain +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, count(1) as cnt from t2 group by key +) x +insert overwrite table t5 + select c1, sum(cnt) group by c1 +insert overwrite table t6 + select c1, sum(cnt) group by c1 +POSTHOOK: type: QUERY +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 1 (GROUP, 2) + Reducer 5 <- Map 4 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 2), Reducer 5 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 70 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: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: t2 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 70 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: 10 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: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Forward + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 35 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.t5 + Group By Operator + aggregations: sum(VALUE._col0) + keys: KEY._col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 35 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.t6 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + value expressions: _col1 (type: bigint) + + 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.t5 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-1 + 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.t6 + + Stage: Stage-4 + Stats-Aggr Operator + +PREHOOK: query: from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, count(1) as cnt from t2 group by key +) x +insert overwrite table t5 + select c1, sum(cnt) group by c1 +insert overwrite table t6 + select c1, sum(cnt) group by c1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t5 +PREHOOK: Output: default@t6 +POSTHOOK: query: from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, count(1) as cnt from t2 group by key +) x +insert overwrite table t5 + select c1, sum(cnt) group by c1 +insert overwrite table t6 + select c1, sum(cnt) group by c1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t5 +POSTHOOK: Output: default@t6 +POSTHOOK: Lineage: t5.c1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), (t2)t2.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t5.cnt EXPRESSION [(t1)t1.null, (t2)t2.null, ] +POSTHOOK: Lineage: t6.c1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), (t2)t2.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t6.cnt EXPRESSION [(t1)t1.null, (t2)t2.null, ] +PREHOOK: query: select * from t5 +PREHOOK: type: QUERY +PREHOOK: Input: default@t5 +#### A masked pattern was here #### +POSTHOOK: query: select * from t5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t5 +#### A masked pattern was here #### +0 6 +2 2 +4 2 +5 6 +8 2 +9 2 +PREHOOK: query: select * from t6 +PREHOOK: type: QUERY +PREHOOK: Input: default@t6 +#### A masked pattern was here #### +POSTHOOK: query: select * from t6 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t6 +#### A masked pattern was here #### +0 6 +2 2 +4 2 +5 6 +8 2 +9 2 +PREHOOK: query: drop table t1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: drop table t1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +PREHOOK: query: drop table t2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: drop table t2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +PREHOOK: query: create table t1 as select * from src where key < 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1 as select * from src where key < 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: create table t2 as select key, count(1) as cnt from src where key < 10 group by key +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@t2 +POSTHOOK: query: create table t2 as select key, count(1) as cnt from src where key < 10 group by key +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t2 +PREHOOK: query: create table t7(c1 string, cnt int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t7 +POSTHOOK: query: create table t7(c1 string, cnt int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t7 +PREHOOK: query: create table t8(c1 string, cnt int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t8 +POSTHOOK: query: create table t8(c1 string, cnt int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t8 +PREHOOK: query: explain +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, cnt from t2 +) x +insert overwrite table t7 + select c1, count(1) group by c1 +insert overwrite table t8 + select c1, count(1) group by c1 +PREHOOK: type: QUERY +POSTHOOK: query: explain +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, cnt from t2 +) x +insert overwrite table t7 + select c1, count(1) group by c1 +insert overwrite table t8 + select c1, count(1) group by c1 +POSTHOOK: type: QUERY +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 1 (GROUP, 2) + Reducer 3 <- Map 4 (GROUP PARTITION-LEVEL SORT, 2), Reducer 2 (GROUP PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 70 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: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Forward + Statistics: Num rows: 11 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: KEY._col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 24 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 24 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.t7 + Group By Operator + aggregations: count(1) + keys: KEY._col0 (type: string) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToInteger(_col1) (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 24 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 5 Data size: 24 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.t8 + + 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.t7 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-1 + 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.t8 + + Stage: Stage-4 + Stats-Aggr Operator + +PREHOOK: query: from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, cnt from t2 +) x +insert overwrite table t7 + select c1, count(1) group by c1 +insert overwrite table t8 + select c1, count(1) group by c1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t7 +PREHOOK: Output: default@t8 +POSTHOOK: query: from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, cnt from t2 +) x +insert overwrite table t7 + select c1, count(1) group by c1 +insert overwrite table t8 + select c1, count(1) group by c1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t7 +POSTHOOK: Output: default@t8 +POSTHOOK: Lineage: t7.c1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), (t2)t2.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t7.cnt EXPRESSION [(t1)t1.null, (t2)t2.null, ] +POSTHOOK: Lineage: t8.c1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), (t2)t2.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t8.cnt EXPRESSION [(t1)t1.null, (t2)t2.null, ] +PREHOOK: query: select * from t7 +PREHOOK: type: QUERY +PREHOOK: Input: default@t7 +#### A masked pattern was here #### +POSTHOOK: query: select * from t7 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t7 +#### A masked pattern was here #### +0 2 +2 2 +4 2 +5 2 +8 2 +9 2 +PREHOOK: query: select * from t8 +PREHOOK: type: QUERY +PREHOOK: Input: default@t8 +#### A masked pattern was here #### +POSTHOOK: query: select * from t8 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t8 +#### A masked pattern was here #### +0 2 +2 2 +4 2 +5 2 +8 2 +9 2 Index: ql/src/test/results/clientpositive/spark/union32.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union32.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union32.q.out (working copy) @@ -0,0 +1,669 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- This tests various union queries which have columns on one side of the query +-- being of double type and those on the other side another + +CREATE TABLE t1 AS SELECT * FROM src WHERE key < 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- This tests various union queries which have columns on one side of the query +-- being of double type and those on the other side another + +CREATE TABLE t1 AS SELECT * FROM src WHERE key < 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: CREATE TABLE t2 AS SELECT * FROM src WHERE key < 10 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@t2 +POSTHOOK: query: CREATE TABLE t2 AS SELECT * FROM src WHERE key < 10 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t2 +PREHOOK: query: -- Test simple union with double +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t1 +UNION ALL +SELECT CAST(key AS BIGINT) AS key FROM t2) a +ORDER BY key +PREHOOK: type: QUERY +POSTHOOK: query: -- Test simple union with double +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t1 +UNION ALL +SELECT CAST(key AS BIGINT) AS key FROM t2) a +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), Map 3 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + Select Operator + expressions: UDFToDouble(key) (type: double) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map 3 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: UDFToDouble(UDFToLong(key)) (type: double) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 20 Data size: 140 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 20 Data size: 140 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 +(SELECT CAST(key AS DOUBLE) AS key FROM t1 +UNION ALL +SELECT CAST(key AS BIGINT) AS key FROM t2) a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t1 +UNION ALL +SELECT CAST(key AS BIGINT) AS key FROM t2) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +2.0 +2.0 +4.0 +4.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +8.0 +8.0 +9.0 +9.0 +PREHOOK: query: -- Test union with join on the left +EXPLAIN +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key FROM t2) a +PREHOOK: type: QUERY +POSTHOOK: query: -- Test union with join on the left +EXPLAIN +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key FROM t2) 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 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: UDFToDouble(key) (type: double) + outputColumnNames: _col0 + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0 + Select Operator + expressions: UDFToDouble(UDFToLong(_col0)) (type: double) + outputColumnNames: _col0 + 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: SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key FROM t2) a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key FROM t2) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +2.0 +2.0 +4.0 +4.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +8.0 +8.0 +9.0 +9.0 +PREHOOK: query: -- Test union with join on the right +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key) a +PREHOOK: type: QUERY +POSTHOOK: query: -- Test union with join on the right +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key) 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 3 <- Map 2 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: UDFToDouble(key) (type: double) + outputColumnNames: _col0 + 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: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0 + Select Operator + expressions: UDFToDouble(UDFToLong(_col0)) (type: double) + outputColumnNames: _col0 + 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: SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key) a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +2.0 +2.0 +4.0 +4.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +5.0 +8.0 +8.0 +9.0 +9.0 +PREHOOK: query: -- Test union with join on the left selecting multiple columns +EXPLAIN +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2) a +PREHOOK: type: QUERY +POSTHOOK: query: -- Test union with join on the left selecting multiple columns +EXPLAIN +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2) 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 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: UDFToDouble(key) (type: double), key (type: string) + 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0, _col5 + Select Operator + expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToString(UDFToDouble(_col5)) (type: string) + 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: SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2) a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +0.0 0 +0.0 0 +0.0 0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +2.0 2 +2.0 2.0 +4.0 4 +4.0 4.0 +5.0 5 +5.0 5 +5.0 5 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +8.0 8 +8.0 8.0 +9.0 9 +9.0 9.0 +PREHOOK: query: -- Test union with join on the right selecting multiple columns +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key) a +PREHOOK: type: QUERY +POSTHOOK: query: -- Test union with join on the right selecting multiple columns +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key) 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 3 <- Map 2 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t2 + Select Operator + expressions: UDFToDouble(key) (type: double), UDFToDouble(key) (type: double) + 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: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 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: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0, _col5 + Select Operator + expressions: UDFToDouble(UDFToLong(_col0)) (type: double), UDFToDouble(_col5) (type: double) + 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: SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key) a +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +0.0 0.0 +2.0 2.0 +2.0 2.0 +4.0 4.0 +4.0 4.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +5.0 5.0 +8.0 8.0 +8.0 8.0 +9.0 9.0 +9.0 9.0 Index: ql/src/test/results/clientpositive/spark/union34.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union34.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union34.q.out (working copy) @@ -0,0 +1,406 @@ +PREHOOK: query: create table src10_1 (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src10_1 +POSTHOOK: query: create table src10_1 (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src10_1 +PREHOOK: query: create table src10_2 (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src10_2 +POSTHOOK: query: create table src10_2 (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src10_2 +PREHOOK: query: create table src10_3 (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src10_3 +POSTHOOK: query: create table src10_3 (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src10_3 +PREHOOK: query: create table src10_4 (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src10_4 +POSTHOOK: query: create table src10_4 (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src10_4 +PREHOOK: query: from (select * from src tablesample (10 rows)) a +insert overwrite table src10_1 select * +insert overwrite table src10_2 select * +insert overwrite table src10_3 select * +insert overwrite table src10_4 select * +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@src10_1 +PREHOOK: Output: default@src10_2 +PREHOOK: Output: default@src10_3 +PREHOOK: Output: default@src10_4 +POSTHOOK: query: from (select * from src tablesample (10 rows)) a +insert overwrite table src10_1 select * +insert overwrite table src10_2 select * +insert overwrite table src10_3 select * +insert overwrite table src10_4 select * +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@src10_1 +POSTHOOK: Output: default@src10_2 +POSTHOOK: Output: default@src10_3 +POSTHOOK: Output: default@src10_4 +POSTHOOK: Lineage: src10_1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src10_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src10_2.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src10_2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src10_3.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src10_3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: src10_4.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: src10_4.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- When we convert the Join of sub1 and sub0 into a MapJoin, +-- we can use a single MR job to evaluate this entire query. +explain +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +PREHOOK: type: QUERY +POSTHOOK: query: -- When we convert the Join of sub1 and sub0 into a MapJoin, +-- we can use a single MR job to evaluate this entire query. +explain +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +POSTHOOK: type: QUERY +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 1 + Map Operator Tree: + TableScan + alias: src10_1 + Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (SORT, 1), Map 4 (SORT, 1), Map 5 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: src10_2 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + input vertices: + 0 Map 1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Map 4 + Map Operator Tree: + TableScan + alias: src10_3 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + value expressions: _col1 (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: src10_4 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + value expressions: _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 353 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 353 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 ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src10_1 +PREHOOK: Input: default@src10_2 +PREHOOK: Input: default@src10_3 +PREHOOK: Input: default@src10_4 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src10_1 +POSTHOOK: Input: default@src10_2 +POSTHOOK: Input: default@src10_3 +POSTHOOK: Input: default@src10_4 +#### A masked pattern was here #### +165 val_165 +165 val_165 +165 val_165 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +27 val_27 +27 val_27 +27 val_27 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +409 val_409 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +484 val_484 +86 val_86 +86 val_86 +86 val_86 +98 val_98 +98 val_98 +98 val_98 +PREHOOK: query: -- When we do not convert the Join of sub1 and sub0 into a MapJoin, +-- we need to use two MR jobs to evaluate this query. +-- The first job is for the Join of sub1 and sub2. The second job +-- is for the UNION ALL and ORDER BY. +explain +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +PREHOOK: type: QUERY +POSTHOOK: query: -- When we do not convert the Join of sub1 and sub0 into a MapJoin, +-- we need to use two MR jobs to evaluate this query. +-- The first job is for the Join of sub1 and sub2. The second job +-- is for the UNION ALL and ORDER BY. +explain +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 5 (SORT, 1), Map 6 (SORT, 1), Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src10_1 + Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: src10_2 + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 114 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: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: src10_3 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + value expressions: _col1 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: src10_4 + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + value expressions: _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 353 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 353 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 ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@src10_1 +PREHOOK: Input: default@src10_2 +PREHOOK: Input: default@src10_3 +PREHOOK: Input: default@src10_4 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src10_1 +POSTHOOK: Input: default@src10_2 +POSTHOOK: Input: default@src10_3 +POSTHOOK: Input: default@src10_4 +#### A masked pattern was here #### +165 val_165 +165 val_165 +165 val_165 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +27 val_27 +27 val_27 +27 val_27 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +409 val_409 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +484 val_484 +86 val_86 +86 val_86 +86 val_86 +98 val_98 +98 val_98 +98 val_98 Index: ql/src/test/results/clientpositive/spark/union_date.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_date.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_date.q.out (working copy) @@ -0,0 +1,142 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +drop table union_date_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: -- SORT_QUERY_RESULTS + +drop table union_date_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table union_date_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table union_date_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table union_date_1 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@union_date_1 +POSTHOOK: query: create table union_date_1 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@union_date_1 +PREHOOK: query: create table union_date_2 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@union_date_2 +POSTHOOK: query: create table union_date_2 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@union_date_2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@union_date_1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@union_date_1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@union_date_2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@union_date_2 +PREHOOK: query: select * from ( + select fl_num, fl_date from union_date_1 + union all + select fl_num, fl_date from union_date_2 +) union_result +PREHOOK: type: QUERY +PREHOOK: Input: default@union_date_1 +PREHOOK: Input: default@union_date_2 +#### A masked pattern was here #### +POSTHOOK: query: select * from ( + select fl_num, fl_date from union_date_1 + union all + select fl_num, fl_date from union_date_2 +) union_result +POSTHOOK: type: QUERY +POSTHOOK: Input: default@union_date_1 +POSTHOOK: Input: default@union_date_2 +#### A masked pattern was here #### +1064 2000-11-20 +1064 2000-11-20 +1064 2000-11-28 +1064 2000-11-28 +1064 2010-10-20 +1064 2010-10-20 +1064 2010-10-28 +1064 2010-10-28 +1142 2000-11-21 +1142 2000-11-21 +1142 2000-11-28 +1142 2000-11-28 +1142 2010-10-21 +1142 2010-10-21 +1142 2010-10-29 +1142 2010-10-29 +1531 2000-11-25 +1531 2000-11-25 +1531 2010-10-25 +1531 2010-10-25 +1599 2000-11-22 +1599 2000-11-22 +1599 2010-10-22 +1599 2010-10-22 +1610 2000-11-26 +1610 2000-11-26 +1610 2010-10-26 +1610 2010-10-26 +3198 2000-11-27 +3198 2000-11-27 +3198 2010-10-27 +3198 2010-10-27 +361 2000-11-23 +361 2000-11-23 +361 2010-10-23 +361 2010-10-23 +897 2000-11-24 +897 2000-11-24 +897 2010-10-24 +897 2010-10-24 +PREHOOK: query: drop table union_date_1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@union_date_1 +PREHOOK: Output: default@union_date_1 +POSTHOOK: query: drop table union_date_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@union_date_1 +POSTHOOK: Output: default@union_date_1 +PREHOOK: query: drop table union_date_2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@union_date_2 +PREHOOK: Output: default@union_date_2 +POSTHOOK: query: drop table union_date_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@union_date_2 +POSTHOOK: Output: default@union_date_2 Index: ql/src/test/results/clientpositive/spark/union_date_trim.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_date_trim.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_date_trim.q.out (working copy) @@ -0,0 +1,54 @@ +PREHOOK: query: drop table if exists testDate +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table if exists testDate +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table testDate(id int, dt date) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@testDate +POSTHOOK: query: create table testDate(id int, dt date) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@testDate +PREHOOK: query: insert into table testDate select 1, '2014-04-07' from src where key=100 limit 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@testdate +POSTHOOK: query: insert into table testDate select 1, '2014-04-07' from src where key=100 limit 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@testdate +POSTHOOK: Lineage: testdate.dt EXPRESSION [] +POSTHOOK: Lineage: testdate.id SIMPLE [] +PREHOOK: query: insert into table testDate select 2, '2014-04-08' from src where key=100 limit 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@testdate +POSTHOOK: query: insert into table testDate select 2, '2014-04-08' from src where key=100 limit 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@testdate +POSTHOOK: Lineage: testdate.dt EXPRESSION [] +POSTHOOK: Lineage: testdate.id SIMPLE [] +PREHOOK: query: insert into table testDate select 3, '2014-04-09' from src where key=100 limit 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@testdate +POSTHOOK: query: insert into table testDate select 3, '2014-04-09' from src where key=100 limit 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@testdate +POSTHOOK: Lineage: testdate.dt EXPRESSION [] +POSTHOOK: Lineage: testdate.id SIMPLE [] +PREHOOK: query: --- without the fix following query will throw HiveException: Incompatible types for union operator +insert into table testDate select id, tm from (select id, dt as tm from testDate where id = 1 union all select id, dt as tm from testDate where id = 2 union all select id, trim(Cast (dt as string)) as tm from testDate where id = 3 ) a +PREHOOK: type: QUERY +PREHOOK: Input: default@testdate +PREHOOK: Output: default@testdate +POSTHOOK: query: --- without the fix following query will throw HiveException: Incompatible types for union operator +insert into table testDate select id, tm from (select id, dt as tm from testDate where id = 1 union all select id, dt as tm from testDate where id = 2 union all select id, trim(Cast (dt as string)) as tm from testDate where id = 3 ) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@testdate +POSTHOOK: Output: default@testdate +POSTHOOK: Lineage: testdate.dt EXPRESSION [(testdate)testdate.FieldSchema(name:dt, type:date, comment:null), (testdate)testdate.FieldSchema(name:dt, type:date, comment:null), (testdate)testdate.FieldSchema(name:dt, type:date, comment:null), ] +POSTHOOK: Lineage: testdate.id EXPRESSION [(testdate)testdate.FieldSchema(name:id, type:int, comment:null), (testdate)testdate.FieldSchema(name:id, type:int, comment:null), (testdate)testdate.FieldSchema(name:id, type:int, comment:null), ] Index: ql/src/test/results/clientpositive/spark/union_lateralview.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_lateralview.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_lateralview.q.out (working copy) @@ -0,0 +1,251 @@ +PREHOOK: query: create table test_union_lateral_view(key int, arr_ele int, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_union_lateral_view +POSTHOOK: query: create table test_union_lateral_view(key int, arr_ele int, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_union_lateral_view +PREHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE test_union_lateral_view +SELECT b.key, d.arr_ele, d.value +FROM ( + SELECT c.arr_ele as arr_ele, a.key as key, a.value as value + FROM ( + SELECT key, value, array(1,2,3) as arr + FROM src + + UNION ALL + + SELECT key, value, array(1,2,3) as arr + FROM srcpart + WHERE ds = '2008-04-08' and hr='12' + ) a LATERAL VIEW EXPLODE(arr) c AS arr_ele +) d +LEFT OUTER JOIN src b +ON d.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE test_union_lateral_view +SELECT b.key, d.arr_ele, d.value +FROM ( + SELECT c.arr_ele as arr_ele, a.key as key, a.value as value + FROM ( + SELECT key, value, array(1,2,3) as arr + FROM src + + UNION ALL + + SELECT key, value, array(1,2,3) as arr + FROM srcpart + WHERE ds = '2008-04-08' and hr='12' + ) a LATERAL VIEW EXPLODE(arr) c AS arr_ele +) d +LEFT OUTER JOIN src b +ON d.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 + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Select Operator + expressions: key (type: string), value (type: string), array(1,2,3) (type: array) + outputColumnNames: _col0, _col1, _col2 + Lateral View Forward + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col3 (type: int), _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + value expressions: _col0 (type: int), _col2 (type: string) + Select Operator + expressions: _col2 (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col3 (type: int), _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + value expressions: _col0 (type: int), _col2 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: key (type: string), value (type: string), array(1,2,3) (type: array) + outputColumnNames: _col0, _col1, _col2 + Lateral View Forward + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col3 (type: int), _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + value expressions: _col0 (type: int), _col2 (type: string) + Select Operator + expressions: _col2 (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col3 (type: int), _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + value expressions: _col0 (type: int), _col2 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: b + 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: key (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 _col1 (type: string) + 1 key (type: string) + outputColumnNames: _col0, _col2, _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToInteger(_col3) (type: int), _col0 (type: int), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2200 Data size: 23372 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_union_lateral_view + + 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.test_union_lateral_view + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: INSERT OVERWRITE TABLE test_union_lateral_view +SELECT b.key, d.arr_ele, d.value +FROM ( + SELECT c.arr_ele as arr_ele, a.key as key, a.value as value + FROM ( + SELECT key, value, array(1,2,3) as arr + FROM src + + UNION ALL + + SELECT key, value, array(1,2,3) as arr + FROM srcpart + WHERE ds = '2008-04-08' and hr='12' + ) a LATERAL VIEW EXPLODE(arr) c AS arr_ele +) d +LEFT OUTER JOIN src b +ON d.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@test_union_lateral_view +POSTHOOK: query: INSERT OVERWRITE TABLE test_union_lateral_view +SELECT b.key, d.arr_ele, d.value +FROM ( + SELECT c.arr_ele as arr_ele, a.key as key, a.value as value + FROM ( + SELECT key, value, array(1,2,3) as arr + FROM src + + UNION ALL + + SELECT key, value, array(1,2,3) as arr + FROM srcpart + WHERE ds = '2008-04-08' and hr='12' + ) a LATERAL VIEW EXPLODE(arr) c AS arr_ele +) d +LEFT OUTER JOIN src b +ON d.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@test_union_lateral_view +POSTHOOK: Lineage: test_union_lateral_view.arr_ele EXPRESSION [] +POSTHOOK: Lineage: test_union_lateral_view.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_union_lateral_view.value EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +PREHOOK: query: select key, arr_ele, value from test_union_lateral_view order by key, arr_ele limit 20 +PREHOOK: type: QUERY +PREHOOK: Input: default@test_union_lateral_view +#### A masked pattern was here #### +POSTHOOK: query: select key, arr_ele, value from test_union_lateral_view order by key, arr_ele limit 20 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_union_lateral_view +#### A masked pattern was here #### +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 2 val_0 +0 2 val_0 Index: ql/src/test/results/clientpositive/spark/union_ppr.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_ppr.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/union_ppr.q.out (working copy) @@ -116,7 +116,7 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2), Map 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 Index: ql/src/test/results/clientpositive/spark/union_remove_12.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_12.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_remove_12.q.out (working copy) @@ -0,0 +1,281 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one is a map-join query), followed by select star and a file sink. +-- The union optimization is applied, and the union is removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one is a map-join query), followed by select star and a file sink. +-- The union optimization is applied, and the union is removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, `values` bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, `values` bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-7 is a root stage + Stage-1 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-1 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + +STAGE PLANS: + Stage: Stage-7 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), UDFToString(1) (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + 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.outputtbl1 + Map 2 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0, _col6 + input vertices: + 1 Map 3 + Select Operator + expressions: _col0 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + 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.outputtbl1 + Local Work: + Map Reduce Local Work + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + 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.outputtbl1 + + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Spark Merge File Work + Merge File Operator + Map Operator Tree: + RCFile Merge Operator + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-4 + Spark +#### A masked pattern was here #### + Vertices: + Spark Merge File Work + Merge File Operator + Map Operator Tree: + RCFile Merge Operator + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)a.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# 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 false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 194 +#### 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: select * from outputTbl1 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 11 +2 1 +2 12 +3 1 +3 13 +7 1 +7 17 +8 1 +8 1 +8 18 +8 18 +8 28 +8 28 Index: ql/src/test/results/clientpositive/spark/union_remove_13.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_13.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_remove_13.q.out (working copy) @@ -0,0 +1,306 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a mapred query, and the +-- other one is a map-join query), followed by select star and a file sink. +-- The union selectstar optimization should be performed, and the union should be removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a mapred query, and the +-- other one is a map-join query), followed by select star and a file sink. +-- The union selectstar optimization should be performed, and the union should be removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, `values` bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, `values` bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as `values` from inputTbl1 group by key +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as `values` from inputTbl1 group by key +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-7 is a root stage + Stage-1 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-1 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + +STAGE PLANS: + Stage: Stage-7 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0, _col6 + input vertices: + 1 Map 4 + Select Operator + expressions: _col0 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + 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.outputtbl1 + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToString(_col1) (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + 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.outputtbl1 + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + 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.outputtbl1 + + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Spark Merge File Work + Merge File Operator + Map Operator Tree: + RCFile Merge Operator + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-4 + Spark +#### A masked pattern was here #### + Vertices: + Spark Merge File Work + Merge File Operator + Map Operator Tree: + RCFile Merge Operator + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as `values` from inputTbl1 group by key +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as `values` from inputTbl1 group by key +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)a.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# 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 false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 271 +#### 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: select * from outputTbl1 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 11 +2 1 +2 12 +3 1 +3 13 +7 1 +7 17 +8 2 +8 18 +8 18 +8 28 +8 28 Index: ql/src/test/results/clientpositive/spark/union_remove_14.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_14.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_remove_14.q.out (working copy) @@ -0,0 +1,283 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a join, which should be performed as a map-join query at runtime), +-- followed by select star and a file sink. +-- The union selectstar optimization should be performed, and the union should be removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a join, which should be performed as a map-join query at runtime), +-- followed by select star and a file sink. +-- The union selectstar optimization should be performed, and the union should be removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, `values` bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, `values` bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-7 is a root stage + Stage-1 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-1 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + +STAGE PLANS: + Stage: Stage-7 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 key (type: string) + 1 key (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), UDFToString(1) (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + 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.outputtbl1 + Map 2 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: string) + 1 key (type: string) + outputColumnNames: _col0, _col6 + input vertices: + 1 Map 3 + Select Operator + expressions: _col0 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + 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.outputtbl1 + Local Work: + Map Reduce Local Work + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + 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.outputtbl1 + + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Spark Merge File Work + Merge File Operator + Map Operator Tree: + RCFile Merge Operator + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-4 + Spark +#### A masked pattern was here #### + Vertices: + Spark Merge File Work + Merge File Operator + Map Operator Tree: + RCFile Merge Operator + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as `values` from inputTbl1 +union all +select a.key as key, b.val as `values` +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)a.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# 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 false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 194 +#### 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: select * from outputTbl1 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 11 +2 1 +2 12 +3 1 +3 13 +7 1 +7 17 +8 1 +8 1 +8 18 +8 18 +8 28 +8 28 Index: ql/src/test/results/clientpositive/spark/union_remove_22.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_22.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_remove_22.q.out (working copy) @@ -0,0 +1,401 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, some columns are repeated. So, union cannot be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which selects columns from +-- both the sub-qeuries of the union) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, some columns are repeated. So, union cannot be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which selects columns from +-- both the sub-qeuries of the union) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, `values` bigint, values2 bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, `values` bigint, values2 bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, a.`values`, a.`values` +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, a.`values`, a.`values` +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) 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 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + 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: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + 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 + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + 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 + name: default.outputtbl1 + + 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.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.`values`, a.`values` +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.`values`, a.`values` +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint +values2 bigint + +# 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 false + numFiles 4 + numRows -1 + rawDataSize -1 + totalSize 60 +#### 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 outputTbl1 +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 1 +1 1 1 +2 1 1 +2 1 1 +3 1 1 +3 1 1 +7 1 1 +7 1 1 +8 2 2 +8 2 2 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, concat(a.`values`, a.`values`), concat(a.`values`, a.`values`) +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, concat(a.`values`, a.`values`), concat(a.`values`, a.`values`) +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) 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 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + 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: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(concat(_col1, _col1)) (type: bigint), UDFToLong(concat(_col1, _col1)) (type: bigint) + outputColumnNames: _col0, _col1, _col2 + 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 + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(concat(_col1, _col1)) (type: bigint), UDFToLong(concat(_col1, _col1)) (type: bigint) + outputColumnNames: _col0, _col1, _col2 + 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 + name: default.outputtbl1 + + 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.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, concat(a.`values`, a.`values`), concat(a.`values`, a.`values`) +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, concat(a.`values`, a.`values`), concat(a.`values`, a.`values`) +FROM ( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 11 11 +1 11 11 +2 11 11 +2 11 11 +3 11 11 +3 11 11 +7 11 11 +7 11 11 +8 22 22 +8 22 22 Index: ql/src/test/results/clientpositive/spark/union_remove_23.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_23.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_remove_23.q.out (working copy) @@ -0,0 +1,265 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. One of the sub-queries +-- would have multiple map-reduce jobs. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. One of the sub-queries +-- would have multiple map-reduce jobs. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, `values` bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, `values` bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as `values` from + (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) subq2 +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as `values` from + (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) subq2 +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 6 <- Map 5 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + 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 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: _col0 (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + 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 + name: default.outputtbl1 + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + 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 + name: default.outputtbl1 + + 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.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as `values` from + (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) subq2 +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as `values` from + (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key +) subq2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)a.null, (inputtbl1)a.null, (inputtbl1)b.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# 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 false + numFiles 4 + numRows -1 + rawDataSize -1 + totalSize 40 +#### 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 outputTbl1 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 4 Index: ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out (working copy) @@ -0,0 +1,1181 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, `values` bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, `values` bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: create table outputTbl2(key string, `values` bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl2 +POSTHOOK: query: create table outputTbl2(key string, `values` bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl2 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +FROM ( + select * from( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key + )subq +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +PREHOOK: type: QUERY +POSTHOOK: query: explain +FROM ( + select * from( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key + )subq +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-0 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + 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: string) + mode: mergepartial + 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 + name: default.outputtbl1 + 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 + name: default.outputtbl2 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + 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 + name: default.outputtbl1 + 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 + name: default.outputtbl2 + + 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.outputtbl1 + + Stage: Stage-1 + 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.outputtbl2 + +PREHOOK: query: FROM ( + select * from( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key + )subq +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +PREHOOK: Output: default@outputtbl2 +POSTHOOK: query: FROM ( + select * from( + SELECT key, count(1) as `values` from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as `values` from inputTbl1 group by key + )subq +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Output: default@outputtbl2 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl2.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl2.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 +PREHOOK: query: select * from outputTbl2 order by key, `values` +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl2 order by key, `values` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl2 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 +PREHOOK: query: -- The following queries guarantee the correctness. +explain +select avg(c) from( + SELECT count(1)-200 as c from src + UNION ALL + SELECT count(1) as c from src +)subq +PREHOOK: type: QUERY +POSTHOOK: query: -- The following queries guarantee the correctness. +explain +select avg(c) from( + SELECT count(1)-200 as c from src + UNION ALL + SELECT count(1) as c from src +)subq +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) + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 3 <- Reducer 2 (GROUP, 1), Reducer 5 (GROUP, 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: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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) + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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 + Select Operator + expressions: (_col0 - 200) (type: bigint) + outputColumnNames: _col0 + Group By Operator + aggregations: avg(_col0) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: struct) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: avg(VALUE._col0) + mode: mergepartial + 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Group By Operator + aggregations: avg(_col0) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: struct) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select avg(c) from( + SELECT count(1)-200 as c from src + UNION ALL + SELECT count(1) as c from src +)subq +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select avg(c) from( + SELECT count(1)-200 as c from src + UNION ALL + SELECT count(1) as c from src +)subq +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +400.0 +PREHOOK: query: explain +select key, avg(c) over w from( + SELECT key, count(1)*2 as c from src group by key + UNION ALL + SELECT key, count(1) as c from src group by key +)subq group by key, c +WINDOW w AS (PARTITION BY key ORDER BY c ROWS UNBOUNDED PRECEDING) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select key, avg(c) over w from( + SELECT key, count(1)*2 as c from src group by key + UNION ALL + SELECT key, count(1) as c from src group by key +)subq group by key, c +WINDOW w AS (PARTITION BY key ORDER BY c ROWS UNBOUNDED PRECEDING) +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 6 <- Map 5 (GROUP, 2) + Reducer 3 <- Reducer 2 (GROUP, 2), Reducer 6 (GROUP, 2) + Reducer 4 <- Reducer 3 (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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + 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: bigint) + Map 5 + 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) + outputColumnNames: key + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + 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: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), (_col1 * 2) (type: bigint) + outputColumnNames: _col0, _col1 + Group By Operator + keys: _col0 (type: string), _col1 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string), KEY._col1 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 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) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: string, _col1: bigint + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: _wcol0 + arguments: _col1 + name: avg + window function: GenericUDAFAverageEvaluatorDouble + window frame: PRECEDING(MAX)~ + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _wcol0 (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 + Reducer 6 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Group By Operator + keys: _col0 (type: string), _col1 (type: bigint) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: bigint) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select key, avg(c) over w from( + SELECT key, count(1)*2 as c from src group by key + UNION ALL + SELECT key, count(1) as c from src group by key +)subq group by key, c +WINDOW w AS (PARTITION BY key ORDER BY c ROWS UNBOUNDED PRECEDING) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select key, avg(c) over w from( + SELECT key, count(1)*2 as c from src group by key + UNION ALL + SELECT key, count(1) as c from src group by key +)subq group by key, c +WINDOW w AS (PARTITION BY key ORDER BY c ROWS UNBOUNDED PRECEDING) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 3.0 +0 4.5 +10 1.0 +10 1.5 +100 2.0 +100 3.0 +103 2.0 +103 3.0 +104 2.0 +104 3.0 +105 1.0 +105 1.5 +11 1.0 +11 1.5 +111 1.0 +111 1.5 +113 2.0 +113 3.0 +114 1.0 +114 1.5 +116 1.0 +116 1.5 +118 2.0 +118 3.0 +119 3.0 +119 4.5 +12 2.0 +12 3.0 +120 2.0 +120 3.0 +125 2.0 +125 3.0 +126 1.0 +126 1.5 +128 3.0 +128 4.5 +129 2.0 +129 3.0 +131 1.0 +131 1.5 +133 1.0 +133 1.5 +134 2.0 +134 3.0 +136 1.0 +136 1.5 +137 2.0 +137 3.0 +138 4.0 +138 6.0 +143 1.0 +143 1.5 +145 1.0 +145 1.5 +146 2.0 +146 3.0 +149 2.0 +149 3.0 +15 2.0 +15 3.0 +150 1.0 +150 1.5 +152 2.0 +152 3.0 +153 1.0 +153 1.5 +155 1.0 +155 1.5 +156 1.0 +156 1.5 +157 1.0 +157 1.5 +158 1.0 +158 1.5 +160 1.0 +160 1.5 +162 1.0 +162 1.5 +163 1.0 +163 1.5 +164 2.0 +164 3.0 +165 2.0 +165 3.0 +166 1.0 +166 1.5 +167 3.0 +167 4.5 +168 1.0 +168 1.5 +169 4.0 +169 6.0 +17 1.0 +17 1.5 +170 1.0 +170 1.5 +172 2.0 +172 3.0 +174 2.0 +174 3.0 +175 2.0 +175 3.0 +176 2.0 +176 3.0 +177 1.0 +177 1.5 +178 1.0 +178 1.5 +179 2.0 +179 3.0 +18 2.0 +18 3.0 +180 1.0 +180 1.5 +181 1.0 +181 1.5 +183 1.0 +183 1.5 +186 1.0 +186 1.5 +187 3.0 +187 4.5 +189 1.0 +189 1.5 +19 1.0 +19 1.5 +190 1.0 +190 1.5 +191 2.0 +191 3.0 +192 1.0 +192 1.5 +193 3.0 +193 4.5 +194 1.0 +194 1.5 +195 2.0 +195 3.0 +196 1.0 +196 1.5 +197 2.0 +197 3.0 +199 3.0 +199 4.5 +2 1.0 +2 1.5 +20 1.0 +20 1.5 +200 2.0 +200 3.0 +201 1.0 +201 1.5 +202 1.0 +202 1.5 +203 2.0 +203 3.0 +205 2.0 +205 3.0 +207 2.0 +207 3.0 +208 3.0 +208 4.5 +209 2.0 +209 3.0 +213 2.0 +213 3.0 +214 1.0 +214 1.5 +216 2.0 +216 3.0 +217 2.0 +217 3.0 +218 1.0 +218 1.5 +219 2.0 +219 3.0 +221 2.0 +221 3.0 +222 1.0 +222 1.5 +223 2.0 +223 3.0 +224 2.0 +224 3.0 +226 1.0 +226 1.5 +228 1.0 +228 1.5 +229 2.0 +229 3.0 +230 5.0 +230 7.5 +233 2.0 +233 3.0 +235 1.0 +235 1.5 +237 2.0 +237 3.0 +238 2.0 +238 3.0 +239 2.0 +239 3.0 +24 2.0 +24 3.0 +241 1.0 +241 1.5 +242 2.0 +242 3.0 +244 1.0 +244 1.5 +247 1.0 +247 1.5 +248 1.0 +248 1.5 +249 1.0 +249 1.5 +252 1.0 +252 1.5 +255 2.0 +255 3.0 +256 2.0 +256 3.0 +257 1.0 +257 1.5 +258 1.0 +258 1.5 +26 2.0 +26 3.0 +260 1.0 +260 1.5 +262 1.0 +262 1.5 +263 1.0 +263 1.5 +265 2.0 +265 3.0 +266 1.0 +266 1.5 +27 1.0 +27 1.5 +272 2.0 +272 3.0 +273 3.0 +273 4.5 +274 1.0 +274 1.5 +275 1.0 +275 1.5 +277 4.0 +277 6.0 +278 2.0 +278 3.0 +28 1.0 +28 1.5 +280 2.0 +280 3.0 +281 2.0 +281 3.0 +282 2.0 +282 3.0 +283 1.0 +283 1.5 +284 1.0 +284 1.5 +285 1.0 +285 1.5 +286 1.0 +286 1.5 +287 1.0 +287 1.5 +288 2.0 +288 3.0 +289 1.0 +289 1.5 +291 1.0 +291 1.5 +292 1.0 +292 1.5 +296 1.0 +296 1.5 +298 3.0 +298 4.5 +30 1.0 +30 1.5 +302 1.0 +302 1.5 +305 1.0 +305 1.5 +306 1.0 +306 1.5 +307 2.0 +307 3.0 +308 1.0 +308 1.5 +309 2.0 +309 3.0 +310 1.0 +310 1.5 +311 3.0 +311 4.5 +315 1.0 +315 1.5 +316 3.0 +316 4.5 +317 2.0 +317 3.0 +318 3.0 +318 4.5 +321 2.0 +321 3.0 +322 2.0 +322 3.0 +323 1.0 +323 1.5 +325 2.0 +325 3.0 +327 3.0 +327 4.5 +33 1.0 +33 1.5 +331 2.0 +331 3.0 +332 1.0 +332 1.5 +333 2.0 +333 3.0 +335 1.0 +335 1.5 +336 1.0 +336 1.5 +338 1.0 +338 1.5 +339 1.0 +339 1.5 +34 1.0 +34 1.5 +341 1.0 +341 1.5 +342 2.0 +342 3.0 +344 2.0 +344 3.0 +345 1.0 +345 1.5 +348 5.0 +348 7.5 +35 3.0 +35 4.5 +351 1.0 +351 1.5 +353 2.0 +353 3.0 +356 1.0 +356 1.5 +360 1.0 +360 1.5 +362 1.0 +362 1.5 +364 1.0 +364 1.5 +365 1.0 +365 1.5 +366 1.0 +366 1.5 +367 2.0 +367 3.0 +368 1.0 +368 1.5 +369 3.0 +369 4.5 +37 2.0 +37 3.0 +373 1.0 +373 1.5 +374 1.0 +374 1.5 +375 1.0 +375 1.5 +377 1.0 +377 1.5 +378 1.0 +378 1.5 +379 1.0 +379 1.5 +382 2.0 +382 3.0 +384 3.0 +384 4.5 +386 1.0 +386 1.5 +389 1.0 +389 1.5 +392 1.0 +392 1.5 +393 1.0 +393 1.5 +394 1.0 +394 1.5 +395 2.0 +395 3.0 +396 3.0 +396 4.5 +397 2.0 +397 3.0 +399 2.0 +399 3.0 +4 1.0 +4 1.5 +400 1.0 +400 1.5 +401 5.0 +401 7.5 +402 1.0 +402 1.5 +403 3.0 +403 4.5 +404 2.0 +404 3.0 +406 4.0 +406 6.0 +407 1.0 +407 1.5 +409 3.0 +409 4.5 +41 1.0 +41 1.5 +411 1.0 +411 1.5 +413 2.0 +413 3.0 +414 2.0 +414 3.0 +417 3.0 +417 4.5 +418 1.0 +418 1.5 +419 1.0 +419 1.5 +42 2.0 +42 3.0 +421 1.0 +421 1.5 +424 2.0 +424 3.0 +427 1.0 +427 1.5 +429 2.0 +429 3.0 +43 1.0 +43 1.5 +430 3.0 +430 4.5 +431 3.0 +431 4.5 +432 1.0 +432 1.5 +435 1.0 +435 1.5 +436 1.0 +436 1.5 +437 1.0 +437 1.5 +438 3.0 +438 4.5 +439 2.0 +439 3.0 +44 1.0 +44 1.5 +443 1.0 +443 1.5 +444 1.0 +444 1.5 +446 1.0 +446 1.5 +448 1.0 +448 1.5 +449 1.0 +449 1.5 +452 1.0 +452 1.5 +453 1.0 +453 1.5 +454 3.0 +454 4.5 +455 1.0 +455 1.5 +457 1.0 +457 1.5 +458 2.0 +458 3.0 +459 2.0 +459 3.0 +460 1.0 +460 1.5 +462 2.0 +462 3.0 +463 2.0 +463 3.0 +466 3.0 +466 4.5 +467 1.0 +467 1.5 +468 4.0 +468 6.0 +469 5.0 +469 7.5 +47 1.0 +47 1.5 +470 1.0 +470 1.5 +472 1.0 +472 1.5 +475 1.0 +475 1.5 +477 1.0 +477 1.5 +478 2.0 +478 3.0 +479 1.0 +479 1.5 +480 3.0 +480 4.5 +481 1.0 +481 1.5 +482 1.0 +482 1.5 +483 1.0 +483 1.5 +484 1.0 +484 1.5 +485 1.0 +485 1.5 +487 1.0 +487 1.5 +489 4.0 +489 6.0 +490 1.0 +490 1.5 +491 1.0 +491 1.5 +492 2.0 +492 3.0 +493 1.0 +493 1.5 +494 1.0 +494 1.5 +495 1.0 +495 1.5 +496 1.0 +496 1.5 +497 1.0 +497 1.5 +498 3.0 +498 4.5 +5 3.0 +5 4.5 +51 2.0 +51 3.0 +53 1.0 +53 1.5 +54 1.0 +54 1.5 +57 1.0 +57 1.5 +58 2.0 +58 3.0 +64 1.0 +64 1.5 +65 1.0 +65 1.5 +66 1.0 +66 1.5 +67 2.0 +67 3.0 +69 1.0 +69 1.5 +70 3.0 +70 4.5 +72 2.0 +72 3.0 +74 1.0 +74 1.5 +76 2.0 +76 3.0 +77 1.0 +77 1.5 +78 1.0 +78 1.5 +8 1.0 +8 1.5 +80 1.0 +80 1.5 +82 1.0 +82 1.5 +83 2.0 +83 3.0 +84 2.0 +84 3.0 +85 1.0 +85 1.5 +86 1.0 +86 1.5 +87 1.0 +87 1.5 +9 1.0 +9 1.5 +90 3.0 +90 4.5 +92 1.0 +92 1.5 +95 2.0 +95 3.0 +96 1.0 +96 1.5 +97 2.0 +97 3.0 +98 2.0 +98 3.0 Index: ql/src/test/results/clientpositive/spark/union_script.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_script.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_script.q.out (working copy) @@ -0,0 +1,1524 @@ +PREHOOK: query: select * from ( + select transform(key) using 'cat' as cola from src)s order by cola +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from ( + select transform(key) using 'cat' as cola from src)s order by cola +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 +0 +0 +10 +100 +100 +103 +103 +104 +104 +105 +11 +111 +113 +113 +114 +116 +118 +118 +119 +119 +119 +12 +12 +120 +120 +125 +125 +126 +128 +128 +128 +129 +129 +131 +133 +134 +134 +136 +137 +137 +138 +138 +138 +138 +143 +145 +146 +146 +149 +149 +15 +15 +150 +152 +152 +153 +155 +156 +157 +158 +160 +162 +163 +164 +164 +165 +165 +166 +167 +167 +167 +168 +169 +169 +169 +169 +17 +170 +172 +172 +174 +174 +175 +175 +176 +176 +177 +178 +179 +179 +18 +18 +180 +181 +183 +186 +187 +187 +187 +189 +19 +190 +191 +191 +192 +193 +193 +193 +194 +195 +195 +196 +197 +197 +199 +199 +199 +2 +20 +200 +200 +201 +202 +203 +203 +205 +205 +207 +207 +208 +208 +208 +209 +209 +213 +213 +214 +216 +216 +217 +217 +218 +219 +219 +221 +221 +222 +223 +223 +224 +224 +226 +228 +229 +229 +230 +230 +230 +230 +230 +233 +233 +235 +237 +237 +238 +238 +239 +239 +24 +24 +241 +242 +242 +244 +247 +248 +249 +252 +255 +255 +256 +256 +257 +258 +26 +26 +260 +262 +263 +265 +265 +266 +27 +272 +272 +273 +273 +273 +274 +275 +277 +277 +277 +277 +278 +278 +28 +280 +280 +281 +281 +282 +282 +283 +284 +285 +286 +287 +288 +288 +289 +291 +292 +296 +298 +298 +298 +30 +302 +305 +306 +307 +307 +308 +309 +309 +310 +311 +311 +311 +315 +316 +316 +316 +317 +317 +318 +318 +318 +321 +321 +322 +322 +323 +325 +325 +327 +327 +327 +33 +331 +331 +332 +333 +333 +335 +336 +338 +339 +34 +341 +342 +342 +344 +344 +345 +348 +348 +348 +348 +348 +35 +35 +35 +351 +353 +353 +356 +360 +362 +364 +365 +366 +367 +367 +368 +369 +369 +369 +37 +37 +373 +374 +375 +377 +378 +379 +382 +382 +384 +384 +384 +386 +389 +392 +393 +394 +395 +395 +396 +396 +396 +397 +397 +399 +399 +4 +400 +401 +401 +401 +401 +401 +402 +403 +403 +403 +404 +404 +406 +406 +406 +406 +407 +409 +409 +409 +41 +411 +413 +413 +414 +414 +417 +417 +417 +418 +419 +42 +42 +421 +424 +424 +427 +429 +429 +43 +430 +430 +430 +431 +431 +431 +432 +435 +436 +437 +438 +438 +438 +439 +439 +44 +443 +444 +446 +448 +449 +452 +453 +454 +454 +454 +455 +457 +458 +458 +459 +459 +460 +462 +462 +463 +463 +466 +466 +466 +467 +468 +468 +468 +468 +469 +469 +469 +469 +469 +47 +470 +472 +475 +477 +478 +478 +479 +480 +480 +480 +481 +482 +483 +484 +485 +487 +489 +489 +489 +489 +490 +491 +492 +492 +493 +494 +495 +496 +497 +498 +498 +498 +5 +5 +5 +51 +51 +53 +54 +57 +58 +58 +64 +65 +66 +67 +67 +69 +70 +70 +70 +72 +72 +74 +76 +76 +77 +78 +8 +80 +82 +83 +83 +84 +84 +85 +86 +87 +9 +90 +90 +90 +92 +95 +95 +96 +97 +97 +98 +98 +PREHOOK: query: select * from ( + select transform(key) using 'cat' as cola from src + union all + select transform(key) using 'cat' as cola from src) s order by cola +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from ( + select transform(key) using 'cat' as cola from src + union all + select transform(key) using 'cat' as cola from src) s order by cola +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +0 +0 +0 +0 +0 +0 +10 +10 +100 +100 +100 +100 +103 +103 +103 +103 +104 +104 +104 +104 +105 +105 +11 +11 +111 +111 +113 +113 +113 +113 +114 +114 +116 +116 +118 +118 +118 +118 +119 +119 +119 +119 +119 +119 +12 +12 +12 +12 +120 +120 +120 +120 +125 +125 +125 +125 +126 +126 +128 +128 +128 +128 +128 +128 +129 +129 +129 +129 +131 +131 +133 +133 +134 +134 +134 +134 +136 +136 +137 +137 +137 +137 +138 +138 +138 +138 +138 +138 +138 +138 +143 +143 +145 +145 +146 +146 +146 +146 +149 +149 +149 +149 +15 +15 +15 +15 +150 +150 +152 +152 +152 +152 +153 +153 +155 +155 +156 +156 +157 +157 +158 +158 +160 +160 +162 +162 +163 +163 +164 +164 +164 +164 +165 +165 +165 +165 +166 +166 +167 +167 +167 +167 +167 +167 +168 +168 +169 +169 +169 +169 +169 +169 +169 +169 +17 +17 +170 +170 +172 +172 +172 +172 +174 +174 +174 +174 +175 +175 +175 +175 +176 +176 +176 +176 +177 +177 +178 +178 +179 +179 +179 +179 +18 +18 +18 +18 +180 +180 +181 +181 +183 +183 +186 +186 +187 +187 +187 +187 +187 +187 +189 +189 +19 +19 +190 +190 +191 +191 +191 +191 +192 +192 +193 +193 +193 +193 +193 +193 +194 +194 +195 +195 +195 +195 +196 +196 +197 +197 +197 +197 +199 +199 +199 +199 +199 +199 +2 +2 +20 +20 +200 +200 +200 +200 +201 +201 +202 +202 +203 +203 +203 +203 +205 +205 +205 +205 +207 +207 +207 +207 +208 +208 +208 +208 +208 +208 +209 +209 +209 +209 +213 +213 +213 +213 +214 +214 +216 +216 +216 +216 +217 +217 +217 +217 +218 +218 +219 +219 +219 +219 +221 +221 +221 +221 +222 +222 +223 +223 +223 +223 +224 +224 +224 +224 +226 +226 +228 +228 +229 +229 +229 +229 +230 +230 +230 +230 +230 +230 +230 +230 +230 +230 +233 +233 +233 +233 +235 +235 +237 +237 +237 +237 +238 +238 +238 +238 +239 +239 +239 +239 +24 +24 +24 +24 +241 +241 +242 +242 +242 +242 +244 +244 +247 +247 +248 +248 +249 +249 +252 +252 +255 +255 +255 +255 +256 +256 +256 +256 +257 +257 +258 +258 +26 +26 +26 +26 +260 +260 +262 +262 +263 +263 +265 +265 +265 +265 +266 +266 +27 +27 +272 +272 +272 +272 +273 +273 +273 +273 +273 +273 +274 +274 +275 +275 +277 +277 +277 +277 +277 +277 +277 +277 +278 +278 +278 +278 +28 +28 +280 +280 +280 +280 +281 +281 +281 +281 +282 +282 +282 +282 +283 +283 +284 +284 +285 +285 +286 +286 +287 +287 +288 +288 +288 +288 +289 +289 +291 +291 +292 +292 +296 +296 +298 +298 +298 +298 +298 +298 +30 +30 +302 +302 +305 +305 +306 +306 +307 +307 +307 +307 +308 +308 +309 +309 +309 +309 +310 +310 +311 +311 +311 +311 +311 +311 +315 +315 +316 +316 +316 +316 +316 +316 +317 +317 +317 +317 +318 +318 +318 +318 +318 +318 +321 +321 +321 +321 +322 +322 +322 +322 +323 +323 +325 +325 +325 +325 +327 +327 +327 +327 +327 +327 +33 +33 +331 +331 +331 +331 +332 +332 +333 +333 +333 +333 +335 +335 +336 +336 +338 +338 +339 +339 +34 +34 +341 +341 +342 +342 +342 +342 +344 +344 +344 +344 +345 +345 +348 +348 +348 +348 +348 +348 +348 +348 +348 +348 +35 +35 +35 +35 +35 +35 +351 +351 +353 +353 +353 +353 +356 +356 +360 +360 +362 +362 +364 +364 +365 +365 +366 +366 +367 +367 +367 +367 +368 +368 +369 +369 +369 +369 +369 +369 +37 +37 +37 +37 +373 +373 +374 +374 +375 +375 +377 +377 +378 +378 +379 +379 +382 +382 +382 +382 +384 +384 +384 +384 +384 +384 +386 +386 +389 +389 +392 +392 +393 +393 +394 +394 +395 +395 +395 +395 +396 +396 +396 +396 +396 +396 +397 +397 +397 +397 +399 +399 +399 +399 +4 +4 +400 +400 +401 +401 +401 +401 +401 +401 +401 +401 +401 +401 +402 +402 +403 +403 +403 +403 +403 +403 +404 +404 +404 +404 +406 +406 +406 +406 +406 +406 +406 +406 +407 +407 +409 +409 +409 +409 +409 +409 +41 +41 +411 +411 +413 +413 +413 +413 +414 +414 +414 +414 +417 +417 +417 +417 +417 +417 +418 +418 +419 +419 +42 +42 +42 +42 +421 +421 +424 +424 +424 +424 +427 +427 +429 +429 +429 +429 +43 +43 +430 +430 +430 +430 +430 +430 +431 +431 +431 +431 +431 +431 +432 +432 +435 +435 +436 +436 +437 +437 +438 +438 +438 +438 +438 +438 +439 +439 +439 +439 +44 +44 +443 +443 +444 +444 +446 +446 +448 +448 +449 +449 +452 +452 +453 +453 +454 +454 +454 +454 +454 +454 +455 +455 +457 +457 +458 +458 +458 +458 +459 +459 +459 +459 +460 +460 +462 +462 +462 +462 +463 +463 +463 +463 +466 +466 +466 +466 +466 +466 +467 +467 +468 +468 +468 +468 +468 +468 +468 +468 +469 +469 +469 +469 +469 +469 +469 +469 +469 +469 +47 +47 +470 +470 +472 +472 +475 +475 +477 +477 +478 +478 +478 +478 +479 +479 +480 +480 +480 +480 +480 +480 +481 +481 +482 +482 +483 +483 +484 +484 +485 +485 +487 +487 +489 +489 +489 +489 +489 +489 +489 +489 +490 +490 +491 +491 +492 +492 +492 +492 +493 +493 +494 +494 +495 +495 +496 +496 +497 +497 +498 +498 +498 +498 +498 +498 +5 +5 +5 +5 +5 +5 +51 +51 +51 +51 +53 +53 +54 +54 +57 +57 +58 +58 +58 +58 +64 +64 +65 +65 +66 +66 +67 +67 +67 +67 +69 +69 +70 +70 +70 +70 +70 +70 +72 +72 +72 +72 +74 +74 +76 +76 +76 +76 +77 +77 +78 +78 +8 +8 +80 +80 +82 +82 +83 +83 +83 +83 +84 +84 +84 +84 +85 +85 +86 +86 +87 +87 +9 +9 +90 +90 +90 +90 +90 +90 +92 +92 +95 +95 +95 +95 +96 +96 +97 +97 +97 +97 +98 +98 +98 +98 Index: ql/src/test/results/clientpositive/spark/union_top_level.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/union_top_level.q.out (revision 0) +++ ql/src/test/results/clientpositive/spark/union_top_level.q.out (working copy) @@ -0,0 +1,1012 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- top level +explain +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: QUERY +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- top level +explain +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +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) + Reducer 4 <- Map 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 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 + Filter Operator + predicate: ((key % 3) = 0) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 1) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 2) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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: select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +165 0 +238 1 +255 0 +27 0 +278 2 +311 2 +409 1 +484 1 +86 2 +PREHOOK: query: explain +select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)a +union all +select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)a +union all +select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 8 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 7 <- Reducer 6 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s1 + 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 + Select Operator + expressions: key (type: string), 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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: s1 + 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 + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 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: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: s1 + 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 + Select Operator + expressions: key (type: string), 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: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: s1 + 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 + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 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: 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 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 10 + 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 + Reducer 6 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 7 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 10 + 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: select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)a +union all +select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)b +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)a +union all +select * from (select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### 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 +10 val_10 +10 val_10 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +111 val_111 +111 val_111 +PREHOOK: query: -- ctas +explain +create table union_top as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: -- ctas +explain +create table union_top as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +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 (GROUP, 1) + Reducer 4 <- Map 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 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 + Filter Operator + predicate: ((key % 3) = 0) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 1) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 2) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + + 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 int + 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.union_top + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: create table union_top as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@union_top +POSTHOOK: query: create table union_top as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@union_top +PREHOOK: query: select * from union_top +PREHOOK: type: QUERY +PREHOOK: Input: default@union_top +#### A masked pattern was here #### +POSTHOOK: query: select * from union_top +POSTHOOK: type: QUERY +POSTHOOK: Input: default@union_top +#### A masked pattern was here #### +165 0 +238 1 +255 0 +27 0 +278 2 +311 2 +409 1 +484 1 +86 2 +PREHOOK: query: truncate table union_top +PREHOOK: type: TRUNCATETABLE +PREHOOK: Output: default@union_top +POSTHOOK: query: truncate table union_top +POSTHOOK: type: TRUNCATETABLE +POSTHOOK: Output: default@union_top +PREHOOK: query: -- insert into +explain +insert into table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: QUERY +POSTHOOK: query: -- insert into +explain +insert into table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +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, 1) + Reducer 4 <- Map 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 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 + Filter Operator + predicate: ((key % 3) = 0) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 1) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 2) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + + 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.union_top + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert into table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@union_top +POSTHOOK: query: insert into table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@union_top +POSTHOOK: Lineage: union_top.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: union_top.value EXPRESSION [] +PREHOOK: query: select * from union_top +PREHOOK: type: QUERY +PREHOOK: Input: default@union_top +#### A masked pattern was here #### +POSTHOOK: query: select * from union_top +POSTHOOK: type: QUERY +POSTHOOK: Input: default@union_top +#### A masked pattern was here #### +165 0 +238 1 +255 0 +27 0 +278 2 +311 2 +409 1 +484 1 +86 2 +PREHOOK: query: explain +insert overwrite table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +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, 1) + Reducer 4 <- Map 3 (GROUP, 1) + Reducer 6 <- Map 5 (GROUP, 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 + Filter Operator + predicate: ((key % 3) = 0) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 1) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((key % 3) = 2) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 3 + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: int) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1 + Limit + Number of rows: 3 + 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 + name: default.union_top + + 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.union_top + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: insert overwrite table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@union_top +POSTHOOK: query: insert overwrite table union_top +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@union_top +POSTHOOK: Lineage: union_top.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: union_top.value EXPRESSION [] +PREHOOK: query: select * from union_top +PREHOOK: type: QUERY +PREHOOK: Input: default@union_top +#### A masked pattern was here #### +POSTHOOK: query: select * from union_top +POSTHOOK: type: QUERY +POSTHOOK: Input: default@union_top +#### A masked pattern was here #### +165 0 +238 1 +255 0 +27 0 +278 2 +311 2 +409 1 +484 1 +86 2 +PREHOOK: query: -- create view +explain +create view union_top_view as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: CREATEVIEW +POSTHOOK: query: -- create view +explain +create view union_top_view as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +POSTHOOK: type: CREATEVIEW +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create View Operator: + Create View + or replace: false + columns: key string, value int + expanded text: select `a`.`key`, `a`.`value` from (select `src`.`key`, 0 as `value` from `default`.`src` where `src`.`key` % 3 == 0 limit 3)`a` +union all +select `b`.`key`, `b`.`value` from (select `src`.`key`, 1 as `value` from `default`.`src` where `src`.`key` % 3 == 1 limit 3)`b` +union all +select `c`.`key`, `c`.`value` from (select `src`.`key`, 2 as `value` from `default`.`src` where `src`.`key` % 3 == 2 limit 3)`c` + name: default.union_top_view + original text: select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c + +PREHOOK: query: create view union_top_view as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +PREHOOK: type: CREATEVIEW +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@union_top_view +POSTHOOK: query: create view union_top_view as +select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a +union all +select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b +union all +select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@union_top_view +PREHOOK: query: select * from union_top_view +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@union_top_view +#### A masked pattern was here #### +POSTHOOK: query: select * from union_top_view +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@union_top_view +#### A masked pattern was here #### +165 0 +238 1 +255 0 +27 0 +278 2 +311 2 +409 1 +484 1 +86 2 +PREHOOK: query: drop table union_top +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@union_top +PREHOOK: Output: default@union_top +POSTHOOK: query: drop table union_top +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@union_top +POSTHOOK: Output: default@union_top +PREHOOK: query: drop view union_top_view +PREHOOK: type: DROPVIEW +PREHOOK: Input: default@union_top_view +PREHOOK: Output: default@union_top_view +POSTHOOK: query: drop view union_top_view +POSTHOOK: type: DROPVIEW +POSTHOOK: Input: default@union_top_view +POSTHOOK: Output: default@union_top_view Index: ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out =================================================================== --- ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out (working copy) @@ -6440,9 +6440,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -6891,9 +6891,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: @@ -7325,8 +7325,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: @@ -7747,10 +7747,10 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) - Reducer 5 <- Reducer 4 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) + Reducer 5 <- Reducer 4 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -8241,9 +8241,9 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) - Reducer 4 <- Reducer 3 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Map 1 @@ -8678,8 +8678,8 @@ Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (SORT, 2) - Reducer 3 <- Reducer 2 (SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) Reducer 4 <- Reducer 3 (PARTITION-LEVEL SORT, 2) #### A masked pattern was here #### Vertices: Index: ql/src/test/results/clientpositive/tez/cbo_limit.q.out =================================================================== --- ql/src/test/results/clientpositive/tez/cbo_limit.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/tez/cbo_limit.q.out (working copy) @@ -70,7 +70,7 @@ #### A masked pattern was here #### 1 12 6 1 2 6 -PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc limit 5 +PREHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5 PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 @@ -78,7 +78,7 @@ PREHOOK: Input: default@cbo_t2@dt=2014 PREHOOK: Input: default@cbo_t3 #### A masked pattern was here #### -POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int desc limit 5 +POSTHOOK: query: select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0) group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 limit 5) cbo_t2 on cbo_t1.a=p left outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 0) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c having cbo_t3.c_int > 0 and (c_int >=1 or c >= 1) and (c_int + c) >= 0 order by cbo_t3.c_int % c asc, cbo_t3.c_int, c desc limit 5 POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 Index: ql/src/test/results/clientpositive/tez/cbo_subq_in.q.out =================================================================== --- ql/src/test/results/clientpositive/tez/cbo_subq_in.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/tez/cbo_subq_in.q.out (working copy) @@ -2,7 +2,7 @@ -- non agg, non corr select * from src_cbo -where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key PREHOOK: type: QUERY PREHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -10,7 +10,7 @@ -- non agg, non corr select * from src_cbo -where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') +where src_cbo.key in (select key from src_cbo s1 where s1.key > '9') order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -35,7 +35,7 @@ (select distinct a.key from src_cbo a where b.value = a.value and a.key > '9' - ) + ) order by b.key PREHOOK: type: QUERY PREHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -49,7 +49,7 @@ (select distinct a.key from src_cbo a where b.value = a.value and a.key > '9' - ) + ) order by b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -92,7 +92,7 @@ from src_cbo b where b.key in (select key from src_cbo where src_cbo.key > '8') group by key, value -having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key PREHOOK: type: QUERY PREHOOK: Input: default@src_cbo #### A masked pattern was here #### @@ -106,30 +106,30 @@ from src_cbo b where b.key in (select key from src_cbo where src_cbo.key > '8') group by key, value -having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) +having count(*) in (select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src_cbo #### A masked pattern was here #### 80 val_80 1 -96 val_96 1 -92 val_92 1 -9 val_9 1 -87 val_87 1 -86 val_86 1 -85 val_85 1 82 val_82 1 +83 val_83 2 84 val_84 2 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 3 +92 val_92 1 95 val_95 2 -83 val_83 2 +96 val_96 1 +97 val_97 2 98 val_98 2 -97 val_97 2 -90 val_90 3 PREHOOK: query: -- non agg, non corr, windowing select p_mfgr, p_name, avg(p_size) from part group by p_mfgr, p_name having p_name in - (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr PREHOOK: type: QUERY PREHOOK: Input: default@part #### A masked pattern was here #### @@ -138,12 +138,12 @@ from part group by p_mfgr, p_name having p_name in - (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) order by p_mfgr POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### Manufacturer#1 almond antique burnished rose metallic 2.0 +Manufacturer#2 almond aquamarine midnight light salmon 2.0 Manufacturer#3 almond antique misty red olive 1.0 +Manufacturer#4 almond aquamarine yellow dodger mint 7.0 Manufacturer#5 almond antique sky peru orange 2.0 -Manufacturer#2 almond aquamarine midnight light salmon 2.0 -Manufacturer#4 almond aquamarine yellow dodger mint 7.0 Index: ql/src/test/results/clientpositive/tez/cbo_subq_not_in.q.out =================================================================== --- ql/src/test/results/clientpositive/tez/cbo_subq_not_in.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/tez/cbo_subq_not_in.q.out (working copy) @@ -146,7 +146,7 @@ (select p_name from (select p_mfgr, p_name, p_size as r from part) a where r < 10 and b.p_mfgr = a.p_mfgr - ) + ) order by p_mfgr,p_size PREHOOK: type: QUERY PREHOOK: Input: default@part #### A masked pattern was here #### @@ -157,28 +157,28 @@ (select p_name from (select p_mfgr, p_name, p_size as r from part) a where r < 10 and b.p_mfgr = a.p_mfgr - ) + ) order by p_mfgr,p_size POSTHOOK: type: QUERY POSTHOOK: Input: default@part #### A masked pattern was here #### -Manufacturer#5 almond antique blue firebrick mint 31 -Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#1 almond aquamarine burnished black steel 28 Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond antique violet turquoise frosted 40 Manufacturer#3 almond antique forest lavender goldenrod 14 -Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#3 almond antique chartreuse khaki white 17 Manufacturer#3 almond antique metallic orange dim 19 Manufacturer#3 almond antique olive coral navajo 45 -Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 Manufacturer#4 almond antique violet mint lemon 39 -Manufacturer#2 almond antique violet turquoise frosted 40 -Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#5 almond azure blanched chiffon midnight 23 +Manufacturer#5 almond antique blue firebrick mint 31 Manufacturer#5 almond aquamarine dodger light gainsboro 46 -Manufacturer#4 almond aquamarine floral ivory bisque 27 -Manufacturer#1 almond aquamarine pink moccasin thistle 42 -Manufacturer#2 almond aquamarine rose maroon antique 25 -Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 -Manufacturer#4 almond azure aquamarine papaya violet 12 -Manufacturer#5 almond azure blanched chiffon midnight 23 PREHOOK: query: -- agg, non corr select p_name, p_size from @@ -272,7 +272,7 @@ from lineitem li where li.l_linenumber = 1 and li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') -group by li.l_partkey +group by li.l_partkey order by li.l_partkey PREHOOK: type: QUERY PREHOOK: Input: default@lineitem #### A masked pattern was here #### @@ -281,7 +281,7 @@ from lineitem li where li.l_linenumber = 1 and li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') -group by li.l_partkey +group by li.l_partkey order by li.l_partkey POSTHOOK: type: QUERY POSTHOOK: Input: default@lineitem #### A masked pattern was here #### Index: ql/src/test/results/clientpositive/tez/cbo_udf_udaf.q.out =================================================================== --- ql/src/test/results/clientpositive/tez/cbo_udf_udaf.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/tez/cbo_udf_udaf.q.out (working copy) @@ -11,12 +11,12 @@ POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 20 18 18 1.0 1 1 -PREHOOK: query: select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int +PREHOOK: query: select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int +POSTHOOK: query: select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0 then 1 when 1 then 2 else 3 end, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) from cbo_t1 group by c_int order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 @@ -34,12 +34,12 @@ POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 20 1 18 1.0 1 1 -PREHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 +PREHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 +POSTHOOK: query: select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0 then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0 then 1 when 1 then 2 else 3 end) as h from cbo_t1 group by c_int) cbo_t1 order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 @@ -68,51 +68,51 @@ POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 1 20 1 1 -PREHOOK: query: select count(c_int) as a, avg(c_float), key from cbo_t1 group by key +PREHOOK: query: select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(c_int) as a, avg(c_float), key from cbo_t1 group by key +POSTHOOK: query: select key,count(c_int) as a, avg(c_float) from cbo_t1 group by key order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -0 NULL NULL -2 1.0 1 -2 1.0 1 -12 1.0 1 -2 1.0 1 -PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float +NULL 0 NULL +1 2 1.0 + 1 2 1.0 + 1 2 1.0 +1 12 1.0 +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 0 NULL 1 1.0 -PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_int order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### 0 NULL 1 1.0 -PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int +PREHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a PREHOOK: type: QUERY PREHOOK: Input: default@cbo_t1 PREHOOK: Input: default@cbo_t1@dt=2014 #### A masked pattern was here #### -POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int +POSTHOOK: query: select count(distinct c_int) as a, avg(c_float) from cbo_t1 group by c_float, c_int order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@cbo_t1 POSTHOOK: Input: default@cbo_t1@dt=2014 Index: ql/src/test/results/clientpositive/union12.q.out =================================================================== --- ql/src/test/results/clientpositive/union12.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union12.q.out (working copy) @@ -1,11 +1,15 @@ -PREHOOK: query: -- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink +PREHOOK: query: -- SORT_QUERY_RESULTS +-- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink + create table tmptable(key string, value int) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tmptable -POSTHOOK: query: -- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink +POSTHOOK: query: -- SORT_QUERY_RESULTS +-- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink + create table tmptable(key string, value int) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default Index: ql/src/test/results/clientpositive/union17.q.out =================================================================== --- ql/src/test/results/clientpositive/union17.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union17.q.out (working copy) @@ -14,7 +14,7 @@ POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@DEST2 -PREHOOK: query: -- SORT_BEFORE_DIFF +PREHOOK: query: -- SORT_QUERY_RESULTS -- union case:map-reduce sub-queries followed by multi-table insert explain @@ -24,7 +24,7 @@ INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value PREHOOK: type: QUERY -POSTHOOK: query: -- SORT_BEFORE_DIFF +POSTHOOK: query: -- SORT_QUERY_RESULTS -- union case:map-reduce sub-queries followed by multi-table insert explain Index: ql/src/test/results/clientpositive/union20.q.out =================================================================== --- ql/src/test/results/clientpositive/union20.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union20.q.out (working copy) @@ -1,4 +1,4 @@ -PREHOOK: query: -- SORT_BEFORE_DIFF +PREHOOK: query: -- SORT_QUERY_RESULTS -- union :map-reduce sub-queries followed by join explain @@ -12,7 +12,7 @@ select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 ON (unionsrc1.key = unionsrc2.key) PREHOOK: type: QUERY -POSTHOOK: query: -- SORT_BEFORE_DIFF +POSTHOOK: query: -- SORT_QUERY_RESULTS -- union :map-reduce sub-queries followed by join explain Index: ql/src/test/results/clientpositive/union21.q.out =================================================================== --- ql/src/test/results/clientpositive/union21.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union21.q.out (working copy) @@ -1,4 +1,4 @@ -PREHOOK: query: -- SORT_BEFORE_DIFF +PREHOOK: query: -- SORT_QUERY_RESULTS -- union of constants, udf outputs, and columns from text table and thrift table explain @@ -16,7 +16,7 @@ ) union_output GROUP BY key PREHOOK: type: QUERY -POSTHOOK: query: -- SORT_BEFORE_DIFF +POSTHOOK: query: -- SORT_QUERY_RESULTS -- union of constants, udf outputs, and columns from text table and thrift table explain @@ -202,7 +202,6 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@src_thrift #### A masked pattern was here #### -NULL 2 0 7 001 2 002 2 @@ -728,6 +727,7 @@ 984 4 991 3 993 2 +NULL 2 record_0 1 record_1 1 record_2 1 Index: ql/src/test/results/clientpositive/union27.q.out =================================================================== --- ql/src/test/results/clientpositive/union27.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union27.q.out (working copy) @@ -1,10 +1,10 @@ -PREHOOK: query: -- SORT_BEFORE_DIFF +PREHOOK: query: -- SORT_QUERY_RESULTS create table jackson_sev_same as select * from src PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@jackson_sev_same -POSTHOOK: query: -- SORT_BEFORE_DIFF +POSTHOOK: query: -- SORT_QUERY_RESULTS create table jackson_sev_same as select * from src POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src Index: ql/src/test/results/clientpositive/union_remove_22.q.out =================================================================== --- ql/src/test/results/clientpositive/union_remove_22.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union_remove_22.q.out (working copy) @@ -1,4 +1,6 @@ -PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization -- Union of 2 map-reduce subqueries is performed followed by select and a file sink -- However, some columns are repeated. So, union cannot be removed. -- It does not matter, whether the output is merged or not. In this case, merging is turned @@ -12,7 +14,9 @@ PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@inputTbl1 -POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization -- Union of 2 map-reduce subqueries is performed followed by select and a file sink -- However, some columns are repeated. So, union cannot be removed. -- It does not matter, whether the output is merged or not. In this case, merging is turned @@ -229,15 +233,15 @@ POSTHOOK: Input: default@outputtbl1 #### A masked pattern was here #### 1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 2 2 1 1 1 2 1 1 +2 1 1 3 1 1 +3 1 1 7 1 1 +7 1 1 8 2 2 +8 2 2 PREHOOK: query: explain insert overwrite table outputTbl1 SELECT a.key, concat(a.`values`, a.`values`), concat(a.`values`, a.`values`) Index: ql/src/test/results/clientpositive/union_remove_6_subq.q.out =================================================================== --- ql/src/test/results/clientpositive/union_remove_6_subq.q.out (revision 1674118) +++ ql/src/test/results/clientpositive/union_remove_6_subq.q.out (working copy) @@ -1,4 +1,6 @@ -PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +PREHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization -- Union of 2 subqueries is performed (all of which are mapred queries) -- followed by select star and a file sink in 2 output tables. -- The optimiaztion does not take affect since it is a multi-table insert. @@ -9,7 +11,9 @@ PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@inputTbl1 -POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +POSTHOOK: query: -- SORT_QUERY_RESULTS + +-- This is to test the union->selectstar->filesink optimization -- Union of 2 subqueries is performed (all of which are mapred queries) -- followed by select star and a file sink in 2 output tables. -- The optimiaztion does not take affect since it is a multi-table insert. Index: serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/LazyObjectInspectorFactory.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/LazyObjectInspectorFactory.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/LazyObjectInspectorFactory.java (working copy) @@ -125,7 +125,11 @@ throw new IllegalArgumentException("Illegal ObjectInspector type [" + option + "]"); } - cachedLazySimpleStructObjectInspector.put(signature, result); + LazySimpleStructObjectInspector prev = + cachedLazySimpleStructObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -154,7 +158,11 @@ if (result == null) { result = new LazyListObjectInspector(listElementObjectInspector, separator, lazyParams); - cachedLazySimpleListObjectInspector.put(signature, result); + LazyListObjectInspector prev = + cachedLazySimpleListObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -189,7 +197,11 @@ if (result == null) { result = new LazyMapObjectInspector(mapKeyObjectInspector, mapValueObjectInspector, itemSeparator, keyValueSeparator, lazyParams); - cachedLazySimpleMapObjectInspector.put(signature, result); + LazyMapObjectInspector prev = + cachedLazySimpleMapObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -218,7 +230,11 @@ .get(signature); if (result == null) { result = new LazyUnionObjectInspector(ois, separator, lazyParams); - cachedLazyUnionObjectInspector.put(signature, result); + LazyUnionObjectInspector prev = + cachedLazyUnionObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java (working copy) @@ -19,9 +19,8 @@ package org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; @@ -85,12 +84,12 @@ // Lazy object inspectors for string/char/varchar will all be cached in the same map. // Map key will be list of [typeInfo, isEscaped, escapeChar] - private static HashMap, AbstractPrimitiveLazyObjectInspector> cachedLazyStringTypeOIs = - new HashMap, AbstractPrimitiveLazyObjectInspector>(); + private static ConcurrentHashMap, AbstractPrimitiveLazyObjectInspector> cachedLazyStringTypeOIs = + new ConcurrentHashMap, AbstractPrimitiveLazyObjectInspector>(); - private static Map> + private static ConcurrentHashMap> cachedPrimitiveLazyObjectInspectors = - new HashMap>(); + new ConcurrentHashMap>(); static { cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.BOOLEAN_TYPE_NAME), LAZY_BOOLEAN_OBJECT_INSPECTOR); @@ -177,7 +176,11 @@ "Primitve type " + typeInfo.getPrimitiveCategory() + " should not take parameters"); } - cachedPrimitiveLazyObjectInspectors.put(typeInfo, poi); + AbstractPrimitiveLazyObjectInspector prev = + cachedPrimitiveLazyObjectInspectors.putIfAbsent(typeInfo, poi); + if (prev != null) { + poi = prev; + } return poi; } @@ -190,7 +193,11 @@ .get(signature); if (result == null) { result = new LazyStringObjectInspector(escaped, escapeChar); - cachedLazyStringTypeOIs.put(signature, result); + AbstractPrimitiveLazyObjectInspector prev = + cachedLazyStringTypeOIs.putIfAbsent(signature, result); + if (prev != null) { + result = (LazyStringObjectInspector) prev; + } } return result; } @@ -205,7 +212,11 @@ .get(signature); if (result == null) { result = new LazyHiveCharObjectInspector(typeInfo, escaped, escapeChar); - cachedLazyStringTypeOIs.put(signature, result); + AbstractPrimitiveLazyObjectInspector prev = + cachedLazyStringTypeOIs.putIfAbsent(signature, result); + if (prev != null) { + result = (LazyHiveCharObjectInspector) prev; + } } return result; } @@ -220,7 +231,11 @@ .get(signature); if (result == null) { result = new LazyHiveVarcharObjectInspector(typeInfo, escaped, escapeChar); - cachedLazyStringTypeOIs.put(signature, result); + AbstractPrimitiveLazyObjectInspector prev = + cachedLazyStringTypeOIs.putIfAbsent(signature, result); + if (prev != null) { + result = (LazyHiveVarcharObjectInspector) prev; + } } return result; } @@ -239,7 +254,11 @@ .get(signature); if (result == null) { result = new LazyTimestampObjectInspector(tsFormats); - cachedLazyStringTypeOIs.put(signature, result); + AbstractPrimitiveLazyObjectInspector prev = + cachedLazyStringTypeOIs.putIfAbsent(signature, result); + if (prev != null) { + result = (LazyTimestampObjectInspector) prev; + } } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java (working copy) @@ -18,14 +18,10 @@ package org.apache.hadoop.hive.serde2.lazybinary; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.serde2.ByteStream.Output; import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput; -import org.apache.hadoop.hive.serde2.WriteBuffers; import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.lazybinary.objectinspector.LazyBinaryObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -37,8 +33,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.io.WritableUtils; /** @@ -47,8 +43,6 @@ */ public final class LazyBinaryUtils { - private static Log LOG = LogFactory.getLog(LazyBinaryUtils.class.getName()); - /** * Convert the byte array to an int starting from the given offset. Refer to * code by aeden on DZone Snippets: @@ -433,7 +427,8 @@ byteStream.write((byte) (v)); } - static HashMap cachedLazyBinaryObjectInspector = new HashMap(); + static ConcurrentHashMap cachedLazyBinaryObjectInspector = + new ConcurrentHashMap(); /** * Returns the lazy binary object inspector that can be used to inspect an @@ -504,7 +499,11 @@ result = null; } } - cachedLazyBinaryObjectInspector.put(typeInfo, result); + ObjectInspector prev = + cachedLazyBinaryObjectInspector.putIfAbsent(typeInfo, result); + if (prev != null) { + result = prev; + } } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/objectinspector/LazyBinaryObjectInspectorFactory.java (working copy) @@ -64,7 +64,11 @@ if (result == null) { result = new LazyBinaryStructObjectInspector(structFieldNames, structFieldObjectInspectors, structFieldComments); - cachedLazyBinaryStructObjectInspector.put(signature, result); + LazyBinaryStructObjectInspector prev = + cachedLazyBinaryStructObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -78,7 +82,11 @@ .get(signature); if (result == null) { result = new LazyBinaryUnionObjectInspector(unionFieldObjectInspectors); - cachedLazyBinaryUnionObjectInspector.put(signature, result); + LazyBinaryUnionObjectInspector prev = + cachedLazyBinaryUnionObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -94,7 +102,11 @@ .get(signature); if (result == null) { result = new LazyBinaryListObjectInspector(listElementObjectInspector); - cachedLazyBinaryListObjectInspector.put(signature, result); + LazyBinaryListObjectInspector prev = + cachedLazyBinaryListObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -113,7 +125,11 @@ if (result == null) { result = new LazyBinaryMapObjectInspector(mapKeyObjectInspector, mapValueObjectInspector); - cachedLazyBinaryMapObjectInspector.put(signature, result); + LazyBinaryMapObjectInspector prev = + cachedLazyBinaryMapObjectInspector.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/MetadataListStructObjectInspector.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/MetadataListStructObjectInspector.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/MetadataListStructObjectInspector.java (working copy) @@ -20,8 +20,8 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hive.serde2.ColumnSet; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; @@ -39,8 +39,8 @@ public class MetadataListStructObjectInspector extends StandardStructObjectInspector { - static HashMap>, MetadataListStructObjectInspector> - cached = new HashMap>, MetadataListStructObjectInspector>(); + static ConcurrentHashMap>, MetadataListStructObjectInspector> + cached = new ConcurrentHashMap>, MetadataListStructObjectInspector>(); // public static MetadataListStructObjectInspector getInstance(int fields) { // return getInstance(ObjectInspectorUtils.getIntegerArray(fields)); @@ -52,7 +52,10 @@ MetadataListStructObjectInspector result = cached.get(columnNames); if (result == null) { result = new MetadataListStructObjectInspector(columnNames); - cached.put(key, result); + MetadataListStructObjectInspector prev = cached.putIfAbsent(key, result); + if (prev != null) { + result = prev; + } } return result; } @@ -65,7 +68,10 @@ MetadataListStructObjectInspector result = cached.get(key); if (result == null) { result = new MetadataListStructObjectInspector(columnNames, columnComments); - cached.put(key, result); + MetadataListStructObjectInspector prev = cached.putIfAbsent(key, result); + if (prev != null) { + result = prev; + } } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java (working copy) @@ -19,8 +19,8 @@ package org.apache.hadoop.hive.serde2.objectinspector.primitive; import java.util.EnumMap; -import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.io.ByteWritable; @@ -97,8 +97,8 @@ new WritableHiveDecimalObjectInspector(TypeInfoFactory.decimalTypeInfo); // Map from PrimitiveTypeInfo to AbstractPrimitiveWritableObjectInspector. - private static HashMap cachedPrimitiveWritableInspectorCache = - new HashMap(); + private static ConcurrentHashMap cachedPrimitiveWritableInspectorCache = + new ConcurrentHashMap(); static { cachedPrimitiveWritableInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.BOOLEAN_TYPE_NAME), writableBooleanObjectInspector); @@ -191,8 +191,8 @@ new JavaHiveDecimalObjectInspector(TypeInfoFactory.decimalTypeInfo); // Map from PrimitiveTypeInfo to AbstractPrimitiveJavaObjectInspector. - private static HashMap cachedPrimitiveJavaInspectorCache = - new HashMap(); + private static ConcurrentHashMap cachedPrimitiveJavaInspectorCache = + new ConcurrentHashMap(); static { cachedPrimitiveJavaInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.BOOLEAN_TYPE_NAME), javaBooleanObjectInspector); @@ -292,7 +292,11 @@ throw new RuntimeException("Failed to create object inspector for " + typeInfo ); } - cachedPrimitiveWritableInspectorCache.put(typeInfo, result); + AbstractPrimitiveWritableObjectInspector prev = + cachedPrimitiveWritableInspectorCache.putIfAbsent(typeInfo, result); + if (prev != null) { + result = prev; + } return result; } @@ -391,7 +395,11 @@ throw new RuntimeException("Failed to create JavaHiveVarcharObjectInspector for " + typeInfo ); } - cachedPrimitiveJavaInspectorCache.put(typeInfo, result); + AbstractPrimitiveJavaObjectInspector prev = + cachedPrimitiveJavaInspectorCache.putIfAbsent(typeInfo, result); + if (prev != null) { + result = prev; + } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java (working copy) @@ -22,13 +22,10 @@ import java.util.List; import java.util.concurrent.ConcurrentHashMap; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveVarchar; -import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry; @@ -40,7 +37,6 @@ * objects that represents the same type. */ public final class TypeInfoFactory { - private static Log LOG = LogFactory.getLog(TypeInfoFactory.class); private TypeInfoFactory() { // prevent instantiation @@ -115,7 +111,10 @@ throw new RuntimeException("Error creating PrimitiveTypeInfo instance for " + typeName); } - cachedPrimitiveTypeInfo.put(typeName, result); + PrimitiveTypeInfo prev = cachedPrimitiveTypeInfo.putIfAbsent(typeName, result); + if (prev != null) { + result = prev; + } return result; } @@ -203,7 +202,10 @@ TypeInfo result = cachedStructTypeInfo.get(signature); if (result == null) { result = new StructTypeInfo(names, typeInfos); - cachedStructTypeInfo.put(signature, result); + TypeInfo prev = cachedStructTypeInfo.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } @@ -215,7 +217,10 @@ TypeInfo result = cachedUnionTypeInfo.get(typeInfos); if (result == null) { result = new UnionTypeInfo(typeInfos); - cachedUnionTypeInfo.put(typeInfos, result); + TypeInfo prev = cachedUnionTypeInfo.putIfAbsent(typeInfos, result); + if (prev != null) { + result = prev; + } } return result; } @@ -226,7 +231,10 @@ TypeInfo result = cachedListTypeInfo.get(elementTypeInfo); if (result == null) { result = new ListTypeInfo(elementTypeInfo); - cachedListTypeInfo.put(elementTypeInfo, result); + TypeInfo prev = cachedListTypeInfo.putIfAbsent(elementTypeInfo, result); + if (prev != null) { + result = prev; + } } return result; } @@ -242,7 +250,10 @@ TypeInfo result = cachedMapTypeInfo.get(signature); if (result == null) { result = new MapTypeInfo(keyTypeInfo, valueTypeInfo); - cachedMapTypeInfo.put(signature, result); + TypeInfo prev = cachedMapTypeInfo.putIfAbsent(signature, result); + if (prev != null) { + result = prev; + } } return result; } Index: serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java (revision 1674118) +++ serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java (working copy) @@ -530,7 +530,7 @@ return parser.parsePrimitiveParts(); } - static Map cachedStandardObjectInspector = + static ConcurrentHashMap cachedStandardObjectInspector = new ConcurrentHashMap(); /** @@ -601,12 +601,16 @@ result = null; } } - cachedStandardObjectInspector.put(typeInfo, result); + ObjectInspector prev = + cachedStandardObjectInspector.putIfAbsent(typeInfo, result); + if (prev != null) { + result = prev; + } } return result; } - static Map cachedStandardJavaObjectInspector = + static ConcurrentHashMap cachedStandardJavaObjectInspector = new ConcurrentHashMap(); /** @@ -678,7 +682,11 @@ result = null; } } - cachedStandardJavaObjectInspector.put(typeInfo, result); + ObjectInspector prev = + cachedStandardJavaObjectInspector.putIfAbsent(typeInfo, result); + if (prev != null) { + result = prev; + } } return result; } Index: service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java =================================================================== --- service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (revision 1674118) +++ service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (working copy) @@ -92,6 +92,8 @@ // TCP Server server = new TThreadPoolServer(sargs); + server.setServerEventHandler(serverEventHandler); + server.serve(); String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); Index: service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java =================================================================== --- service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (revision 1674118) +++ service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (working copy) @@ -36,11 +36,25 @@ import org.apache.hive.service.ServiceUtils; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.cli.*; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.FetchType; +import org.apache.hive.service.cli.GetInfoType; +import org.apache.hive.service.cli.GetInfoValue; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.TableSchema; import org.apache.hive.service.cli.session.SessionManager; import org.apache.hive.service.server.HiveServer2; import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.ServerContext; import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TServerEventHandler; +import org.apache.thrift.transport.TTransport; /** * ThriftCLIService. @@ -52,7 +66,6 @@ protected CLIService cliService; private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS); - private static final TStatus ERROR_STATUS = new TStatus(TStatusCode.ERROR_STATUS); protected static HiveAuthFactory hiveAuthFactory; protected int portNum; @@ -70,9 +83,57 @@ protected int maxWorkerThreads; protected long workerKeepAliveTime; - public ThriftCLIService(CLIService cliService, String serviceName) { + protected TServerEventHandler serverEventHandler; + protected ThreadLocal currentServerContext; + + static class ThriftCLIServerContext implements ServerContext { + private SessionHandle sessionHandle = null; + + public void setSessionHandle(SessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public SessionHandle getSessionHandle() { + return sessionHandle; + } + } + + public ThriftCLIService(CLIService service, String serviceName) { super(serviceName); - this.cliService = cliService; + this.cliService = service; + currentServerContext = new ThreadLocal(); + serverEventHandler = new TServerEventHandler() { + @Override + public ServerContext createContext( + TProtocol input, TProtocol output) { + return new ThriftCLIServerContext(); + } + + @Override + public void deleteContext(ServerContext serverContext, + TProtocol input, TProtocol output) { + ThriftCLIServerContext context = (ThriftCLIServerContext)serverContext; + SessionHandle sessionHandle = context.getSessionHandle(); + if (sessionHandle != null) { + LOG.info("Session disconnected without closing properly, close it now"); + try { + cliService.closeSession(sessionHandle); + } catch (HiveSQLException e) { + LOG.warn("Failed to close session: " + e, e); + } + } + } + + @Override + public void preServe() { + } + + @Override + public void processContext(ServerContext serverContext, + TTransport input, TTransport output) { + currentServerContext.set(serverContext); + } + }; } @Override @@ -238,6 +299,11 @@ // TODO: set real configuration map resp.setConfiguration(new HashMap()); resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(sessionHandle); + } } catch (Exception e) { LOG.warn("Error opening session: ", e); resp.setStatus(HiveSQLException.toTStatus(e)); @@ -381,6 +447,11 @@ SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); cliService.closeSession(sessionHandle); resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(null); + } } catch (Exception e) { LOG.warn("Error closing session: ", e); resp.setStatus(HiveSQLException.toTStatus(e)); Index: 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 (revision 1674118) +++ shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java (working copy) @@ -428,6 +428,18 @@ } } + private void configureImpersonation(Configuration conf) { + String user; + try { + user = Utils.getUGI().getShortUserName(); + } catch (Exception e) { + String msg = "Cannot obtain username: " + e; + throw new IllegalStateException(msg, e); + } + conf.set("hadoop.proxyuser." + user + ".groups", "*"); + conf.set("hadoop.proxyuser." + user + ".hosts", "*"); + } + /** * Returns a shim to wrap MiniSparkOnYARNCluster */ @@ -447,9 +459,10 @@ public MiniSparkShim(Configuration conf, int numberOfTaskTrackers, String nameNode, int numDir) throws IOException { - mr = new MiniSparkOnYARNCluster("sparkOnYarn"); conf.set("fs.defaultFS", nameNode); + conf.set("yarn.resourcemanager.scheduler.class", "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler"); + configureImpersonation(conf); mr.init(conf); mr.start(); this.conf = mr.getConfig(); @@ -504,6 +517,7 @@ int numDataNodes, boolean format, String[] racks) throws IOException { + configureImpersonation(conf); MiniDFSCluster miniDFSCluster = new MiniDFSCluster(conf, numDataNodes, format, racks); // Need to set the client's KeyProvider to the NN's for JKS, Index: spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java (revision 1674118) +++ spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java (working copy) @@ -17,6 +17,7 @@ package org.apache.hive.spark.client; +import java.io.File; import java.util.List; import java.util.Map; import java.util.Set; @@ -58,4 +59,9 @@ */ List getAddedJars(); + /** + * Returns a local tmp dir specific to the context + */ + File getLocalTmpDir(); + } Index: spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java (revision 1674118) +++ spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java (working copy) @@ -17,6 +17,7 @@ package org.apache.hive.spark.client; +import java.io.File; import java.util.List; import java.util.Map; import java.util.Set; @@ -34,12 +35,14 @@ private final ThreadLocal monitorCb; private final Map>> monitoredJobs; private final List addedJars; + private final File localTmpDir; - public JobContextImpl(JavaSparkContext sc) { + public JobContextImpl(JavaSparkContext sc, File localTmpDir) { this.sc = sc; this.monitorCb = new ThreadLocal(); monitoredJobs = new ConcurrentHashMap>>(); addedJars = new CopyOnWriteArrayList(); + this.localTmpDir = localTmpDir; } @@ -65,6 +68,11 @@ return addedJars; } + @Override + public File getLocalTmpDir() { + return localTmpDir; + } + void setMonitorCb(MonitorCallback cb) { monitorCb.set(cb); } Index: spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java (revision 1674118) +++ spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java (working copy) @@ -18,9 +18,12 @@ package org.apache.hive.spark.client; import com.google.common.base.Throwables; +import com.google.common.io.Files; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.nio.NioEventLoopGroup; +import java.io.File; +import java.io.IOException; import java.io.Serializable; import java.util.Iterator; import java.util.List; @@ -34,6 +37,7 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hive.spark.client.metrics.Metrics; import org.apache.hive.spark.client.rpc.Rpc; @@ -57,6 +61,8 @@ import org.apache.spark.scheduler.SparkListenerTaskGettingResult; import org.apache.spark.scheduler.SparkListenerTaskStart; import org.apache.spark.scheduler.SparkListenerUnpersistRDD; +import org.apache.spark.scheduler.SparkListenerExecutorRemoved; +import org.apache.spark.scheduler.SparkListenerExecutorAdded; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +89,8 @@ private final NioEventLoopGroup egroup; private final Rpc clientRpc; private final DriverProtocol protocol; + // a local temp dir specific to this driver + private final File localTmpDir; // Used to queue up requests while the SparkContext is being created. private final List> jobQueue = Lists.newLinkedList(); @@ -96,6 +104,7 @@ this.activeJobs = Maps.newConcurrentMap(); this.jcLock = new Object(); this.shutdownLock = new Object(); + localTmpDir = Files.createTempDir(); SparkConf conf = new SparkConf(); String serverAddress = null; @@ -160,11 +169,11 @@ JavaSparkContext sc = new JavaSparkContext(conf); sc.sc().addSparkListener(new ClientListener()); synchronized (jcLock) { - jc = new JobContextImpl(sc); + jc = new JobContextImpl(sc, localTmpDir); jcLock.notifyAll(); } } catch (Exception e) { - LOG.error("Failed to start SparkContext.", e); + LOG.error("Failed to start SparkContext: " + e, e); shutdown(e); synchronized (jcLock) { jcLock.notifyAll(); @@ -186,6 +195,11 @@ } } executor.shutdownNow(); + try { + FileUtils.deleteDirectory(localTmpDir); + } catch (IOException e) { + LOG.warn("Failed to delete local tmp dir: " + localTmpDir, e); + } } private void submit(JobWrapper job) { @@ -201,7 +215,11 @@ private synchronized void shutdown(Throwable error) { if (running) { - LOG.info("Shutting down remote driver."); + if (error == null) { + LOG.info("Shutting down remote driver."); + } else { + LOG.error("Shutting down remote driver due to error: " + error, error); + } running = false; for (JobWrapper job : activeJobs.values()) { cancelJob(job); @@ -425,6 +443,16 @@ private final Map stageToJobId = Maps.newHashMap(); @Override + public void onExecutorRemoved(SparkListenerExecutorRemoved removed) { + + } + + @Override + public void onExecutorAdded(SparkListenerExecutorAdded added) { + + } + + @Override public void onJobStart(SparkListenerJobStart jobStart) { synchronized (stageToJobId) { for (int i = 0; i < jobStart.stageIds().length(); i++) { Index: spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java (revision 1674118) +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java (working copy) @@ -24,12 +24,14 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.io.Resources; import io.netty.channel.ChannelHandlerContext; import io.netty.util.concurrent.GenericFutureListener; import io.netty.util.concurrent.Promise; import java.io.BufferedReader; +import java.io.ByteArrayInputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -39,6 +41,7 @@ import java.io.Serializable; import java.io.Writer; import java.net.URI; +import java.net.URL; import java.util.List; import java.util.Map; import java.util.Properties; @@ -47,6 +50,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.shims.Utils; import org.apache.hive.spark.client.rpc.Rpc; import org.apache.hive.spark.client.rpc.RpcConfiguration; import org.apache.hive.spark.client.rpc.RpcServer; @@ -246,8 +250,21 @@ if (!properties.setReadable(false) || !properties.setReadable(true, true)) { throw new IOException("Cannot change permissions of job properties file."); } + properties.deleteOnExit(); Properties allProps = new Properties(); + // first load the defaults from spark-defaults.conf if available + try { + URL sparkDefaultsUrl = Thread.currentThread().getContextClassLoader().getResource("spark-defaults.conf"); + if (sparkDefaultsUrl != null) { + LOG.info("Loading spark defaults: " + sparkDefaultsUrl); + allProps.load(new ByteArrayInputStream(Resources.toByteArray(sparkDefaultsUrl))); + } + } catch (Exception e) { + String msg = "Exception trying to load spark-defaults.conf: " + e; + throw new IOException(msg, e); + } + // then load the SparkClientImpl config for (Map.Entry e : conf.entrySet()) { allProps.put(e.getKey(), conf.get(e.getKey())); } @@ -350,6 +367,21 @@ } } + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { + try { + String currentUser = Utils.getUGI().getShortUserName(); + // do not do impersonation in CLI mode + if (!currentUser.equals(System.getProperty("user.name"))) { + LOG.info("Attempting impersonation of " + currentUser); + argv.add("--proxy-user"); + argv.add(currentUser); + } + } catch (Exception e) { + String msg = "Cannot obtain username: " + e; + throw new IllegalStateException(msg, e); + } + } + argv.add("--properties-file"); argv.add(properties.getAbsolutePath()); argv.add("--class"); @@ -374,7 +406,7 @@ argv.add(String.format("%s=%s", hiveSparkConfKey, value)); } - LOG.debug("Running client driver with argv: {}", Joiner.on(" ").join(argv)); + LOG.info("Running client driver with argv: {}", Joiner.on(" ").join(argv)); ProcessBuilder pb = new ProcessBuilder(argv.toArray(new String[argv.size()])); if (isTesting != null) { Index: spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java (revision 1674118) +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java (working copy) @@ -28,6 +28,9 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; public class SparkClientUtilities { protected static final transient Log LOG = LogFactory.getLog(SparkClientUtilities.class); @@ -37,20 +40,22 @@ * * @param newPaths Array of classpath elements */ - public static void addToClassPath(String[] newPaths) throws Exception { + public static void addToClassPath(String[] newPaths, Configuration conf, File localTmpDir) + throws Exception { ClassLoader cloader = Thread.currentThread().getContextClassLoader(); URLClassLoader loader = (URLClassLoader) cloader; List curPath = Lists.newArrayList(loader.getURLs()); for (String newPath : newPaths) { - URL newUrl = urlFromPathString(newPath); + URL newUrl = urlFromPathString(newPath, conf, localTmpDir); if (newUrl != null && !curPath.contains(newUrl)) { curPath.add(newUrl); LOG.info("Added jar[" + newUrl + "] to classpath."); } } - URLClassLoader newLoader = new URLClassLoader(curPath.toArray(new URL[curPath.size()]), loader); + URLClassLoader newLoader = + new URLClassLoader(curPath.toArray(new URL[curPath.size()]), loader); Thread.currentThread().setContextClassLoader(newLoader); } @@ -60,16 +65,24 @@ * @param path path string * @return */ - private static URL urlFromPathString(String path) { + private static URL urlFromPathString(String path, Configuration conf, File localTmpDir) { URL url = null; try { if (StringUtils.indexOf(path, "file:/") == 0) { url = new URL(path); + } else if (StringUtils.indexOf(path, "hdfs:/") == 0) { + Path remoteFile = new Path(path); + Path localFile = + new Path(localTmpDir.getAbsolutePath() + File.separator + remoteFile.getName()); + LOG.info("Copying " + remoteFile + " to " + localFile); + FileSystem fs = remoteFile.getFileSystem(conf); + fs.copyToLocalFile(remoteFile, localFile); + return urlFromPathString(localFile.toString(), conf, localTmpDir); } else { url = new File(path).toURL(); } } catch (Exception err) { - LOG.error("Bad URL " + path + ", ignoring path"); + LOG.error("Bad URL " + path + ", ignoring path", err); } return url; } Index: spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java =================================================================== --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java (revision 1674118) +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java (working copy) @@ -35,6 +35,7 @@ import javax.security.sasl.SaslException; import javax.security.sasl.SaslServer; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -126,6 +127,12 @@ */ public Future registerClient(final String clientId, String secret, RpcDispatcher serverDispatcher) { + return registerClient(clientId, secret, serverDispatcher, config.getServerConnectTimeoutMs()); + } + + @VisibleForTesting + Future registerClient(final String clientId, String secret, + RpcDispatcher serverDispatcher, long clientTimeoutMs) { final Promise promise = group.next().newPromise(); Runnable timeout = new Runnable() { @@ -135,7 +142,7 @@ } }; ScheduledFuture timeoutFuture = group.schedule(timeout, - config.getServerConnectTimeoutMs(), + clientTimeoutMs, TimeUnit.MILLISECONDS); final ClientInfo client = new ClientInfo(clientId, promise, secret, serverDispatcher, timeoutFuture); @@ -147,7 +154,7 @@ promise.addListener(new GenericFutureListener>() { @Override public void operationComplete(Promise p) { - if (p.isCancelled()) { + if (!p.isSuccess()) { pendingClients.remove(clientId); } } Index: spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java =================================================================== --- spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java (revision 1674118) +++ spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java (working copy) @@ -23,6 +23,7 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import javax.security.sasl.SaslException; @@ -41,9 +42,7 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; public class TestRpc { @@ -190,6 +189,33 @@ assertEquals(outbound.message, reply.message); } + @Test + public void testClientTimeout() throws Exception { + Map conf = ImmutableMap.builder() + .putAll(emptyConfig) + .build(); + RpcServer server = autoClose(new RpcServer(conf)); + String secret = server.createSecret(); + + try { + autoClose(server.registerClient("client", secret, new TestDispatcher(), 1L).get()); + fail("Server should have timed out client."); + } catch (ExecutionException ee) { + assertTrue(ee.getCause() instanceof TimeoutException); + } + + NioEventLoopGroup eloop = new NioEventLoopGroup(); + Future clientRpcFuture = Rpc.createClient(conf, eloop, + "localhost", server.getPort(), "client", secret, new TestDispatcher()); + try { + autoClose(clientRpcFuture.get()); + fail("Client should have failed to connect to server."); + } catch (ExecutionException ee) { + // Error should not be a timeout. + assertFalse(ee.getCause() instanceof TimeoutException); + } + } + private void transfer(Rpc serverRpc, Rpc clientRpc) { EmbeddedChannel client = (EmbeddedChannel) clientRpc.getChannel(); EmbeddedChannel server = (EmbeddedChannel) serverRpc.getChannel();