diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 443bc3b..c73bd92 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -725,6 +725,11 @@ // Whether to show the unquoted partition names in query results. HIVE_DECODE_PARTITION_NAME("hive.decode.partition.name", false), + + // ptf partition constants + HIVE_PTF_PARTITION_PERSISTENCE_CLASS("hive.ptf.partition.persistence", + "org.apache.hadoop.hive.ql.exec.PTFPersistence$PartitionedByteBasedList"), + HIVE_PTF_PARTITION_PERSISTENT_SIZE("hive.ptf.partition.persistence.memsize", (int) Math.pow(2, (6 + 10 + 10)) ), // 64MB ; public final String varname; diff --git data/files/flights_tiny.txt data/files/flights_tiny.txt new file mode 100644 index 0000000..4841c92 --- /dev/null +++ data/files/flights_tiny.txt @@ -0,0 +1,137 @@ +BaltimoreNew York20101020-30.01064 +BaltimoreNew York2010102023.01142 +BaltimoreNew York201010206.01599 +ChicagoNew York2010102042.0361 +ChicagoNew York2010102024.0897 +ChicagoNew York2010102015.01531 +ChicagoNew York20101020-6.01610 +ChicagoNew York20101020-2.03198 +BaltimoreNew York2010102117.01064 +BaltimoreNew York20101021105.01142 +BaltimoreNew York2010102128.01599 +ChicagoNew York20101021142.0361 +ChicagoNew York2010102177.0897 +ChicagoNew York2010102153.01531 +ChicagoNew York20101021-5.01610 +ChicagoNew York2010102151.03198 +BaltimoreNew York20101022-12.01064 +BaltimoreNew York2010102254.01142 +BaltimoreNew York2010102218.01599 +ChicagoNew York201010222.0361 +ChicagoNew York2010102224.0897 +ChicagoNew York2010102216.01531 +ChicagoNew York20101022-6.01610 +ChicagoNew York20101022-11.03198 +BaltimoreNew York2010102318.0272 +BaltimoreNew York20101023-10.01805 +BaltimoreNew York201010236.03171 +ChicagoNew York201010233.0384 +ChicagoNew York2010102332.0426 +ChicagoNew York201010231.0650 +ChicagoNew York2010102311.03085 +BaltimoreNew York2010102412.01599 +BaltimoreNew York2010102420.02571 +ChicagoNew York2010102410.0361 +ChicagoNew York20101024113.0897 +ChicagoNew York20101024-5.01531 +ChicagoNew York20101024-17.01610 +ChicagoNew York20101024-3.03198 +BaltimoreNew York20101025-25.01064 +BaltimoreNew York2010102592.01142 +BaltimoreNew York20101025106.01599 +ChicagoNew York2010102531.0361 +ChicagoNew York20101025-1.0897 +ChicagoNew York2010102543.01531 +ChicagoNew York201010256.01610 +ChicagoNew York20101025-16.03198 +BaltimoreNew York20101026-22.01064 +BaltimoreNew York20101026123.01142 +BaltimoreNew York2010102690.01599 +ChicagoNew York2010102612.0361 +ChicagoNew York201010260.0897 +ChicagoNew York2010102629.01531 +ChicagoNew York20101026-17.01610 +ChicagoNew York201010266.03198 +BaltimoreNew York20101027-18.01064 +BaltimoreNew York2010102749.01142 +BaltimoreNew York2010102792.01599 +ChicagoNew York20101027148.0361 +ChicagoNew York20101027-11.0897 +ChicagoNew York2010102770.01531 +ChicagoNew York201010278.01610 +ChicagoNew York2010102721.03198 +BaltimoreNew York20101028-4.01064 +BaltimoreNew York20101028-14.01142 +BaltimoreNew York20101028-14.01599 +ChicagoNew York201010282.0361 +ChicagoNew York201010282.0897 +ChicagoNew York20101028-11.01531 +ChicagoNew York201010283.01610 +ChicagoNew York20101028-18.03198 +BaltimoreNew York20101029-24.01064 +BaltimoreNew York2010102921.01142 +BaltimoreNew York20101029-2.01599 +ChicagoNew York20101029-12.0361 +ChicagoNew York20101029-11.0897 +ChicagoNew York2010102915.01531 +ChicagoNew York20101029-18.01610 +ChicagoNew York20101029-4.03198 +BaltimoreNew York2010103014.0272 +BaltimoreNew York20101030-1.01805 +BaltimoreNew York201010305.03171 +ChicagoNew York20101030-6.0384 +ChicagoNew York20101030-10.0426 +ChicagoNew York20101030-5.0650 +ChicagoNew York20101030-5.03085 +BaltimoreNew York20101031-1.01599 +BaltimoreNew York20101031-14.02571 +ChicagoNew York20101031-25.0361 +ChicagoNew York20101031-18.0897 +ChicagoNew York20101031-4.01531 +ChicagoNew York20101031-22.01610 +ChicagoNew York20101031-15.03198 +ClevelandNew York20101030-23.02018 +ClevelandNew York20101030-12.02932 +ClevelandNew York20101029-4.02630 +ClevelandNew York20101029-19.02646 +ClevelandNew York20101029-12.03014 +ClevelandNew York201010283.02630 +ClevelandNew York20101028-6.02646 +ClevelandNew York201010281.03014 +ClevelandNew York2010102716.02630 +ClevelandNew York2010102727.03014 +ClevelandNew York201010264.02630 +ClevelandNew York20101026-27.02646 +ClevelandNew York20101026-11.02662 +ClevelandNew York2010102613.03014 +ClevelandNew York20101025-4.02630 +ClevelandNew York2010102581.02646 +ClevelandNew York2010102542.03014 +ClevelandNew York201010245.02254 +ClevelandNew York20101024-11.02630 +ClevelandNew York20101024-20.02646 +ClevelandNew York20101024-9.03014 +ClevelandNew York20101023-21.02932 +ClevelandNew York201010221.02630 +ClevelandNew York20101022-25.02646 +ClevelandNew York20101022-3.03014 +ClevelandNew York201010213.02630 +ClevelandNew York2010102129.02646 +ClevelandNew York2010102172.03014 +ClevelandNew York20101020-8.02630 +ClevelandNew York20101020-15.03014 +WashingtonNew York20101023-25.05832 +WashingtonNew York20101023-21.05904 +WashingtonNew York20101023-18.05917 +WashingtonNew York20101030-27.05904 +WashingtonNew York20101030-16.05917 +WashingtonNew York20101020-2.07291 +WashingtonNew York2010102122.07291 +WashingtonNew York20101023-16.07274 +WashingtonNew York20101024-26.07282 +WashingtonNew York201010259.07291 +WashingtonNew York201010264.07291 +WashingtonNew York2010102726.07291 +WashingtonNew York2010102845.07291 +WashingtonNew York201010291.07291 +WashingtonNew York20101031-18.07282 diff --git data/files/part.rc data/files/part.rc new file mode 100644 index 0000000..e39e72e Binary files /dev/null and data/files/part.rc differ diff --git data/files/part.seq data/files/part.seq new file mode 100644 index 0000000..b919eb2 Binary files /dev/null and data/files/part.seq differ diff --git data/files/part_tiny.txt data/files/part_tiny.txt new file mode 100644 index 0000000..f6973fd --- /dev/null +++ data/files/part_tiny.txt @@ -0,0 +1,26 @@ +121152almond antique burnished rose metallicManufacturer#1Brand#14PROMO PLATED TIN2JUMBO BOX1173.15e pinto beans h +121152almond antique burnished rose metallicManufacturer#1Brand#14PROMO PLATED TIN2JUMBO BOX1173.15e pinto beans h +85768almond antique chartreuse lavender yellowManufacturer#1Brand#12LARGE BRUSHED STEEL34SM BAG1753.76refull +110592almond antique salmon chartreuse burlywoodManufacturer#1Brand#15PROMO BURNISHED NICKEL6JUMBO PKG1602.59 to the furiously +86428almond aquamarine burnished black steelManufacturer#1Brand#12STANDARD ANODIZED STEEL28WRAP BAG1414.42arefully +65667almond aquamarine pink moccasin thistleManufacturer#1Brand#12LARGE BURNISHED STEEL42JUMBO CASE1632.66e across the expr +105685almond antique violet chocolate turquoiseManufacturer#2Brand#22MEDIUM ANODIZED COPPER14MED CAN1690.68ly pending requ +191709almond antique violet turquoise frostedManufacturer#2Brand#22ECONOMY POLISHED STEEL40MED BOX1800.7 haggle +146985almond aquamarine midnight light salmonManufacturer#2Brand#23MEDIUM BURNISHED COPPER2SM CASE2031.98s cajole caref +132666almond aquamarine rose maroon antiqueManufacturer#2Brand#24SMALL POLISHED NICKEL25MED BOX1698.66even +195606almond aquamarine sandy cyan gainsboroManufacturer#2Brand#25STANDARD PLATED TIN18SM PKG1701.6ic de +90681almond antique chartreuse khaki whiteManufacturer#3Brand#31MEDIUM BURNISHED TIN17SM CASE1671.68are slyly after the sl +17273almond antique forest lavender goldenrodManufacturer#3Brand#35PROMO ANODIZED TIN14JUMBO CASE1190.27along the +112398almond antique metallic orange dimManufacturer#3Brand#32MEDIUM BURNISHED BRASS19JUMBO JAR1410.39ole car +40982almond antique misty red oliveManufacturer#3Brand#32ECONOMY PLATED COPPER1LG PKG1922.98c foxes can s +144293almond antique olive coral navajoManufacturer#3Brand#34STANDARD POLISHED STEEL45JUMBO CAN1337.29ag furiously about +49671almond antique gainsboro frosted violetManufacturer#4Brand#41SMALL BRUSHED BRASS10SM BOX1620.67ccounts run quick +48427almond antique violet mint lemonManufacturer#4Brand#42PROMO POLISHED STEEL39SM CASE1375.42hely ironic i +45261almond aquamarine floral ivory bisqueManufacturer#4Brand#42SMALL PLATED STEEL27WRAP CASE1206.26careful +17927almond aquamarine yellow dodger mintManufacturer#4Brand#41ECONOMY BRUSHED COPPER7SM PKG1844.92ites. eve +33357almond azure aquamarine papaya violetManufacturer#4Brand#41STANDARD ANODIZED TIN12WRAP CASE1290.35reful +192697almond antique blue firebrick mintManufacturer#5Brand#52MEDIUM BURNISHED TIN31LG DRUM1789.69ickly ir +42669almond antique medium spring khakiManufacturer#5Brand#51STANDARD BURNISHED TIN6MED CAN1611.66sits haggl +155733almond antique sky peru orangeManufacturer#5Brand#53SMALL PLATED BRASS2WRAP DRUM1788.73furiously. bra +15103almond aquamarine dodger light gainsboroManufacturer#5Brand#53ECONOMY BURNISHED STEEL46LG PACK1018.1packages hinder carefu +78486almond azure blanched chiffon midnightManufacturer#5Brand#52LARGE BRUSHED BRASS23MED BAG1464.48hely blith diff --git ql/if/queryplan.thrift ql/if/queryplan.thrift index 4427929..304ff08 100644 --- ql/if/queryplan.thrift +++ ql/if/queryplan.thrift @@ -53,6 +53,7 @@ enum OperatorType { LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, + PTF, } struct Operator { diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.cpp ql/src/gen/thrift/gen-cpp/queryplan_types.cpp index 184f8a8..3e2bc86 100644 --- ql/src/gen/thrift/gen-cpp/queryplan_types.cpp +++ ql/src/gen/thrift/gen-cpp/queryplan_types.cpp @@ -48,7 +48,8 @@ int _kOperatorTypeValues[] = { OperatorType::LATERALVIEWJOIN, OperatorType::LATERALVIEWFORWARD, OperatorType::HASHTABLESINK, - OperatorType::HASHTABLEDUMMY + OperatorType::HASHTABLEDUMMY, + OperatorType::PTF }; const char* _kOperatorTypeNames[] = { "JOIN", @@ -68,9 +69,10 @@ const char* _kOperatorTypeNames[] = { "LATERALVIEWJOIN", "LATERALVIEWFORWARD", "HASHTABLESINK", - "HASHTABLEDUMMY" + "HASHTABLEDUMMY", + "PTF" }; -const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(18, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); +const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(19, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); int _kTaskTypeValues[] = { TaskType::MAP, diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.h ql/src/gen/thrift/gen-cpp/queryplan_types.h index ed12cc1..ed4fb46 100644 --- ql/src/gen/thrift/gen-cpp/queryplan_types.h +++ ql/src/gen/thrift/gen-cpp/queryplan_types.h @@ -53,7 +53,8 @@ struct OperatorType { LATERALVIEWJOIN = 14, LATERALVIEWFORWARD = 15, HASHTABLESINK = 16, - HASHTABLEDUMMY = 17 + HASHTABLEDUMMY = 17, + PTF = 18 }; }; diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java index 47079ce..f1c9e2d 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Operator.java @@ -810,7 +810,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Operator struct) th for (int _i25 = 0; _i25 < _map24.size; ++_i25) { String _key26; // required - String _val27; // optional + String _val27; // required _key26 = iprot.readString(); _val27 = iprot.readString(); struct.operatorAttributes.put(_key26, _val27); @@ -830,7 +830,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Operator struct) th for (int _i29 = 0; _i29 < _map28.size; ++_i29) { String _key30; // required - long _val31; // optional + long _val31; // required _key30 = iprot.readString(); _val31 = iprot.readI64(); struct.operatorCounters.put(_key30, _val31); @@ -1003,7 +1003,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Operator struct) thr for (int _i37 = 0; _i37 < _map36.size; ++_i37) { String _key38; // required - String _val39; // optional + String _val39; // required _key38 = iprot.readString(); _val39 = iprot.readString(); struct.operatorAttributes.put(_key38, _val39); @@ -1018,7 +1018,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Operator struct) thr for (int _i41 = 0; _i41 < _map40.size; ++_i41) { String _key42; // required - long _val43; // optional + long _val43; // required _key42 = iprot.readString(); _val43 = iprot.readI64(); struct.operatorCounters.put(_key42, _val43); diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java index 7c4c413..f03af2f 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java @@ -29,7 +29,8 @@ LATERALVIEWJOIN(14), LATERALVIEWFORWARD(15), HASHTABLESINK(16), - HASHTABLEDUMMY(17); + HASHTABLEDUMMY(17), + PTF(18); private final int value; @@ -86,6 +87,8 @@ public static OperatorType findByValue(int value) { return HASHTABLESINK; case 17: return HASHTABLEDUMMY; + case 18: + return PTF; default: return null; } diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java index 0a10e54..e0d77e8 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Query.java @@ -983,7 +983,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Query struct) throw for (int _i101 = 0; _i101 < _map100.size; ++_i101) { String _key102; // required - String _val103; // optional + String _val103; // required _key102 = iprot.readString(); _val103 = iprot.readString(); struct.queryAttributes.put(_key102, _val103); @@ -1003,7 +1003,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Query struct) throw for (int _i105 = 0; _i105 < _map104.size; ++_i105) { String _key106; // required - long _val107; // optional + long _val107; // required _key106 = iprot.readString(); _val107 = iprot.readI64(); struct.queryCounters.put(_key106, _val107); @@ -1239,7 +1239,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Query struct) throws for (int _i118 = 0; _i118 < _map117.size; ++_i118) { String _key119; // required - String _val120; // optional + String _val120; // required _key119 = iprot.readString(); _val120 = iprot.readString(); struct.queryAttributes.put(_key119, _val120); @@ -1254,7 +1254,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Query struct) throws for (int _i122 = 0; _i122 < _map121.size; ++_i122) { String _key123; // required - long _val124; // optional + long _val124; // required _key123 = iprot.readString(); _val124 = iprot.readI64(); struct.queryCounters.put(_key123, _val124); diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java index cf2a52b..c341db2 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Stage.java @@ -911,7 +911,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Stage struct) throw for (int _i73 = 0; _i73 < _map72.size; ++_i73) { String _key74; // required - String _val75; // optional + String _val75; // required _key74 = iprot.readString(); _val75 = iprot.readString(); struct.stageAttributes.put(_key74, _val75); @@ -931,7 +931,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Stage struct) throw for (int _i77 = 0; _i77 < _map76.size; ++_i77) { String _key78; // required - long _val79; // optional + long _val79; // required _key78 = iprot.readString(); _val79 = iprot.readI64(); struct.stageCounters.put(_key78, _val79); @@ -1147,7 +1147,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Stage struct) throws for (int _i90 = 0; _i90 < _map89.size; ++_i90) { String _key91; // required - String _val92; // optional + String _val92; // required _key91 = iprot.readString(); _val92 = iprot.readString(); struct.stageAttributes.put(_key91, _val92); @@ -1162,7 +1162,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Stage struct) throws for (int _i94 = 0; _i94 < _map93.size; ++_i94) { String _key95; // required - long _val96; // optional + long _val96; // required _key95 = iprot.readString(); _val96 = iprot.readI64(); struct.stageCounters.put(_key95, _val96); diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java index f7bfabf..fc4313f 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/Task.java @@ -996,7 +996,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Task struct) throws for (int _i45 = 0; _i45 < _map44.size; ++_i45) { String _key46; // required - String _val47; // optional + String _val47; // required _key46 = iprot.readString(); _val47 = iprot.readString(); struct.taskAttributes.put(_key46, _val47); @@ -1016,7 +1016,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Task struct) throws for (int _i49 = 0; _i49 < _map48.size; ++_i49) { String _key50; // required - long _val51; // optional + long _val51; // required _key50 = iprot.readString(); _val51 = iprot.readI64(); struct.taskCounters.put(_key50, _val51); @@ -1256,7 +1256,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Task struct) throws for (int _i62 = 0; _i62 < _map61.size; ++_i62) { String _key63; // required - String _val64; // optional + String _val64; // required _key63 = iprot.readString(); _val64 = iprot.readString(); struct.taskAttributes.put(_key63, _val64); @@ -1271,7 +1271,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Task struct) throws for (int _i66 = 0; _i66 < _map65.size; ++_i66) { String _key67; // required - long _val68; // optional + long _val68; // required _key67 = iprot.readString(); _val68 = iprot.readI64(); struct.taskCounters.put(_key67, _val68); diff --git ql/src/gen/thrift/gen-php/Types.php ql/src/gen/thrift/gen-php/Types.php index 71b25b7..978dde4 100644 --- ql/src/gen/thrift/gen-php/Types.php +++ ql/src/gen/thrift/gen-php/Types.php @@ -53,6 +53,7 @@ final class OperatorType { const LATERALVIEWFORWARD = 15; const HASHTABLESINK = 16; const HASHTABLEDUMMY = 17; + const PTF = 18; static public $__names = array( 0 => 'JOIN', 1 => 'MAPJOIN', @@ -72,6 +73,7 @@ final class OperatorType { 15 => 'LATERALVIEWFORWARD', 16 => 'HASHTABLESINK', 17 => 'HASHTABLEDUMMY', + 18 => 'PTF', ); } diff --git ql/src/gen/thrift/gen-py/queryplan/ttypes.py ql/src/gen/thrift/gen-py/queryplan/ttypes.py index 0658b61..0a5bab4 100644 --- ql/src/gen/thrift/gen-py/queryplan/ttypes.py +++ ql/src/gen/thrift/gen-py/queryplan/ttypes.py @@ -63,6 +63,7 @@ class OperatorType: LATERALVIEWFORWARD = 15 HASHTABLESINK = 16 HASHTABLEDUMMY = 17 + PTF = 18 _VALUES_TO_NAMES = { 0: "JOIN", @@ -83,6 +84,7 @@ class OperatorType: 15: "LATERALVIEWFORWARD", 16: "HASHTABLESINK", 17: "HASHTABLEDUMMY", + 18: "PTF", } _NAMES_TO_VALUES = { @@ -104,6 +106,7 @@ class OperatorType: "LATERALVIEWFORWARD": 15, "HASHTABLESINK": 16, "HASHTABLEDUMMY": 17, + "PTF": 18, } class TaskType: diff --git ql/src/gen/thrift/gen-rb/queryplan_types.rb ql/src/gen/thrift/gen-rb/queryplan_types.rb index 46d7881..0ec7ed3 100644 --- ql/src/gen/thrift/gen-rb/queryplan_types.rb +++ ql/src/gen/thrift/gen-rb/queryplan_types.rb @@ -39,8 +39,9 @@ module OperatorType LATERALVIEWFORWARD = 15 HASHTABLESINK = 16 HASHTABLEDUMMY = 17 - VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY"} - VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY]).freeze + PTF = 18 + VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF"} + VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF]).freeze end module TaskType diff --git ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java index 83e5042..ff66229 100644 --- ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java +++ ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java @@ -37,6 +37,8 @@ boolean hasOrderBy = false; boolean hasSortBy = false; boolean hasJoinFollowedByGroupBy = false; + boolean hasPTF = false; + boolean hasWindowing = false; // does the query have a using clause boolean usesScript = false; @@ -107,4 +109,20 @@ public boolean hasClusterBy() { public void setHasClusterBy(boolean hasClusterBy) { this.hasClusterBy = hasClusterBy; } + + public boolean hasPTF() { + return hasPTF; + } + + public void setHasPTF(boolean hasPTF) { + this.hasPTF = hasPTF; + } + + public boolean hasWindowing() { + return hasWindowing; + } + + public void setHasWindowing(boolean hasWindowing) { + this.hasWindowing = hasWindowing; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java index f2dc874..e8ee227 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java @@ -135,15 +135,25 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCovariance; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCovarianceSample; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCumeDist; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFDenseRank; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEWAHBitmap; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFFirstValue; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFHistogramNumeric; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFLastValue; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMin; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFNTile; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFParameterInfo; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentRank; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFRank; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFLead; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFLag; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver2; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFRowNumber; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStd; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStdSample; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum; @@ -172,6 +182,8 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInFile; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIndex; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInstr; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag.GenericUDFLag; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag.GenericUDFLead; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLocate; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMap; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMapKeys; @@ -201,8 +213,8 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFStruct; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFTimestamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToBinary; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDecimal; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUtcTimestamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFTranslate; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUnion; @@ -215,6 +227,11 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFParseUrlTuple; import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFStack; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; +import org.apache.hadoop.hive.ql.udf.ptf.NPath.NPathResolver; +import org.apache.hadoop.hive.ql.udf.ptf.Noop.NoopResolver; +import org.apache.hadoop.hive.ql.udf.ptf.NoopWithMap.NoopWithMapResolver; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver; +import org.apache.hadoop.hive.ql.udf.ptf.WindowingTableFunction.WindowingTableFunctionResolver; import org.apache.hadoop.hive.ql.udf.xml.GenericUDFXPath; import org.apache.hadoop.hive.ql.udf.xml.UDFXPathBoolean; import org.apache.hadoop.hive.ql.udf.xml.UDFXPathDouble; @@ -237,6 +254,7 @@ import org.w3c.dom.Element; import org.w3c.dom.NodeList; + /** * FunctionRegistry. */ @@ -248,6 +266,24 @@ * The mapping from expression function names to expression classes. */ static Map mFunctions = Collections.synchronizedMap(new LinkedHashMap()); + + /* + * PTF variables + * */ + + public static final String LEAD_FUNC_NAME = "lead"; + public static final String LAG_FUNC_NAME = "lag"; + + public static final String WINDOWING_TABLE_FUNCTION = "windowingtablefunction"; + public static final String NOOP_TABLE_FUNCTION = "noop"; + public static final String NOOP_MAP_TABLE_FUNCTION = "noopwithmap"; + + static Map tableFunctions = Collections.synchronizedMap(new LinkedHashMap()); + static Map windowFunctions = Collections.synchronizedMap(new LinkedHashMap()); + + public static final ArrayList RANKING_FUNCTIONS = new ArrayList(); + public static final ArrayList NAVIGATION_FUNCTIONS = new ArrayList(); + static { registerUDF("concat", UDFConcat.class, false); registerUDF("substr", UDFSubstr.class, false); @@ -482,6 +518,36 @@ registerGenericUDTF("json_tuple", GenericUDTFJSONTuple.class); registerGenericUDTF("parse_url_tuple", GenericUDTFParseUrlTuple.class); registerGenericUDTF("stack", GenericUDTFStack.class); + + //PTF declarations + registerGenericUDF(true, LEAD_FUNC_NAME, GenericUDFLead.class); + registerGenericUDF(true, LAG_FUNC_NAME, GenericUDFLag.class); + + registerHiveUDAFsAsWindowFunctions(); + registerWindowFunction("row_number", new GenericUDAFRowNumber()); + registerWindowFunction("rank", new GenericUDAFRank()); + registerWindowFunction("dense_rank", new GenericUDAFDenseRank()); + registerWindowFunction("percent_rank", new GenericUDAFPercentRank()); + registerWindowFunction("cume_dist", new GenericUDAFCumeDist()); + registerWindowFunction("ntile", new GenericUDAFNTile()); + registerWindowFunction("first_value", new GenericUDAFFirstValue()); + registerWindowFunction("last_value", new GenericUDAFLastValue()); + registerWindowFunction(LEAD_FUNC_NAME, new GenericUDAFLead(), false); + registerWindowFunction(LAG_FUNC_NAME, new GenericUDAFLag(), false); + + RANKING_FUNCTIONS.add("rank"); + RANKING_FUNCTIONS.add("dense_rank"); + RANKING_FUNCTIONS.add("percent_rank"); + + NAVIGATION_FUNCTIONS.add(LEAD_FUNC_NAME); + NAVIGATION_FUNCTIONS.add(LAG_FUNC_NAME); + NAVIGATION_FUNCTIONS.add("first_value"); + NAVIGATION_FUNCTIONS.add("last_value"); + + registerTableFunction(NOOP_TABLE_FUNCTION, NoopResolver.class); + registerTableFunction(NOOP_MAP_TABLE_FUNCTION, NoopWithMapResolver.class); + registerTableFunction(WINDOWING_TABLE_FUNCTION, WindowingTableFunctionResolver.class); + registerTableFunction("npath", NPathResolver.class); } public static void registerTemporaryUDF(String functionName, @@ -795,6 +861,26 @@ public static GenericUDAFEvaluator getGenericUDAFEvaluator(String name, return udafEvaluator; } + @SuppressWarnings("deprecation") + public static GenericUDAFEvaluator getGenericWindowingEvaluator(String name, + List argumentOIs, boolean isDistinct, + boolean isAllColumns) throws SemanticException { + + WindowFunctionInfo finfo = windowFunctions.get(name.toLowerCase()); + if (finfo == null) { return null;} + if ( !name.toLowerCase().equals(LEAD_FUNC_NAME) && + !name.toLowerCase().equals(LAG_FUNC_NAME) ) { + return getGenericUDAFEvaluator(name, argumentOIs, isDistinct, isAllColumns); + } + + // this must be lead/lag UDAF + ObjectInspector args[] = new ObjectInspector[argumentOIs.size()]; + GenericUDAFResolver udafResolver = finfo.getfInfo().getGenericUDAFResolver(); + GenericUDAFParameterInfo paramInfo = new SimpleGenericUDAFParameterInfo( + argumentOIs.toArray(args), isDistinct, isAllColumns); + return ((GenericUDAFResolver2) udafResolver).getEvaluator(paramInfo); + } + /** * This method is shared between UDFRegistry and UDAFRegistry. methodName will * be "evaluate" for UDFRegistry, and "aggregate"/"evaluate"/"evaluatePartial" @@ -1033,36 +1119,36 @@ public static Method getMethodInternal(Class udfClass, List mlist, bo } if (udfMethods.size() > 1) { - // if the only difference is numeric types, pick the method + // if the only difference is numeric types, pick the method // with the smallest overall numeric type. int lowestNumericType = Integer.MAX_VALUE; boolean multiple = true; Method candidate = null; List referenceArguments = null; - + for (Method m: udfMethods) { int maxNumericType = 0; - + List argumentsAccepted = TypeInfoUtils.getParameterTypeInfos(m, argumentsPassed.size()); - + if (referenceArguments == null) { - // keep the arguments for reference - we want all the non-numeric + // keep the arguments for reference - we want all the non-numeric // arguments to be the same referenceArguments = argumentsAccepted; } - + Iterator referenceIterator = referenceArguments.iterator(); - + for (TypeInfo accepted: argumentsAccepted) { TypeInfo reference = referenceIterator.next(); - + if (numericTypes.containsKey(accepted)) { // We're looking for the udf with the smallest maximum numeric type. int typeValue = numericTypes.get(accepted); maxNumericType = typeValue > maxNumericType ? typeValue : maxNumericType; } else if (!accepted.equals(reference)) { // There are non-numeric arguments that don't match from one UDF to - // another. We give up at this point. + // another. We give up at this point. throw new AmbiguousMethodException(udfClass, argumentsPassed, mlist); } } @@ -1351,4 +1437,95 @@ public static void registerFunctionsFromPluginJar( private FunctionRegistry() { // prevent instantiation } + + + //---------PTF functions------------ + + public static void registerWindowFunction(String name, GenericUDAFResolver wFn) + { + registerWindowFunction(name, wFn, true); + } + + /** + * Typically a WindowFunction is the same as a UDAF. The only exceptions are Lead & Lag UDAFs. These + * are not registered as regular UDAFs because + * - we plan to support Lead & Lag as UDFs (usable only within argument expressions + * of UDAFs when windowing is involved). Since mFunctions holds both UDFs and UDAFs we cannot + * add both FunctionInfos to mFunctions. + * We choose to only register UDFs in mFunctions. The implication of this is that Lead/Lag UDAFs + * are only usable when windowing is involved. + * + * @param name + * @param wFn + * @param registerAsUDAF + */ + public static void registerWindowFunction(String name, GenericUDAFResolver wFn, boolean registerAsUDAF) + { + FunctionInfo fInfo = null; + if (registerAsUDAF) { + registerGenericUDAF(true, name, wFn); + fInfo = getFunctionInfo(name); + } + else { + fInfo = new FunctionInfo(true, + name.toLowerCase(), wFn); + } + + WindowFunctionInfo wInfo = new WindowFunctionInfo(fInfo); + windowFunctions.put(name.toLowerCase(), wInfo); + } + + public static boolean isWindowFunction(String name) + { + WindowFunctionInfo wFInfo = windowFunctions.get(name.toLowerCase()); + return wFInfo != null; + } + + public static WindowFunctionInfo getWindowFunctionInfo(String name) + { + return windowFunctions.get(name.toLowerCase()); + } + + static void registerHiveUDAFsAsWindowFunctions() + { + Set fNames = getFunctionNames(); + for(String fName : fNames) + { + FunctionInfo fInfo = getFunctionInfo(fName); + if ( fInfo.isGenericUDAF()) + { + WindowFunctionInfo wInfo = new WindowFunctionInfo(fInfo); + windowFunctions.put(fName, wInfo); + } + } + } + + public static boolean isTableFunction(String name) + { + PTFFunctionInfo tFInfo = tableFunctions.get(name.toLowerCase()); + return tFInfo != null && !tFInfo.isInternal(); + } + + public static TableFunctionResolver getTableFunctionResolver(String name) + { + PTFFunctionInfo tfInfo = tableFunctions.get(name.toLowerCase()); + return (TableFunctionResolver) ReflectionUtils.newInstance(tfInfo.getFunctionResolver(), null); + } + + public static TableFunctionResolver getWindowingTableFunction() + { + return getTableFunctionResolver(WINDOWING_TABLE_FUNCTION); + } + + public static TableFunctionResolver getNoopTableFunction() + { + return getTableFunctionResolver(NOOP_TABLE_FUNCTION); + } + + public static void registerTableFunction(String name, Class tFnCls) + { + PTFFunctionInfo tInfo = new PTFFunctionInfo(name, tFnCls); + tableFunctions.put(name.toLowerCase(), tInfo); + } + } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java index 69fff0e..8f52456 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.plan.LimitDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; import org.apache.hadoop.hive.ql.plan.ScriptDesc; @@ -74,6 +75,7 @@ public OpTuple(Class descClass, Class> opClass) { opvec.add(new OpTuple(FileSinkDesc.class, FileSinkOperator.class)); opvec.add(new OpTuple(CollectDesc.class, CollectOperator.class)); opvec.add(new OpTuple(ScriptDesc.class, ScriptOperator.class)); + opvec.add(new OpTuple(PTFDesc.class, PTFOperator.class)); opvec.add(new OpTuple(ReduceSinkDesc.class, ReduceSinkOperator.class)); opvec.add(new OpTuple(ExtractDesc.class, ExtractOperator.class)); opvec.add(new OpTuple(GroupByDesc.class, GroupByOperator.class)); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/PTFFunctionInfo.java ql/src/java/org/apache/hadoop/hive/ql/exec/PTFFunctionInfo.java new file mode 100644 index 0000000..884e723 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/PTFFunctionInfo.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver; + +class PTFFunctionInfo +{ + String displayName; + Class functionResolver; + boolean isInternal; + + public PTFFunctionInfo(String displayName, Class tFnCls) + { + super(); + this.displayName = displayName; + this.functionResolver = tFnCls; + isInternal = false; + PartitionTableFunctionDescription def = functionResolver.getAnnotation(PartitionTableFunctionDescription.class); + if ( def != null) + { + isInternal = def.isInternal(); + } + } + + public String getDisplayName() + { + return displayName; + } + + public Class getFunctionResolver() + { + return functionResolver; + } + + public boolean isInternal() + { + return isInternal; + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java new file mode 100644 index 0000000..d91a89d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java @@ -0,0 +1,453 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Stack; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowTableFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDeserializer; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionEvaluator; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; + +public class PTFOperator extends Operator implements Serializable +{ + + private static final long serialVersionUID = 1L; + PTFPartition inputPart; + boolean isMapOperator; + + transient KeyWrapperFactory keyWrapperFactory; + protected transient KeyWrapper currentKeys; + protected transient KeyWrapper newKeys; + transient HiveConf hiveConf; + + /* + * 1. Find out if the operator is invoked at Map-Side or Reduce-side + * 2. Get the deserialized QueryDef + * 3. Reconstruct the transient variables in QueryDef + * 4. Create input partition to store rows coming from previous operator + */ + @Override + protected void initializeOp(Configuration jobConf) throws HiveException + { + hiveConf = new HiveConf(jobConf, PTFOperator.class); + // if the parent is ExtractOperator, this invocation is from reduce-side + Operator parentOp = getParentOperators().get(0); + if (parentOp instanceof ExtractOperator) + { + isMapOperator = false; + } + else + { + isMapOperator = true; + } + + reconstructQueryDef(hiveConf); + inputPart = createFirstPartitionForChain( + inputObjInspectors[0], hiveConf, isMapOperator); + + if (isMapOperator) + { + PartitionedTableFunctionDef tDef = conf.getStartOfChain(); + outputObjInspector = tDef.getRawInputShape().getOI(); + } + else + { + outputObjInspector = conf.getFuncDef().getOutputShape().getOI(); + } + + setupKeysWrapper(inputObjInspectors[0]); + + super.initializeOp(jobConf); + } + + @Override + protected void closeOp(boolean abort) throws HiveException + { + super.closeOp(abort); + if(inputPart.size() != 0){ + if (isMapOperator) + { + processMapFunction(); + } + else + { + processInputPartition(); + } + } + } + + @Override + public void processOp(Object row, int tag) throws HiveException + { + if (!isMapOperator ) + { + /* + * checkif current row belongs to the current accumulated Partition: + * - If not: + * - process the current Partition + * - reset input Partition + * - set currentKey to the newKey if it is null or has changed. + */ + newKeys.getNewKey(row, inputPart.getOI()); + boolean keysAreEqual = (currentKeys != null && newKeys != null)? + newKeys.equals(currentKeys) : false; + + if (currentKeys != null && !keysAreEqual) + { + processInputPartition(); + inputPart = createFirstPartitionForChain(inputObjInspectors[0], hiveConf, isMapOperator); + } + + if (currentKeys == null || !keysAreEqual) + { + if (currentKeys == null) + { + currentKeys = newKeys.copyKey(); + } + else + { + currentKeys.copyKey(newKeys); + } + } + } + + // add row to current Partition. + inputPart.append(row); + } + + /** + * Initialize the visitor to use the QueryDefDeserializer Use the order + * defined in QueryDefWalker to visit the QueryDef + * + * @param hiveConf + * @throws HiveException + */ + protected void reconstructQueryDef(HiveConf hiveConf) throws HiveException + { + + PTFDeserializer dS = + new PTFDeserializer(conf, (StructObjectInspector)inputObjInspectors[0], hiveConf); + dS.initializePTFChain(conf.getFuncDef()); + } + + protected void setupKeysWrapper(ObjectInspector inputOI) throws HiveException + { + PartitionDef pDef = conf.getStartOfChain().getPartition(); + ArrayList exprs = pDef.getExpressions(); + int numExprs = exprs.size(); + ExprNodeEvaluator[] keyFields = new ExprNodeEvaluator[numExprs]; + ObjectInspector[] keyOIs = new ObjectInspector[numExprs]; + ObjectInspector[] currentKeyOIs = new ObjectInspector[numExprs]; + + for(int i=0; i pItr = outPart.iterator(); + while (pItr.hasNext()) + { + Object oRow = pItr.next(); + forward(oRow, outputObjInspector); + } + } + } + + protected void processMapFunction() throws HiveException + { + PartitionedTableFunctionDef tDef = conf.getStartOfChain(); + PTFPartition outPart = tDef.getTFunction().transformRawInput(inputPart); + PTFPartitionIterator pItr = outPart.iterator(); + while (pItr.hasNext()) + { + Object oRow = pItr.next(); + forward(oRow, outputObjInspector); + } + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return getOperatorName(); + } + + static public String getOperatorName() { + return "PTF"; + } + + + @Override + public OperatorType getType() + { + return OperatorType.PTF; + } + + /** + * For all the table functions to be applied to the input + * hive table or query, push them on a stack. + * For each table function popped out of the stack, + * execute the function on the input partition + * and return an output partition. + * @param part + * @return + * @throws HiveException + */ + private PTFPartition executeChain(PTFPartition part) + throws HiveException + { + Stack fnDefs = new Stack(); + PTFInputDef iDef = conf.getFuncDef(); + while (true) + { + if (iDef instanceof PartitionedTableFunctionDef) + { + fnDefs.push((PartitionedTableFunctionDef) iDef); + iDef = ((PartitionedTableFunctionDef) iDef).getInput(); + } + else + { + break; + } + } + + PartitionedTableFunctionDef currFnDef; + while (!fnDefs.isEmpty()) + { + currFnDef = fnDefs.pop(); + part = currFnDef.getTFunction().execute(part); + } + return part; + } + + /** + * If WindowingSpec contains any Windowing Expressions or has a + * Having condition then these are processed + * and forwarded on. Whereas when there is no having or WdwExprs + * just forward the rows in the output Partition. + * + * For e.g. Consider the following query: + *
+   * {@code
+   *  select rank(), lead(rank(),1),...
+   *  from xyz
+   *  ...
+   *  having rank() > 1
+   *  }
+   * 
+ * rank() gets processed as a WdwFn; Its in the oPart(output partition) + * argument to executeWindowExprs. Here we first evaluate the having expression. + * So the first row of oPart gets filtered out. + * Next we evaluate lead(rank()) which is held as a WindowExpression and add it to the output. + * + * @param ptfDesc + * @param oPart output partition after Window Fns are processed. + * @param op + * @throws HiveException + */ + private void executeWindowExprs(PTFPartition oPart) + throws HiveException + { + WindowTableFunctionDef wTFnDef = (WindowTableFunctionDef) conf.getFuncDef(); + /* + * inputOI represents the row with WindowFn results present. + * So in the e.g. above it will have a column for 'rank()' + */ + StructObjectInspector inputOI = wTFnDef.getOutputFromWdwFnProcessing().getOI(); + /* + * outputOI represents the final row with the Windowing Expressions added. + * So in the e.g. above it will have a column for 'lead(rank())' in addition to + * all columns in inputOI. + */ + StructObjectInspector outputOI = wTFnDef.getOutputShape().getOI(); + int numCols = outputOI.getAllStructFieldRefs().size(); + ArrayList wdwExprs = wTFnDef.getWindowExpressions(); + int numWdwExprs = wdwExprs == null ? 0 : wdwExprs.size(); + Object[] output = new Object[numCols]; + + PTFExpressionDef havingExpr = wTFnDef.getHavingExpression(); + boolean applyHaving = havingExpr != null; + Converter hvgConverter = !applyHaving ? null + : ObjectInspectorConverters + .getConverter( + havingExpr.getOI(), + PrimitiveObjectInspectorFactory.javaBooleanObjectInspector); + ExprNodeEvaluator havingCondEval = !applyHaving ? null : havingExpr.getExprEvaluator(); + /* + * If this Windowing invocation has no Window Expressions and doesn't need to be filtered, + * we can just forward the row in the oPart partition. + */ + boolean forwardRowsUntouched = !applyHaving && (wdwExprs == null || wdwExprs.size() == 0 ); + + PTFPartitionIterator pItr = oPart.iterator(); + PTFOperator.connectLeadLagFunctionsToPartition(conf, pItr); + while (pItr.hasNext()) + { + int colCnt = 0; + Object oRow = pItr.next(); + + /* + * when there is no Windowing expressions or having; + * just forward the Object coming out of the Partition. + */ + if ( forwardRowsUntouched ) { + forward(oRow, outputObjInspector); + continue; + } + + if (applyHaving) + { + Object hvgCond = null; + hvgCond = havingCondEval.evaluate(oRow); + hvgCond = hvgConverter.convert(hvgCond); + if (!((Boolean) hvgCond).booleanValue()) + { + continue; + } + } + + /* + * Setup the output row columns in the following order + * - the columns in the SelectList processed by the PTF + * (ie the Select Exprs that have navigation expressions) + * - the columns from the final PTF. + */ + + if ( wdwExprs != null ) { + for (WindowExpressionDef wdwExpr : wdwExprs) + { + Object newCol = wdwExpr.getExprEvaluator().evaluate(oRow); + output[colCnt++] = newCol; + } + } + + for(; colCnt < numCols; ) { + StructField field = inputOI.getAllStructFieldRefs().get(colCnt - numWdwExprs); + output[colCnt++] = + ObjectInspectorUtils.copyToStandardObject(inputOI.getStructFieldData(oRow, field), + field.getFieldObjectInspector()); + } + + forward(output, outputObjInspector); + } + } + + /** + * Create a new Partition. + * A partition has 2 OIs: the OI for the rows being put in and the OI for the rows + * coming out. You specify the output OI by giving the Serde to use to Serialize. + * Typically these 2 OIs are the same; but not always. For the + * first PTF in a chain the OI of the incoming rows is dictated by the Parent Op + * to this PTFOp. The output OI from the Partition is typically LazyBinaryStruct, but + * not always. In the case of Noop/NoopMap we keep the Strcuture the same as + * what is given to us. + *

+ * The Partition we want to create here is for feeding the First table function in the chain. + * So for map-side processing use the Serde from the output Shape its InputDef. + * For reduce-side processing use the Serde from its RawInputShape(the shape + * after map-side processing). + * @param oi + * @param hiveConf + * @param isMapSide + * @return + * @throws HiveException + */ + public PTFPartition createFirstPartitionForChain(ObjectInspector oi, + HiveConf hiveConf, boolean isMapSide) throws HiveException + { + PartitionedTableFunctionDef tabDef = conf.getStartOfChain(); + TableFunctionEvaluator tEval = tabDef.getTFunction(); + String partClassName = tEval.getPartitionClass(); + int partMemSize = tEval.getPartitionMemSize(); + + PTFPartition part = null; + SerDe serde = isMapSide ? tabDef.getInput().getOutputShape().getSerde() : + tabDef.getRawInputShape().getSerde(); + part = new PTFPartition(partClassName, partMemSize, serde, + (StructObjectInspector) oi); + return part; + + } + + public static void connectLeadLagFunctionsToPartition(PTFDesc ptfDesc, + PTFPartitionIterator pItr) throws HiveException + { + List llFnDescs = ptfDesc.getLlInfo().getLeadLagExprs(); + if (llFnDescs == null) { + return; + } + for (ExprNodeGenericFuncDesc llFnDesc : llFnDescs) + { + GenericUDFLeadLag llFn = (GenericUDFLeadLag) llFnDesc + .getGenericUDF(); + llFn.setpItr(pItr); + } + } + + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java new file mode 100644 index 0000000..61c45fa --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java @@ -0,0 +1,280 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.util.ConcurrentModificationException; +import java.util.Iterator; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.PTFPersistence.ByteBasedList; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.Writable; + +/* + * represents a collection of rows that is acted upon by a TableFunction or a WindowFunction. + */ +public class PTFPartition +{ + SerDe serDe; + StructObjectInspector OI; + private ByteBasedList elems; + private Writable wRow; + private int sz; + + public PTFPartition(HiveConf cfg, SerDe serDe, StructObjectInspector oI) throws HiveException + { + String partitionClass = HiveConf.getVar(cfg, ConfVars.HIVE_PTF_PARTITION_PERSISTENCE_CLASS); + int partitionMemSize = HiveConf.getIntVar(cfg, ConfVars.HIVE_PTF_PARTITION_PERSISTENT_SIZE); + init(partitionClass, partitionMemSize, serDe, oI); + } + + public PTFPartition(String partitionClass, int partitionMemSize, SerDe serDe, StructObjectInspector oI) throws HiveException + { + init(partitionClass, partitionMemSize, serDe, oI); + } + + private void init(String partitionClass, int partitionMemSize, SerDe serDe, StructObjectInspector oI) throws HiveException + { + this.serDe = serDe; + OI = oI; + elems = PTFPersistence.createList(partitionClass, partitionMemSize); + sz = 0; + wRow = createWritable(); + } + + public SerDe getSerDe() + { + return serDe; + } + public void setSerDe(SerDe serDe) + { + this.serDe = serDe; + } + public StructObjectInspector getOI() + { + return OI; + } + public void setOI(StructObjectInspector oI) + { + OI = oI; + } + + private Writable createWritable() throws HiveException + { + try + { + return serDe.getSerializedClass().newInstance(); + } + catch(Throwable t) + { + throw new HiveException(t); + } + } + + public Object getAt(int i) throws HiveException + { + try + { + elems.get(i, wRow); + Object o = serDe.deserialize(wRow); + return o; + } + catch(SerDeException se) + { + throw new HiveException(se); + } + } + + public Object getWritableAt(int i) throws HiveException + { + elems.get(i, wRow); + return wRow; + } + + public void append(Writable o) throws HiveException + { + elems.append(o); + sz++; + } + + public void append(Object o) throws HiveException + { + try + { + append(serDe.serialize(o, OI)); + } + catch(SerDeException e) + { + throw new HiveException(e); + } + } + + public int size() + { + return sz; + } + + public PTFPartitionIterator iterator() + { + return new PItr(0, size()); + } + + public PTFPartitionIterator range(int start, int end) + { + assert(start >= 0); + assert(end <= size()); + assert(start <= end); + return new PItr(start, end); + } + + class PItr implements PTFPartitionIterator + { + int idx; + final int start; + final int end; + final int createTimeSz; + + PItr(int start, int end) + { + this.idx = start; + this.start = start; + this.end = end; + createTimeSz = PTFPartition.this.size(); + } + + public boolean hasNext() + { + checkForComodification() ; + return idx < end; + } + + public Object next() + { + checkForComodification(); + try + { + return PTFPartition.this.getAt(idx++); + } + catch(HiveException e) + { + throw new RuntimeException(e); + } + } + + public void remove() + { + throw new UnsupportedOperationException(); + } + + final void checkForComodification() + { + if (createTimeSz != PTFPartition.this.size()) { + throw new ConcurrentModificationException(); + } + } + + @Override + public int getIndex() + { + return idx; + } + + private Object getAt(int i) + { + try + { + return PTFPartition.this.getAt(i); + } + catch(HiveException e) + { + throw new RuntimeException(e); + } + } + + @Override + public Object lead(int amt) + { + int i = idx + amt; + i = i >= end ? end - 1 : i; + return getAt(i); + } + + @Override + public Object lag(int amt) + { + int i = idx - amt; + i = i < start ? start : i; + return getAt(i); + } + + @Override + public Object resetToIndex(int idx) + { + if ( idx < start || idx >= end ) + { + return null; + } + Object o = getAt(idx); + this.idx = idx + 1; + return o; + } + + @Override + public PTFPartition getPartition() + { + return PTFPartition.this; + } + + @Override + public void reset() + { + idx = start; + } + }; + + /* + * provide an Iterator on the rows in a Partiton. + * Iterator exposes the index of the next location. + * Client can invoke lead/lag relative to the next location. + */ + public static interface PTFPartitionIterator extends Iterator + { + int getIndex(); + + T lead(int amt); + + T lag(int amt); + + /* + * after a lead and lag call, allow Object associated with SerDe and writable associated with partition to be reset + * to the value for the current Index. + */ + Object resetToIndex(int idx); + + PTFPartition getPartition(); + + void reset(); + } + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPersistence.java ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPersistence.java new file mode 100644 index 0000000..e91d803 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPersistence.java @@ -0,0 +1,1072 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.lang.ref.SoftReference; +import java.lang.reflect.Constructor; +import java.nio.ByteBuffer; +import java.nio.IntBuffer; +import java.nio.channels.FileChannel; +import java.util.ArrayList; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.Serializer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.Writable; + +/* + * contains all the classes to support persisting a PTF partition, + */ +public class PTFPersistence { + + @SuppressWarnings("unchecked") + public static ByteBasedList createList(String clsName, int capacity) throws HiveException + { + try + { + Class cls = (Class) Class.forName(clsName); + Constructor cons = cls.getConstructor(Integer.TYPE); + return cons.newInstance(capacity); + } + catch(Exception e) + { + throw new HiveException(e); + } + } + + public static class ByteBasedList + { + int startOffset; + + /* + * (offset,size) of Writables. + * entry i at position i << 1 + * this array is resizable. + */ + int[] offsetsArray; + + /* + * contains actual bytes of Writables. + * not resizable + */ + byte[] bytes; + int bytesUsed; + + int currentSize; + ReentrantReadWriteLock lock; + volatile long lastModified; + + + public ByteBasedList(int startOffset, int capacity) + { + this.startOffset = startOffset; + bytes = new byte[capacity]; + offsetsArray = new int[INCREMENT_SIZE]; + bytesUsed = 0; + currentSize = 0; + lock = new ReentrantReadWriteLock(); + lastModified = System.nanoTime(); + } + + public ByteBasedList() + { + this(0, MEDIUM_SIZE); + } + + public ByteBasedList(int capacity) + { + this(0, capacity); + } + + /* + * internal api; used by {@link PersistentByteBasedList} to setup BBList from a file. + */ + protected ByteBasedList(File file) + { + lock = new ReentrantReadWriteLock(); + } + + private void ensureCapacity(int wlen) throws ListFullException + { + if ( bytesUsed + wlen > bytes.length) + { + throw new ListFullException(); + } + + if ( (2 * currentSize + 1) > offsetsArray.length ) + { + int[] na = new int[offsetsArray.length + INCREMENT_SIZE]; + System.arraycopy(offsetsArray, 0, na, 0, offsetsArray.length); + offsetsArray = na; + } + } + + private int index(int i) throws HiveException + { + int j = i - startOffset; + j = j << 1; + if ( j > 2 * currentSize ) + { + throw new HiveException(String.format("index invalid %d", i)); + } + return j; + } + + private void write(Writable w) throws HiveException, IOException + { + DataOStream dos = PTFPersistence.dos.get(); + ByteArrayOS bos = dos.getUnderlyingStream(); + bos.reset(); + w.write(dos); + ensureCapacity(bos.len()); + int i = currentSize * 2; + System.arraycopy(bos.bytearray(), 0, bytes, bytesUsed, bos.len()); + offsetsArray[i] = bytesUsed; + offsetsArray[i+1] = bos.len(); + currentSize += 1; + bytesUsed += bos.len(); + lastModified = System.nanoTime(); + } + + + public int size() throws HiveException + { + PTFPersistence.lock(lock.readLock()); + try + { + return currentSize; + } + finally + { + lock.readLock().unlock(); + } + } + + public void get(int i, Writable wObj) throws HiveException + { + PTFPersistence.lock(lock.readLock()); + try + { + i = index(i); + DataIStream dis = PTFPersistence.dis.get(); + ByteArrayIS bis = dis.getUnderlyingStream(); + bis.setBuffer(bytes, offsetsArray[i], offsetsArray[i+1]); + wObj.readFields(dis); + } + catch(IOException ie) + { + throw new HiveException(ie); + } + finally + { + lock.readLock().unlock(); + } + } + + public void append(Writable obj) throws HiveException + { + PTFPersistence.lock(lock.writeLock()); + try + { + write(obj); + } + catch(IOException ie) + { + throw new HiveException(ie); + } + finally + { + lock.writeLock().unlock(); + } + + } + + public Object get(int i, Deserializer deserializer, Writable wObj) throws HiveException + { + try + { + get(i, wObj); + return deserializer.deserialize(wObj); + } + catch(SerDeException ie) + { + throw new HiveException(ie); + } + } + + public void append(Object obj, ObjectInspector OI, Serializer serializer) throws HiveException + { + try + { + append(serializer.serialize(obj, OI)); + } + catch(SerDeException ie) + { + throw new HiveException(ie); + } + } + + public Iterator iterator(Writable wObj) throws HiveException + { + return new WIterator(wObj, startOffset); + } + + public Iterator iterator(Deserializer deserializer, Writable wObj) throws HiveException + { + return new OIterator(deserializer, wObj); + } + + public void dump(StringBuilder bldr, Writable wObj) throws IOException, HiveException + { + bldr.append("["); + Iterator wi = iterator(wObj); + while(wi.hasNext()) + { + wObj = wi.next(); + bldr.append(wObj).append(", "); + } + bldr.append("]\n"); + } + + public void dump(StringBuilder bldr, Deserializer deserializer, Writable wObj) throws IOException, HiveException + { + bldr.append("["); + Iterator oi = iterator(deserializer, wObj); + while(oi.hasNext()) + { + bldr.append(oi.next()).append(", "); + } + bldr.append("]\n"); + } + + class WIterator implements Iterator + { + Writable wObj; + long checkTime; + int i; + + WIterator(Writable wObj, int offset) + { + this.wObj = wObj; + checkTime = lastModified; + i = offset; + } + + @Override + public boolean hasNext() + { + return i < currentSize; + } + + @Override + public Writable next() + { + if (checkTime != lastModified) { + throw new ConcurrentModificationException(); + } + try + { + get(i++, wObj); + return wObj; + } + catch(HiveException be) + { + throw new RuntimeException(be); + } + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + } + + class OIterator implements Iterator + { + Deserializer deserializer; + Iterator wi; + + OIterator(Deserializer deserializer, Writable wObj) throws HiveException + { + wi = iterator(wObj); + this.deserializer = deserializer; + } + + @Override + public boolean hasNext() + { + return wi.hasNext(); + } + + @Override + public Object next() + { + Writable wObj = wi.next(); + try + { + return deserializer.deserialize(wObj); + }catch(SerDeException se) + { + throw new RuntimeException(se); + } + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + } + + public static class ListFullException extends HiveException + { + private static final long serialVersionUID = 4745303310812778989L; + + public ListFullException() + { + super(); + } + + public ListFullException(String message, Throwable cause) + { + super(message, cause); + } + + public ListFullException(String message) + { + super(message); + } + + public ListFullException(Throwable cause) + { + super(cause); + } + + } + + private static final int INCREMENT_SIZE = (int) Math.pow(2, 16); + + public static final int SMALL_SIZE = (int) Math.pow(2, 6 +10); // 64KB + public static final int MEDIUM_SIZE = (int) Math.pow(2, (10 + 10 + 3)); // 8 MB + public static final int LARGE_SIZE = (int) Math.pow(2, (6 + 10 + 10)); // 64 MB + + } + + public static class PartitionedByteBasedList extends ByteBasedList + { + ArrayList partitions; + ArrayList partitionOffsets; + File dir; + int batchSize; + + public PartitionedByteBasedList(int batchSize) throws HiveException + { + this.batchSize = batchSize; + currentSize = 0; + dir = PartitionedByteBasedList.createTempDir(); + Runtime.getRuntime().addShutdownHook(new ShutdownHook(dir)); + + partitions = new ArrayList(); + partitionOffsets = new ArrayList(); + addPartition(); + } + + public PartitionedByteBasedList() throws HiveException + { + this(ByteBasedList.LARGE_SIZE); + } + + private void addPartition() throws HiveException + { + try + { + if ( partitions.size() > 0 ) + { + int idx = partitions.size() - 1; + ByteBasedList bl = partitions.get(idx); + File f = File.createTempFile("wdw", null, dir); + PersistentByteBasedList.store(bl, f); + partitions.set(idx, new PersistentByteBasedList(f, bl)); + + } + ByteBasedList bl = new ByteBasedList(currentSize, batchSize); + partitions.add(bl); + partitionOffsets.add(currentSize); + } + catch(IOException ie) + { + throw new HiveException(ie); + } + } + + private ByteBasedList getPartition(int i) throws HiveException + { + PTFPersistence.lock(lock.readLock()); + try + { + int numSplits = partitions.size(); + if ( numSplits == 0) { + return partitions.get(0); + } + int start = 0; + int end = numSplits - 1; + + while(start < end) + { + int mid = (start + end + 1) >>> 1; + int val = partitionOffsets.get(mid); + if ( val == i ) + { + return partitions.get(mid); + } + else if ( val < i ) + { + if ( end == mid) + { + return partitions.get(end); + } + start = mid; + } + else + { + end = mid - 1; + } + } + return partitions.get(start); + } + finally + { + lock.readLock().unlock(); + } + } + + @Override + public void get(int i, Writable wObj) throws HiveException + { + ByteBasedList bl = getPartition(i); + bl.get(i, wObj); + } + + @Override + public void append(Writable obj) throws HiveException + { + PTFPersistence.lock(lock.writeLock()); + try + { + partitions.get(partitions.size() -1).append(obj); + currentSize += 1; + lastModified = System.nanoTime(); + } + catch(ListFullException le) + { + addPartition(); + append(obj); + } + finally + { + lock.writeLock().unlock(); + } + + } + + @Override + public Object get(int i, Deserializer deserializer, Writable wObj) throws HiveException + { + ByteBasedList bl = getPartition(i); + return bl.get(i, deserializer, wObj); + } + + @Override + public void append(Object obj, ObjectInspector OI, Serializer serializer) throws HiveException + { + PTFPersistence.lock(lock.writeLock()); + try + { + partitions.get(partitions.size() -1).append(obj, OI, serializer); + currentSize += 1; + lastModified = System.nanoTime(); + } + catch(ListFullException le) + { + addPartition(); + append(obj, OI, serializer); + } + finally + { + lock.writeLock().unlock(); + } + } + + @Override + public Iterator iterator(Writable wObj) throws HiveException + { + return new WIterator(wObj); + } + + class WIterator implements Iterator + { + Writable wObj; + long checkTime; + int i; + Iterator pIter; + + WIterator(Writable wObj) throws HiveException + { + this.wObj = wObj; + checkTime = lastModified; + i = 0; + pIter = partitions.get(i).iterator(wObj); + } + + @Override + public boolean hasNext() + { + if ( pIter.hasNext() ) { + return true; + } + if (checkTime != lastModified) { + throw new ConcurrentModificationException(); + } + try + { + if ( i < partitions.size() ) + { + pIter = partitions.get(i++).iterator(wObj); + return hasNext(); + } + return false; + } + catch(HiveException e) + { + throw new RuntimeException(e); + } + } + + @Override + public Writable next() + { + if (checkTime != lastModified) { + throw new ConcurrentModificationException(); + } + return pIter.next(); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + } + + static class ShutdownHook extends Thread + { + File dir; + + public ShutdownHook(File dir) + { + this.dir = dir; + } + + @Override + public void run() + { + try + { + PartitionedByteBasedList.deleteRecursively(dir); + } + catch(IOException ie) + { + } + } + + } + + // copied completely from guavar09 source + /** + * Deletes a file or directory and all contents recursively. + * + *

+ * If the file argument is a symbolic link the link will be deleted but not + * the target of the link. If the argument is a directory, symbolic links + * within the directory will not be followed. + * + * @param file + * the file to delete + * @throws IOException + * if an I/O error occurs + * @see #deleteDirectoryContents + */ + public static void deleteRecursively(File file) throws IOException + { + if (file.isDirectory()) + { + deleteDirectoryContents(file); + } + if (!file.delete()) + { + throw new IOException("Failed to delete " + file); + } + } + + // copied completely from guavar09 source + /** + * Deletes all the files within a directory. Does not delete the directory + * itself. + * + *

+ * If the file argument is a symbolic link or there is a symbolic link in + * the path leading to the directory, this method will do nothing. Symbolic + * links within the directory are not followed. + * + * @param directory + * the directory to delete the contents of + * @throws IllegalArgumentException + * if the argument is not a directory + * @throws IOException + * if an I/O error occurs + * @see #deleteRecursively + */ + public static void deleteDirectoryContents(File directory) + throws IOException + { + /*Preconditions.checkArgument(directory.isDirectory(), + "Not a directory: %s", directory); + */ + if ( !directory.isDirectory()) + { + throw new IOException(String.format("Not a directory: %s", directory)); + } + + // Symbolic links will have different canonical and absolute paths + if (!directory.getCanonicalPath().equals(directory.getAbsolutePath())) + { + return; + } + File[] files = directory.listFiles(); + if (files == null) + { + throw new IOException("Error listing files for " + directory); + } + for (File file : files) + { + deleteRecursively(file); + } + } + + // copied completely from guava to remove dependency on guava + /** Maximum loop count when creating temp directories. */ + private static final int TEMP_DIR_ATTEMPTS = 10000; + public static File createTempDir() + { + File baseDir = new File(System.getProperty("java.io.tmpdir")); + String baseName = System.currentTimeMillis() + "-"; + + for (int counter = 0; counter < TEMP_DIR_ATTEMPTS; counter++) + { + File tempDir = new File(baseDir, baseName + counter); + if (tempDir.mkdir()) + { + return tempDir; + } + } + throw new IllegalStateException("Failed to create directory within " + + TEMP_DIR_ATTEMPTS + " attempts (tried " + baseName + "0 to " + + baseName + (TEMP_DIR_ATTEMPTS - 1) + ')'); + } + + } + + static class PersistentByteBasedList extends ByteBasedList + { + private static int headerSize() { return (Integer.SIZE + Integer.SIZE + Integer.SIZE + Long.SIZE) / Byte.SIZE;} + protected static void store(ByteBasedList l, File f) throws IOException + { + /* + * write startOffset:bytesUsed:currentSize:lastModified + */ + int hdrSize = headerSize(); + ByteBuffer buf = ByteBuffer.allocate(hdrSize); + + buf.putInt(l.startOffset); + buf.putInt(l.bytesUsed); + buf.putInt(l.currentSize); + buf.putLong(l.lastModified); + buf.flip(); + + /* + * note: could save this space by using Memory-Mapped I/O and directly writing to the MM buffer. + */ + ByteBuffer offsetB = ByteBuffer.allocate((Integer.SIZE/Byte.SIZE) * 2 * l.currentSize); + IntBuffer iB = offsetB.asIntBuffer(); + iB.put(l.offsetsArray, 0, l.currentSize * 2); + + ByteBuffer bytesB = ByteBuffer.wrap(l.bytes, 0, l.bytesUsed); + + ByteBuffer[] bufs = new ByteBuffer[] { buf, offsetB, bytesB}; + FileOutputStream fos = new FileOutputStream(f); + try + { + FileChannel fc = fos.getChannel(); + while (fc.write(bufs, 0, bufs.length) > 0) { + ; + } + } + finally + { + fos.close(); + } + } + + protected static void load(ByteBasedList l, File f) throws IOException + { + int hdr = headerSize(); + FileInputStream fis = new FileInputStream(f); + try + { + FileChannel fc = fis.getChannel(); + ByteBuffer buf0 = ByteBuffer.allocate(hdr); + while (buf0.hasRemaining()) { + fc.read(buf0); + } + buf0.flip(); + l.startOffset = buf0.getInt(); + l.bytesUsed = buf0.getInt(); + l.currentSize = buf0.getInt(); + l.lastModified = buf0.getLong(); + + /* + * note: could save this space by using Memory-Mapped I/O and directly writing to the MM buffer. + */ + ByteBuffer offsetB = ByteBuffer.allocate((Integer.SIZE/Byte.SIZE) * 2 * l.currentSize); + ByteBuffer bytesB = ByteBuffer.allocate(l.bytesUsed); + ByteBuffer[] bufs = new ByteBuffer[] { offsetB, bytesB }; + while (fc.read(bufs) > 0) { + ; + } + + l.offsetsArray = new int[l.currentSize * 2]; + offsetB.flip(); + IntBuffer iB = offsetB.asIntBuffer(); + iB.get(l.offsetsArray); + l.bytes = bytesB.array(); + } + finally + { + fis.close(); + } + } + + File file; + SoftReference memList; + + protected PersistentByteBasedList(File file, ByteBasedList l) + { + super(file); + this.file = file; + memList = new SoftReference(l); + } + + protected PersistentByteBasedList(File file) + { + this(file, null); + } + + private ByteBasedList getList() throws HiveException + { + PTFPersistence.lock(lock.readLock()); + try + { + ByteBasedList list = memList.get(); + if (list == null) + { + try + { + list = new ByteBasedList(file); + load(list, file); + memList = new SoftReference(list); + } + catch(Exception ie) + { + throw new RuntimeException(ie); + } + } + return list; + } + finally + { + lock.readLock().unlock(); + } + } + + @Override + public int size() throws HiveException + { + return getList().size(); + } + + @Override + public void get(int i, Writable wObj) throws HiveException + { + getList().get(i, wObj); + } + + @Override + public void append(Writable obj) throws HiveException + { + throw new UnsupportedOperationException("Cannot append to a Persisted List"); + } + + @Override + public Object get(int i, Deserializer deserializer, Writable wObj) throws HiveException + { + return getList().get(i, deserializer, wObj); + } + + @Override + public void append(Object obj, ObjectInspector OI, Serializer serializer) throws HiveException + { + throw new UnsupportedOperationException("Cannot append to a Persisted List"); + } + + @Override + public Iterator iterator(Writable wObj) throws HiveException + { + return getList().iterator(wObj); + } + + @Override + public Iterator iterator(Deserializer deserializer, Writable wObj) throws HiveException + { + return getList().iterator(deserializer, wObj); + } + + @Override + public void dump(StringBuilder bldr, Writable wObj) throws IOException, HiveException + { + getList().dump(bldr, wObj); + } + + @Override + public void dump(StringBuilder bldr, Deserializer deserializer, Writable wObj) throws IOException, HiveException + { + getList().dump(bldr, deserializer, wObj); + } + } + + public static class ByteBufferInputStream extends InputStream + { + ByteBuffer buffer; + int mark = -1; + + public void intialize(ByteBuffer buffer) + { + this.buffer = buffer; + } + + public void intialize(ByteBuffer buffer, int off, int len) + { + buffer = buffer.duplicate(); + buffer.position(off); + buffer.limit(off + len); + this.buffer = buffer.slice(); + } + + @Override + public int read() throws IOException + { + return buffer.hasRemaining() ? (buffer.get() & 0xff) : -1; + } + + @Override + public int read(byte b[], int off, int len) throws IOException + { + int remaining = buffer.remaining(); + len= len <= remaining ? len : remaining; + buffer.get(b, off, len); + return len; + } + + @Override + public boolean markSupported() { return true; } + + @Override + public void mark(int readAheadLimit) + { + mark = buffer.position(); + } + + @Override + public void reset() + { + if ( mark == -1 ) { + throw new IllegalStateException(); + } + buffer.position(mark); + mark = -1; + } + } + + public static class ByteBufferOutputStream extends OutputStream + { + ByteBuffer buffer; + + public void intialize(ByteBuffer buffer) + { + this.buffer = buffer; + } + + public void intialize(ByteBuffer buffer, int off, int len) + { + buffer = buffer.duplicate(); + buffer.position(off); + buffer.limit(off + len); + this.buffer = buffer.slice(); + } + + @Override + public void write(int b) throws IOException + { + buffer.put((byte) b); + } + + @Override + public void write(byte b[], int off, int len) + { + int remaining = buffer.remaining(); + if ( len > remaining ) + { + throw new IndexOutOfBoundsException(); + } + buffer.put(b, off, len); + } + } + + public static ThreadLocal bis = new ThreadLocal() + { + @Override + protected ByteArrayIS initialValue() + { + return new ByteArrayIS(); + } + }; + + public static ThreadLocal dis = new ThreadLocal() + { + @Override + protected DataIStream initialValue() + { + return new DataIStream(bis.get()); + } + }; + + public static ThreadLocal bos = new ThreadLocal() + { + @Override + protected ByteArrayOS initialValue() + { + return new ByteArrayOS(); + } + }; + + public static ThreadLocal dos = new ThreadLocal() + { + @Override + protected DataOStream initialValue() + { + return new DataOStream(bos.get()); + } + }; + + + public static class DataIStream extends DataInputStream + { + public DataIStream(ByteArrayIS in) + { + super(in); + } + + public ByteArrayIS getUnderlyingStream() { return (ByteArrayIS) in; } + } + + public static class DataOStream extends DataOutputStream + { + public DataOStream(ByteArrayOS out) + { + super(out); + } + + public ByteArrayOS getUnderlyingStream() { return (ByteArrayOS) out; } + } + + public static class ByteArrayOS extends ByteArrayOutputStream + { + public ByteArrayOS() { super(); } + public ByteArrayOS(int size) { super(size); } + public final byte[] bytearray() { return buf; } + public final int len() { return count; } + + } + + public static class ByteArrayIS extends ByteArrayInputStream + { + public ByteArrayIS() { super(new byte[0]); } + public final byte[] bytearray() { return buf; } + public final void setBuffer(byte[] buf, int offset, int len) + { + this.buf = buf; + this.pos = offset; + this.count = Math.min(offset + len, buf.length); + this.mark = offset; + } + + } + + public static void lock(Lock lock) throws HiveException + { + try + { + lock.lockInterruptibly(); + + } + catch(InterruptedException ie) + { + Thread.currentThread().interrupt(); + throw new HiveException("Operation interrupted", ie); + } + } + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/PTFUtils.java ql/src/java/org/apache/hadoop/hive/ql/exec/PTFUtils.java new file mode 100644 index 0000000..defeada --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/PTFUtils.java @@ -0,0 +1,286 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.beans.BeanInfo; +import java.beans.Encoder; +import java.beans.ExceptionListener; +import java.beans.Expression; +import java.beans.IntrospectionException; +import java.beans.Introspector; +import java.beans.PersistenceDelegate; +import java.beans.PropertyDescriptor; +import java.beans.Statement; +import java.beans.XMLDecoder; +import java.beans.XMLEncoder; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.antlr.runtime.CommonToken; +import org.antlr.runtime.tree.BaseTree; +import org.antlr.runtime.tree.CommonTree; +import org.apache.hadoop.hive.ql.exec.Utilities.EnumDelegate; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.Direction; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +public class PTFUtils { + + public static String toString(List col) + { + StringBuilder buf = new StringBuilder(); + buf.append("["); + boolean first = true; + for (Object o : col) + { + if (first) { + first = false; + } else { + buf.append(", "); + } + buf.append(o.toString()); + } + buf.append("]"); + return buf.toString(); + } + + public static String toString(Map col) + { + StringBuilder buf = new StringBuilder(); + buf.append("["); + boolean first = true; + for (Map.Entry o : col.entrySet()) + { + if (first) { + first = false; + } else { + buf.append(", "); + } + buf.append(o.getKey().toString()).append(" : ") + .append(o.getValue().toString()); + } + buf.append("]"); + return buf.toString(); + } + + public static String unescapeQueryString(String qry) + { + qry = qry.replace("\\\"", "\""); + qry = qry.replace("\\'", "'"); + return qry; + } + + public static class ReverseIterator implements Iterator + { + Stack stack; + + public ReverseIterator(Iterator it) + { + stack = new Stack(); + while (it.hasNext()) + { + stack.push(it.next()); + } + } + + @Override + public boolean hasNext() + { + return !stack.isEmpty(); + } + + @Override + public T next() + { + return stack.pop(); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + } + + public static abstract class Predicate + { + public abstract boolean apply(T obj); + }; + + + + /* + * serialization functions + */ + public static void serialize(OutputStream out, Object o) + { + XMLEncoder e = new XMLEncoder(out); + e.setExceptionListener(new EL()); + PTFUtils.addPersistenceDelegates(e); + e.writeObject(o); + e.close(); + } + + public static Object deserialize(InputStream in1) + { + XMLDecoder d = null; + try + { + d = new XMLDecoder(in1, null, null); + return d.readObject(); + } + finally + { + if (null != d) + { + d.close(); + } + } + } + + public static void addPersistenceDelegates(XMLEncoder e) + { + addAntlrPersistenceDelegates(e); + addHivePersistenceDelegates(e); + addEnumDelegates(e); + } + + public static void addEnumDelegates(XMLEncoder e) + { + e.setPersistenceDelegate(Direction.class, new EnumDelegate()); + } + + public static void addAntlrPersistenceDelegates(XMLEncoder e) + { + e.setPersistenceDelegate(ASTNode.class, new PersistenceDelegate() + { + + @Override + protected Expression instantiate(Object oldInstance, Encoder out) + { + return new Expression(oldInstance, oldInstance.getClass(), + "new", new Object[] + { ((ASTNode) oldInstance).getToken() }); + } + }); + e.setPersistenceDelegate(CommonTree.class, new PersistenceDelegate() + { + @Override + protected Expression instantiate(Object oldInstance, Encoder out) + { + return new Expression(oldInstance, oldInstance.getClass(), + "new", new Object[] + { ((CommonTree) oldInstance).getToken() }); + } + }); + e.setPersistenceDelegate(BaseTree.class, new PersistenceDelegate() + { + @Override + protected Expression instantiate(Object oldInstance, Encoder out) + { + return new Expression(oldInstance, oldInstance.getClass(), + "new", new Object[] + {}); + } + + @Override + @SuppressWarnings("rawtypes") + protected void initialize(Class type, Object oldInstance, + Object newInstance, Encoder out) + { + super.initialize(type, oldInstance, newInstance, out); + + BaseTree t = (BaseTree) oldInstance; + + for (int i = 0; i < t.getChildCount(); i++) + { + out.writeStatement(new Statement(oldInstance, "addChild", + new Object[] + { t.getChild(i) })); + } + } + }); + e.setPersistenceDelegate(CommonToken.class, new PersistenceDelegate() + { + @Override + protected Expression instantiate(Object oldInstance, Encoder out) + { + return new Expression(oldInstance, oldInstance.getClass(), + "new", new Object[] + { ((CommonToken) oldInstance).getType(), + ((CommonToken) oldInstance).getText() }); + } + }); + } + + public static void addHivePersistenceDelegates(XMLEncoder e) + { + e.setPersistenceDelegate(PrimitiveTypeInfo.class, + new PersistenceDelegate() + { + @Override + protected Expression instantiate(Object oldInstance, + Encoder out) + { + return new Expression(oldInstance, + TypeInfoFactory.class, "getPrimitiveTypeInfo", + new Object[] + { ((PrimitiveTypeInfo) oldInstance) + .getTypeName() }); + } + }); + } + + static class EL implements ExceptionListener + { + public void exceptionThrown(Exception e) + { + e.printStackTrace(); + throw new RuntimeException("Cannot serialize the query plan", e); + } + } + + public static void makeTransient(Class beanClass, String pdName) + { + BeanInfo info; + try + { + info = Introspector.getBeanInfo(beanClass); + PropertyDescriptor[] propertyDescriptors = info + .getPropertyDescriptors(); + for (int i = 0; i < propertyDescriptors.length; ++i) + { + PropertyDescriptor pd = propertyDescriptors[i]; + if (pd.getName().equals(pdName)) + { + pd.setValue("transient", Boolean.TRUE); + } + } + } + catch (IntrospectionException ie) + { + throw new RuntimeException(ie); + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionTableFunctionDescription.java ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionTableFunctionDescription.java new file mode 100644 index 0000000..bd48531 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionTableFunctionDescription.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.hadoop.hive.ql.udf.ptf.WindowingTableFunction; + +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +@Documented +public @interface PartitionTableFunctionDescription +{ + Description description (); + + /** + * if true it is not usable in the language. {@link WindowingTableFunction} is the only internal function. + */ + boolean isInternal() default false; +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionDescription.java ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionDescription.java new file mode 100644 index 0000000..ba4c136 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionDescription.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver2; +import org.apache.hadoop.hive.ql.udf.ptf.WindowingTableFunction; + +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +@Documented +public @interface WindowFunctionDescription +{ + Description description (); + /** + * controls whether this function can be applied to a Window. + *

+ * Ranking function: Rank, Dense_Rank, Percent_Rank and Cume_Dist don't operate on Windows. + * Why? a window specification implies a row specific range i.e. every row gets its own set of rows to process the UDAF on. + * For ranking defining a set of rows for every row makes no sense. + *

+ * All other UDAFs can be computed for a Window. + */ + boolean supportsWindow() default true; + /** + * A WindowFunc is implemented as {@link GenericUDAFResolver2}. It returns only one value. + * If this is true then the function must return a List which is taken to be the column for this function in the Output table returned by the + * {@link WindowingTableFunction}. Otherwise the output is assumed to be a single value, the column of the Output will contain the same value + * for all the rows. + */ + boolean pivotResult() default false; +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java new file mode 100644 index 0000000..5153089 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.hadoop.hive.ql.exec.FunctionInfo; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver; + +@SuppressWarnings("deprecation") +public class WindowFunctionInfo +{ + boolean supportsWindow = true; + boolean pivotResult = false; + FunctionInfo fInfo; + + WindowFunctionInfo(FunctionInfo fInfo) + { + assert fInfo.isGenericUDAF(); + this.fInfo = fInfo; + Class wfnCls = fInfo.getGenericUDAFResolver().getClass(); + WindowFunctionDescription def = wfnCls.getAnnotation(WindowFunctionDescription.class); + if ( def != null) + { + supportsWindow = def.supportsWindow(); + pivotResult = def.pivotResult(); + } + } + + public boolean isSupportsWindow() + { + return supportsWindow; + } + + public boolean isPivotResult() + { + return pivotResult; + } + + public FunctionInfo getfInfo() + { + return fInfo; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageInfo.java ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageInfo.java index b687a2f..22a8785 100644 --- ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageInfo.java +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageInfo.java @@ -47,7 +47,8 @@ * set operations like union on columns on other tables * e.g. T2.c1 = T1.c1 + T3.c1. * 4. SCRIPT - Indicates that the column is derived from the output - * of a user script through a TRANSFORM, MAP or REDUCE syntax. + * of a user script through a TRANSFORM, MAP or REDUCE syntax + * or from the output of a PTF chain execution. */ public static enum DependencyType { SIMPLE, EXPRESSION, SCRIPT diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java index 582ef14..58a9b59 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java @@ -23,19 +23,19 @@ import java.util.LinkedHashMap; import java.util.Map; -import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.FilterOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.SelectOperator; -import org.apache.hadoop.hive.ql.exec.ScriptOperator; import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator; +import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.UnionOperator; -import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.PTFOperator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator; -import org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.ScriptOperator; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; import org.apache.hadoop.hive.ql.lib.Dispatcher; @@ -114,6 +114,9 @@ public ParseContext transform(ParseContext pactx) throws SemanticException { opRules.put(new RuleRegExp("R9", LateralViewForwardOperator.getOperatorName() + "%"), ColumnPrunerProcFactory.getLateralViewForwardProc()); + opRules.put(new RuleRegExp("R10", + PTFOperator.getOperatorName() + "%"), + ColumnPrunerProcFactory.getPTFProc()); // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along Dispatcher disp = new DefaultRuleDispatcher(ColumnPrunerProcFactory diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java index a9a93ad..d49ddc8 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -31,6 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; +import org.apache.hadoop.hive.ql.exec.ExtractOperator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.FilterOperator; import org.apache.hadoop.hive.ql.exec.GroupByOperator; @@ -40,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.LimitOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.PTFOperator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; import org.apache.hadoop.hive.ql.exec.RowSchema; import org.apache.hadoop.hive.ql.exec.ScriptOperator; @@ -62,11 +65,21 @@ import org.apache.hadoop.hive.ql.plan.JoinDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.ShapeDetails; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowTableFunctionDef; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.SelectDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; /** * Factory for generating the different node processors used by ColumnPruner. @@ -148,6 +161,174 @@ public static ColumnPrunerGroupByProc getGroupByProc() { } /** + * - Pruning can only be done for Windowing. PTFs are black boxes, + * we assume all columns are needed. + * - add column names referenced in WindowFn args and in WindowFn expressions + * to the pruned list of the child Select Op. + * - Prune the Column names & types serde properties in each of the Shapes in the PTF Chain: + * - the InputDef's output shape + * - Window Tabl Functions: window output shape & output shape. + * - Why is pruning the Column names & types in the serde properties enough? + * - because during runtime we rebuild the OIs using these properties. + * - finally we set the prunedColList on the ColumnPrunerContx; + * and update the RR & signature on the PTFOp. + */ + public static class ColumnPrunerPTFProc implements NodeProcessor { + public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, + Object... nodeOutputs) throws SemanticException { + + PTFOperator op = (PTFOperator) nd; + PTFDesc conf = op.getConf(); + //Since we cannot know what columns will be needed by a PTF chain, + //we do not prune columns on PTFOperator for PTF chains. + if (!conf.forWindowing()) { + return getDefaultProc().process(nd, stack, ctx, nodeOutputs); + } + + ColumnPrunerProcCtx cppCtx = (ColumnPrunerProcCtx) ctx; + WindowTableFunctionDef def = (WindowTableFunctionDef) conf.getFuncDef(); + ArrayList sig = new ArrayList(); + + List prunedCols = cppCtx.getPrunedColList(op.getChildOperators().get(0)); + //we create a copy of prunedCols to create a list of pruned columns for PTFOperator + prunedCols = new ArrayList(prunedCols); + prunedColumnsList(prunedCols, def); + setSerdePropsOfShape(def.getInput().getOutputShape(), prunedCols); + setSerdePropsOfShape(def.getOutputFromWdwFnProcessing(), prunedCols); + setSerdePropsOfShape(def.getOutputShape(), prunedCols); + + RowResolver oldRR = cppCtx.getOpToParseCtxMap().get(op).getRowResolver(); + RowResolver newRR = buildPrunedRR(prunedCols, oldRR, sig); + cppCtx.getPrunedColLists().put(op, prunedInputList(prunedCols, def)); + cppCtx.getOpToParseCtxMap().get(op).setRowResolver(newRR); + op.getSchema().setSignature(sig); + return null; + } + + private static RowResolver buildPrunedRR(List prunedCols, + RowResolver oldRR, ArrayList sig) throws SemanticException{ + RowResolver newRR = new RowResolver(); + for (String col : prunedCols) { + String[] nm = oldRR.reverseLookup(col); + ColumnInfo colInfo = oldRR.get(nm[0], nm[1]); + if (colInfo != null) { + newRR.put(nm[0], nm[1], colInfo); + sig.add(colInfo); + } + } + return newRR; + } + + /* + * add any input columns referenced in WindowFn args or expressions. + */ + private void prunedColumnsList(List prunedCols, WindowTableFunctionDef tDef) { + if ( tDef.getWindowFunctions() != null ) { + for(WindowFunctionDef wDef : tDef.getWindowFunctions() ) { + if ( wDef.getArgs() == null) { + continue; + } + for(PTFExpressionDef arg : wDef.getArgs()) { + ExprNodeDesc exprNode = arg.getExprNode(); + Utilities.mergeUniqElems(prunedCols, exprNode.getCols()); + } + } + } + if ( tDef.getWindowExpressions() != null ) { + for(WindowExpressionDef expr : tDef.getWindowExpressions()) { + ExprNodeDesc exprNode = expr.getExprNode(); + Utilities.mergeUniqElems(prunedCols, exprNode.getCols()); + } + } + if(tDef.getPartition() != null){ + for(PTFExpressionDef col : tDef.getPartition().getExpressions()){ + ExprNodeDesc exprNode = col.getExprNode(); + Utilities.mergeUniqElems(prunedCols, exprNode.getCols()); + } + } + if(tDef.getOrder() != null){ + for(PTFExpressionDef col : tDef.getOrder().getExpressions()){ + ExprNodeDesc exprNode = col.getExprNode(); + Utilities.mergeUniqElems(prunedCols, exprNode.getCols()); + } + } + if(tDef.getHavingExpression() != null){ + ExprNodeDesc exprNode = tDef.getHavingExpression().getExprNode(); + Utilities.mergeUniqElems(prunedCols, exprNode.getCols()); + } + } + + private List getLowerCasePrunedCols(List prunedCols){ + List lowerCasePrunedCols = new ArrayList(); + for (String col : prunedCols) { + lowerCasePrunedCols.add(col.toLowerCase()); + } + return lowerCasePrunedCols; + } + + /* + * reconstruct Column names & types list based on the prunedCols list. + */ + private void setSerdePropsOfShape(ShapeDetails shp, List prunedCols) { + List columnNames = Arrays.asList(shp.getSerdeProps().get( + org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS).split(",")); + List columnTypes = TypeInfoUtils + .getTypeInfosFromTypeString(shp.getSerdeProps().get( + org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES)); + /* + * fieldNames in OI are lower-cased. So we compare lower cased names for now. + */ + prunedCols = getLowerCasePrunedCols(prunedCols); + + StringBuilder cNames = new StringBuilder(); + StringBuilder cTypes = new StringBuilder(); + + boolean addComma = false; + for(int i=0; i < columnNames.size(); i++) { + if ( prunedCols.contains(columnNames.get(i)) ) { + cNames.append(addComma ? "," : ""); + cTypes.append(addComma ? "," : ""); + cNames.append(columnNames.get(i)); + cTypes.append(columnTypes.get(i)); + addComma = true; + } + } + shp.getSerdeProps().put( + org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS, cNames.toString()); + shp.getSerdeProps().put( + org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES, cTypes.toString()); + } + + /* + * from the prunedCols list filter out columns that refer to WindowFns or WindowExprs + * the returned list is set as the prunedList needed by the PTFOp. + */ + private ArrayList prunedInputList(List prunedCols, + WindowTableFunctionDef tDef) { + ArrayList prunedInputCols = new ArrayList(); + + StructObjectInspector OI = tDef.getInput().getOutputShape().getOI(); + for(StructField f : OI.getAllStructFieldRefs()) { + String fName = f.getFieldName(); + if ( prunedCols.contains(fName)) { + prunedInputCols.add(fName); + } + } + + return prunedInputCols; + } + } + + /** + * Factory method to get the ColumnPrunerGroupByProc class. + * + * @return ColumnPrunerGroupByProc + */ + public static ColumnPrunerPTFProc getPTFProc() { + return new ColumnPrunerPTFProc(); + } + + /** * The Default Node Processor for Column Pruning. */ public static class ColumnPrunerDefaultProc implements NodeProcessor { @@ -285,6 +466,39 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, } Collections.sort(colLists); pruneReduceSinkOperator(flags, op, cppCtx); + } else if ((childOperators.size() == 1) + && (childOperators.get(0) instanceof ExtractOperator ) + && (childOperators.get(0).getChildOperators().size() == 1) + && (childOperators.get(0).getChildOperators().get(0) instanceof PTFOperator ) + && ((PTFOperator)childOperators.get(0). + getChildOperators().get(0)).getConf().forWindowing() ) { + + /* + * For RS that are followed by Extract & PTFOp for windowing + * - do the same thing as above. Reconstruct ValueColumn list based on what is required + * by the PTFOp. + */ + + assert parentOperators.size() == 1; + + PTFOperator ptfOp = (PTFOperator) childOperators.get(0).getChildOperators().get(0); + List childCols = cppCtx.getPrunedColList(ptfOp); + boolean[] flags = new boolean[conf.getValueCols().size()]; + for (int i = 0; i < flags.length; i++) { + flags[i] = false; + } + if (childCols != null && childCols.size() > 0) { + ArrayList outColNames = op.getConf().getOutputValueColumnNames(); + for(int i=0; i < outColNames.size(); i++ ) { + if ( childCols.contains(outColNames.get(i))) { + ExprNodeDesc exprNode = op.getConf().getValueCols().get(i); + flags[i] = true; + Utilities.mergeUniqElems(colLists, exprNode.getCols()); + } + } + } + Collections.sort(colLists); + pruneReduceSinkOperator(flags, op, cppCtx); } else { // Reduce Sink contains the columns needed - no need to aggregate from // children @@ -831,4 +1045,4 @@ public static ColumnPrunerMapJoinProc getMapJoinProc() { return new ColumnPrunerMapJoinProc(); } -} +} \ No newline at end of file diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java index ccbbd28..51bef04 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java @@ -22,17 +22,17 @@ import java.util.LinkedHashMap; import java.util.Map; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.SelectOperator; -import org.apache.hadoop.hive.ql.exec.ScriptOperator; +import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; +import org.apache.hadoop.hive.ql.exec.PTFOperator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.ScriptOperator; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.UDTFOperator; import org.apache.hadoop.hive.ql.exec.UnionOperator; -import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; import org.apache.hadoop.hive.ql.lib.Dispatcher; import org.apache.hadoop.hive.ql.lib.GraphWalker; @@ -82,6 +82,8 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { OpProcFactory.getReduceSinkProc()); opRules.put(new RuleRegExp("R9", LateralViewJoinOperator.getOperatorName() + "%"), OpProcFactory.getLateralViewJoinProc()); + opRules.put(new RuleRegExp("R10", PTFOperator.getOperatorName() + "%"), + OpProcFactory.getTransformProc()); // The dispatcher fires the processor corresponding to the closest matching rule and passes the context along Dispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(), opRules, lCtx); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/ASTNode.java ql/src/java/org/apache/hadoop/hive/ql/parse/ASTNode.java index a6fe4c8..7dbd96f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/ASTNode.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/ASTNode.java @@ -37,10 +37,6 @@ public ASTNode() { } - public ASTNode(ASTNode copy){ - super(copy); - } - /** * Constructor. * @@ -51,6 +47,16 @@ public ASTNode(Token t) { super(t); } + public ASTNode(ASTNode node) { + super(node); + this.origin = node.origin; + } + + @Override + public Tree dupNode() { + return new ASTNode(this); + } + /* * (non-Javadoc) * @@ -95,12 +101,6 @@ public void setOrigin(ASTNodeOrigin origin) { this.origin = origin; } - @Override - public Tree dupNode() { - - return new ASTNode(this); - } - public String dump() { StringBuilder sb = new StringBuilder(); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g index 91acc05..674eb1d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g +++ ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g @@ -139,7 +139,7 @@ fromSource @init { gParent.msgs.push("from source"); } @after { gParent.msgs.pop(); } : - (tableSource | subQuerySource) (lateralView^)* + ((Identifier LPAREN)=> partitionedTableFunction | tableSource | subQuerySource) (lateralView^)* ; tableBucketSample @@ -202,6 +202,98 @@ subQuerySource LPAREN queryStatementExpression RPAREN identifier -> ^(TOK_SUBQUERY queryStatementExpression identifier) ; +//---------------------- Rules for parsing PTF clauses ----------------------------- +partitioningSpec +@init { gParent.msgs.push("partitioningSpec clause"); } +@after { gParent.msgs.pop(); } + : + partitionByClause orderByClause? -> ^(TOK_PARTITIONINGSPEC partitionByClause orderByClause?) | + distributeByClause sortByClause? -> ^(TOK_PARTITIONINGSPEC distributeByClause sortByClause?) | + clusterByClause -> ^(TOK_PARTITIONINGSPEC clusterByClause) + ; + +partitionTableFunctionSource +@init { gParent.msgs.push("partitionTableFunctionSource clause"); } +@after { gParent.msgs.pop(); } + : + subQuerySource | + tableSource | + partitionedTableFunction + ; + +partitionedTableFunction +@init { gParent.msgs.push("ptf clause"); } +@after { gParent.msgs.pop(); } + : + name=Identifier + LPAREN KW_ON ptfsrc=partitionTableFunctionSource partitioningSpec? + ((Identifier LPAREN expression RPAREN ) => Identifier LPAREN expression RPAREN ( COMMA Identifier LPAREN expression RPAREN)*)? + RPAREN alias=Identifier? + -> ^(TOK_PTBLFUNCTION $name $alias? partitionTableFunctionSource partitioningSpec? expression*) + ; + +//---------------------- Rules for windowing clauses ------------------------------- +window_clause +@init { gParent.msgs.push("window_clause"); } +@after { gParent.msgs.pop(); } +: + KW_WINDOW window_defn (COMMA window_defn)* -> ^(KW_WINDOW window_defn+) +; + +window_defn +@init { gParent.msgs.push("window_defn"); } +@after { gParent.msgs.pop(); } +: + Identifier KW_AS window_specification -> ^(TOK_WINDOWDEF Identifier window_specification) +; + +window_specification +@init { gParent.msgs.push("window_specification"); } +@after { gParent.msgs.pop(); } +: + (Identifier | ( LPAREN Identifier? partitioningSpec? window_frame? RPAREN)) -> ^(TOK_WINDOWSPEC Identifier? partitioningSpec? window_frame?) +; + +window_frame : + window_range_expression | + window_value_expression +; + +window_range_expression +@init { gParent.msgs.push("window_range_expression"); } +@after { gParent.msgs.pop(); } +: + KW_ROWS KW_UNBOUNDED KW_PRECEDING -> ^(TOK_WINDOWRANGE ^(KW_PRECEDING KW_UNBOUNDED) ^(KW_CURRENT)) | + KW_ROWS KW_BETWEEN s=rowsboundary KW_AND end=rowsboundary -> ^(TOK_WINDOWRANGE $s $end) +; + +rowsboundary +@init { gParent.msgs.push("rowsboundary"); } +@after { gParent.msgs.pop(); } +: + KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING) -> ^($r KW_UNBOUNDED) | + KW_CURRENT KW_ROW -> ^(KW_CURRENT) | + Number (d=KW_PRECEDING | d=KW_FOLLOWING ) -> ^($d Number) +; + +window_value_expression +@init { gParent.msgs.push("window_value_expression"); } +@after { gParent.msgs.pop(); } +: + KW_RANGE KW_UNBOUNDED KW_PRECEDING -> ^(TOK_WINDOWVALUES ^(KW_PRECEDING KW_UNBOUNDED) ^(KW_CURRENT)) | + KW_RANGE KW_BETWEEN s=valuesboundary KW_AND end=valuesboundary -> ^(TOK_WINDOWVALUES $s $end) +; + +valuesboundary +@init { gParent.msgs.push("valuesboundary"); } +@after { gParent.msgs.pop(); } +: + KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING) -> ^($r KW_UNBOUNDED) | + KW_CURRENT KW_ROW -> ^(KW_CURRENT) | + rowExp=expression rngExp=Number (d=KW_LESS | d=KW_MORE ) -> ^($d $rowExp $rngExp) +; + + //----------------------- Rules for parsing whereClause ----------------------------- // where a=b and ... whereClause diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g index 2271627..31a5889 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g +++ ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g @@ -241,6 +241,14 @@ KW_ROLLUP: 'ROLLUP'; KW_CUBE: 'CUBE'; KW_DIRECTORIES: 'DIRECTORIES'; KW_FOR: 'FOR'; +KW_WINDOW: 'WINDOW'; +KW_UNBOUNDED: 'UNBOUNDED'; +KW_PRECEDING: 'PRECEDING'; +KW_FOLLOWING: 'FOLLOWING'; +KW_CURRENT: 'CURRENT'; +KW_LESS: 'LESS'; +KW_MORE: 'MORE'; +KW_OVER: 'OVER'; KW_GROUPING: 'GROUPING'; KW_SETS: 'SETS'; KW_TRUNCATE: 'TRUNCATE'; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g index d379875..d2f647c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g +++ ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g @@ -287,6 +287,12 @@ TOK_SKEWED_LOCATIONS; TOK_SKEWED_LOCATION_LIST; TOK_SKEWED_LOCATION_MAP; TOK_STOREDASDIRS; +TOK_PARTITIONINGSPEC; +TOK_PTBLFUNCTION; +TOK_WINDOWDEF; +TOK_WINDOWSPEC; +TOK_WINDOWVALUES; +TOK_WINDOWRANGE; TOK_IGNOREPROTECTION; } @@ -1789,9 +1795,10 @@ regular_body clusterByClause? distributeByClause? sortByClause? + window_clause? limitClause? -> ^(TOK_QUERY fromClause ^(TOK_INSERT insertClause selectClause whereClause? groupByClause? havingClause? orderByClause? clusterByClause? - distributeByClause? sortByClause? limitClause?)) + distributeByClause? sortByClause? window_clause? limitClause?)) | selectStatement ; @@ -1807,9 +1814,10 @@ selectStatement clusterByClause? distributeByClause? sortByClause? + window_clause? limitClause? -> ^(TOK_QUERY fromClause ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE)) selectClause whereClause? groupByClause? havingClause? orderByClause? clusterByClause? - distributeByClause? sortByClause? limitClause?)) + distributeByClause? sortByClause? window_clause? limitClause?)) ; @@ -1824,9 +1832,10 @@ body clusterByClause? distributeByClause? sortByClause? + window_clause? limitClause? -> ^(TOK_INSERT insertClause? selectClause whereClause? groupByClause? havingClause? orderByClause? clusterByClause? - distributeByClause? sortByClause? limitClause?) + distributeByClause? sortByClause? window_clause? limitClause?) | selectClause whereClause? @@ -1836,9 +1845,10 @@ body clusterByClause? distributeByClause? sortByClause? + window_clause? limitClause? -> ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE)) selectClause whereClause? groupByClause? havingClause? orderByClause? clusterByClause? - distributeByClause? sortByClause? limitClause?) + distributeByClause? sortByClause? window_clause? limitClause?) ; insertClause diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g index fe35d2e..76a5762 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g +++ ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g @@ -124,7 +124,18 @@ clusterByClause | KW_CLUSTER KW_BY expression - ( COMMA expression )* -> ^(TOK_CLUSTERBY expression+) + ( (COMMA)=>COMMA expression )* -> ^(TOK_CLUSTERBY expression+) + ; + +partitionByClause +@init { gParent.msgs.push("partition by clause"); } +@after { gParent.msgs.pop(); } + : + KW_PARTITION KW_BY + LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_DISTRIBUTEBY expression+) + | + KW_PARTITION KW_BY + expression ((COMMA)=> COMMA expression)* -> ^(TOK_DISTRIBUTEBY expression+) ; distributeByClause @@ -135,7 +146,7 @@ distributeByClause LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_DISTRIBUTEBY expression+) | KW_DISTRIBUTE KW_BY - expression (COMMA expression)* -> ^(TOK_DISTRIBUTEBY expression+) + expression ((COMMA)=> COMMA expression)* -> ^(TOK_DISTRIBUTEBY expression+) ; sortByClause @@ -148,7 +159,7 @@ sortByClause | KW_SORT KW_BY columnRefOrder - ( COMMA columnRefOrder)* -> ^(TOK_SORTBY columnRefOrder+) + ( (COMMA)=> COMMA columnRefOrder)* -> ^(TOK_SORTBY columnRefOrder+) ; // fun(par1, par2, par3) diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java new file mode 100644 index 0000000..3ea982b --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java @@ -0,0 +1,515 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse; + +import java.util.ArrayList; + +import org.apache.hadoop.hive.ql.exec.PTFUtils; + +public class PTFInvocationSpec { + + PartitionedTableFunctionSpec function; + + public PartitionedTableFunctionSpec getFunction() { + return function; + } + + public void setFunction(PartitionedTableFunctionSpec function) { + this.function = function; + } + + public PartitionedTableFunctionSpec getStartOfChain() { + return function == null ? null : function.getStartOfChain(); + } + + public String getQueryInputName() { + return function == null ? null : function.getQueryInputName(); + } + + public PTFQueryInputSpec getQueryInput() { + return function == null ? null : function.getQueryInput(); + } + + /* + * A PTF Input represents the input to a PTF Function. An Input can be a Hive SubQuery or Table + * or another PTF Function. An Input instance captures the ASTNode that this instance was created from. + */ + public abstract static class PTFInputSpec { + ASTNode astNode; + + public ASTNode getAstNode() { + return astNode; + } + + public void setAstNode(ASTNode astNode) { + this.astNode = astNode; + } + + public abstract PTFInputSpec getInput(); + + public abstract String getQueryInputName(); + public abstract PTFQueryInputSpec getQueryInput(); + } + + public static enum PTFQueryInputType { + TABLE, + SUBQUERY, + PTFCOMPONENT, + WINDOWING; + } + + /* + * A PTF input that represents a source in the overall Query. This could be a Table or a SubQuery. + * If a PTF chain requires execution by multiple PTF Operators; + * then the original Invocation object is decomposed into a set of Component Invocations. + * Every component Invocation but the first one ends in a PTFQueryInputSpec instance. + * During the construction of the Operator plan a PTFQueryInputSpec object in the chain implies connect the PTF Operator to the + * 'input' i.e. has been generated so far. + */ + public static class PTFQueryInputSpec extends PTFInputSpec { + String source; + PTFQueryInputType type; + + public String getSource() { + return source; + } + public void setSource(String source) { + this.source = source; + } + public PTFQueryInputType getType() { + return type; + } + public void setType(PTFQueryInputType type) { + this.type = type; + } + + @Override + public PTFInputSpec getInput() { + return null; + } + + @Override + public String getQueryInputName() { + return getSource(); + } + @Override + public PTFQueryInputSpec getQueryInput() { + return this; + } + } + + /* + * Represents a PTF Invocation. Captures: + * - function name and alias + * - the Partitioning details about its input + * - its arguments. The ASTNodes representing the arguments are captured here. + * - a reference to its Input + */ + public static class PartitionedTableFunctionSpec extends PTFInputSpec { + String name; + String alias; + ArrayList args; + PartitioningSpec partitioning; + PTFInputSpec input; + public String getName() { + return name; + } + public void setName(String name) { + this.name = name; + } + public String getAlias() { + return alias; + } + public void setAlias(String alias) { + this.alias = alias; + } + public ArrayList getArgs() { + return args; + } + public void setArgs(ArrayList args) { + this.args = args; + } + public PartitioningSpec getPartitioning() { + return partitioning; + } + public void setPartitioning(PartitioningSpec partitioning) { + this.partitioning = partitioning; + } + @Override + public PTFInputSpec getInput() { + return input; + } + public void setInput(PTFInputSpec input) { + this.input = input; + } + public PartitionSpec getPartition() { + return getPartitioning() == null ? null : getPartitioning().getPartSpec(); + } + public void setPartition(PartitionSpec partSpec) { + partitioning = partitioning == null ? new PartitioningSpec() : partitioning; + partitioning.setPartSpec(partSpec); + } + public OrderSpec getOrder() { + return getPartitioning() == null ? null : getPartitioning().getOrderSpec(); + } + public void setOrder(OrderSpec orderSpec) { + partitioning = partitioning == null ? new PartitioningSpec() : partitioning; + partitioning.setOrderSpec(orderSpec); + } + public void addArg(ASTNode arg) + { + args = args == null ? new ArrayList() : args; + args.add(arg); + } + + public PartitionedTableFunctionSpec getStartOfChain() { + if ( input instanceof PartitionedTableFunctionSpec ) { + return ((PartitionedTableFunctionSpec)input).getStartOfChain(); + } + return this; + } + @Override + public String getQueryInputName() { + return input.getQueryInputName(); + } + @Override + public PTFQueryInputSpec getQueryInput() { + return input.getQueryInput(); + } + } + + /* + * Captures how the Input to a PTF Function should be partitioned and + * ordered. Refers to a /Partition/ and /Order/ instance. + */ + public static class PartitioningSpec { + PartitionSpec partSpec; + OrderSpec orderSpec; + public PartitionSpec getPartSpec() { + return partSpec; + } + public void setPartSpec(PartitionSpec partSpec) { + this.partSpec = partSpec; + } + public OrderSpec getOrderSpec() { + return orderSpec; + } + public void setOrderSpec(OrderSpec orderSpec) { + this.orderSpec = orderSpec; + } + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((orderSpec == null) ? 0 : orderSpec.hashCode()); + result = prime * result + ((partSpec == null) ? 0 : partSpec.hashCode()); + return result; + } + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + PartitioningSpec other = (PartitioningSpec) obj; + if (orderSpec == null) { + if (other.orderSpec != null) { + return false; + } + } else if (!orderSpec.equals(other.orderSpec)) { + return false; + } + if (partSpec == null) { + if (other.partSpec != null) { + return false; + } + } else if (!partSpec.equals(other.partSpec)) { + return false; + } + return true; + } + } + + /* + * Captures how an Input should be Partitioned. This is captured as a + * list of ASTNodes that are the expressions in the Distribute/Cluster + * by clause specifying the partitioning applied for a PTF invocation. + */ + public static class PartitionSpec { + ArrayList expressions; + + public ArrayList getExpressions() + { + return expressions; + } + + public void setExpressions(ArrayList columns) + { + this.expressions = columns; + } + + public void addExpression(PartitionExpression c) + { + expressions = expressions == null ? new ArrayList() : expressions; + expressions.add(c); + } + + @Override + public int hashCode() + { + final int prime = 31; + int result = 1; + result = prime * result + ((expressions == null) ? 0 : expressions.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + PartitionSpec other = (PartitionSpec) obj; + if (expressions == null) + { + if (other.expressions != null) { + return false; + } + } + else if (!expressions.equals(other.expressions)) { + return false; + } + return true; + } + + @Override + public String toString() + { + return String.format("partitionColumns=%s",PTFUtils.toString(expressions)); + } + } + + public static class PartitionExpression + { + ASTNode expression; + + public PartitionExpression() {} + + public PartitionExpression(PartitionExpression peSpec) + { + expression = peSpec.getExpression(); + } + + public ASTNode getExpression() { + return expression; + } + + public void setExpression(ASTNode expression) { + this.expression = expression; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((expression == null) ? 0 : expression.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + PartitionExpression other = (PartitionExpression) obj; + if (expression == null) { + if (other.expression != null) { + return false; + } + } else if (!expression.toStringTree().equals(other.expression.toStringTree())) { + return false; + } + return true; + } + + @Override + public String toString() + { + return expression.toStringTree(); + } + + } + + /* + * Captures how the Input should be Ordered. This is captured as a list + * of ASTNodes that are the expressions in the Sort By clause in a + * PTF invocation. + */ + public static class OrderSpec + { + ArrayList expressions; + + public OrderSpec() {} + + public OrderSpec(PartitionSpec pSpec) + { + for(PartitionExpression peSpec : pSpec.getExpressions()) + { + addExpression(new OrderExpression(peSpec)); + } + } + + public ArrayList getExpressions() + { + return expressions; + } + + public void setExpressions(ArrayList columns) + { + this.expressions = columns; + } + + public void addExpression(OrderExpression c) + { + expressions = expressions == null ? new ArrayList() : expressions; + expressions.add(c); + } + + @Override + public int hashCode() + { + final int prime = 31; + int result = 1; + result = prime * result + ((expressions == null) ? 0 : expressions.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + OrderSpec other = (OrderSpec) obj; + if (expressions == null) + { + if (other.expressions != null) { + return false; + } + } + else if (!expressions.equals(other.expressions)) { + return false; + } + return true; + } + + @Override + public String toString() + { + return String.format("orderColumns=%s",PTFUtils.toString(expressions)); + } + } + + public static enum Order + { + ASC, + DESC; + } + + public static class OrderExpression extends PartitionExpression + { + Order order; + + public OrderExpression() {} + + public OrderExpression(PartitionExpression peSpec) + { + super(peSpec); + order = Order.ASC; + } + + public Order getOrder() + { + return order; + } + + public void setOrder(Order order) + { + this.order = order; + } + + @Override + public int hashCode() + { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((order == null) ? 0 : order.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + OrderExpression other = (OrderExpression) obj; + if (order != other.order) { + return false; + } + return true; + } + + @Override + public String toString() + { + return String.format("%s %s", super.toString(), order); + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java new file mode 100644 index 0000000..c25d53d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java @@ -0,0 +1,1503 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Stack; + +import org.antlr.runtime.tree.TreeWizard; +import org.antlr.runtime.tree.TreeWizard.ContextVisitor; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.exec.WindowFunctionInfo; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderExpression; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFInputSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputType; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionExpression; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionedTableFunctionSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitioningSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.CurrentRowSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.RangeBoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.ValueBoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowExpressionSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFrameSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFunctionSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowSpec; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.BoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.CurrentRowDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.OrderDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.OrderExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFQueryInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.RangeBoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.ShapeDetails; +import org.apache.hadoop.hive.ql.plan.PTFDesc.ValueBoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowFrameDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowTableFunctionDef; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionEvaluator; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver; +import org.apache.hadoop.hive.ql.udf.ptf.WindowingTableFunction.WindowingTableFunctionResolver; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; + +public class PTFTranslator { + + private static final Log LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.parse"); + + HiveConf hCfg; + LeadLagInfo llInfo; + SemanticAnalyzer semAly; + UnparseTranslator unparseT; + RowResolver inputRR; + PTFDesc ptfDesc; + PTFInvocationSpec ptfInvocation; + WindowingSpec windowingSpec; + + private void init(SemanticAnalyzer semAly, + HiveConf hCfg, + RowResolver inputRR, + UnparseTranslator unparseT) { + this.semAly = semAly; + this.hCfg = hCfg; + this.inputRR = inputRR; + this.unparseT = unparseT; + llInfo = new LeadLagInfo(); + + } + + public PTFDesc translate(PTFInvocationSpec qSpec, + SemanticAnalyzer semAly, + HiveConf hCfg, + RowResolver inputRR, + UnparseTranslator unparseT) + throws SemanticException { + init(semAly, hCfg, inputRR, unparseT); + this.ptfInvocation = qSpec; + ptfDesc = new PTFDesc(); + ptfDesc.setLlInfo(llInfo); + translatePTFChain(); + return ptfDesc; + } + + public PTFDesc translate(WindowingSpec wdwSpec, SemanticAnalyzer semAly, HiveConf hCfg, RowResolver inputRR, + UnparseTranslator unparseT) + throws SemanticException { + init(semAly, hCfg, inputRR, unparseT); + this.windowingSpec = wdwSpec; + ptfDesc = new PTFDesc(); + ptfDesc.setLlInfo(llInfo); + WindowTableFunctionDef wdwTFnDef = new WindowTableFunctionDef(); + ptfDesc.setFuncDef(wdwTFnDef); + + PTFQueryInputSpec inpSpec = new PTFQueryInputSpec(); + inpSpec.setType(PTFQueryInputType.WINDOWING); + wdwTFnDef.setInput(translate(inpSpec, 0)); + ShapeDetails inpShape = wdwTFnDef.getInput().getOutputShape(); + + WindowingTableFunctionResolver tFn = (WindowingTableFunctionResolver) + FunctionRegistry.getTableFunctionResolver(FunctionRegistry.WINDOWING_TABLE_FUNCTION); + if (tFn == null) { + throw new SemanticException(String.format("INternal Error: Unknown Table Function %s", + FunctionRegistry.WINDOWING_TABLE_FUNCTION)); + } + wdwTFnDef.setName(FunctionRegistry.WINDOWING_TABLE_FUNCTION); + wdwTFnDef.setResolverClassName(tFn.getClass().getName()); + wdwTFnDef.setAlias("ptf_" + 1 ); + wdwTFnDef.setExpressionTreeString(null); + wdwTFnDef.setTransformsRawInput(false); + tFn.initialize(hCfg, ptfDesc, wdwTFnDef); + TableFunctionEvaluator tEval = tFn.getEvaluator(); + wdwTFnDef.setTFunction(tEval); + wdwTFnDef.setCarryForwardNames(tFn.carryForwardNames()); + wdwTFnDef.setRawInputShape(inpShape); + + PartitioningSpec partiSpec = wdwSpec.getQueryPartitioningSpec(); + if ( partiSpec == null ) { + throw new SemanticException( + "Invalid use of Windowing: there is no Partitioning associated with Windowing"); + } + PartitionDef partDef = translate(inpShape, wdwSpec.getQueryPartitionSpec()); + OrderDef ordDef = translate(inpShape, wdwSpec.getQueryOrderSpec(), partDef); + + wdwTFnDef.setPartition(partDef); + wdwTFnDef.setOrder(ordDef); + + /* + * process Wdw functions + */ + ArrayList windowFunctions = new ArrayList(); + if ( wdwSpec.getWindowExpressions() != null ) { + for(WindowExpressionSpec expr : wdwSpec.getWindowExpressions()) { + if ( expr instanceof WindowFunctionSpec) { + WindowFunctionDef wFnDef = translate(wdwTFnDef, (WindowFunctionSpec)expr); + windowFunctions.add(wFnDef); + } + } + wdwTFnDef.setWindowFunctions(windowFunctions); + } + + /* + * set outputFromWdwFnProcessing + */ + if ( windowFunctions.size() > 0 ) { + ArrayList aliases = new ArrayList(); + ArrayList fieldOIs = new ArrayList(); + for(WindowFunctionDef wFnDef : windowFunctions) { + aliases.add(wFnDef.getAlias()); + if ( wFnDef.isPivotResult() ) { + fieldOIs.add(((ListObjectInspector)wFnDef.getOI()).getListElementObjectInspector()); + } else { + fieldOIs.add(wFnDef.getOI()); + } + } + PTFTranslator.addInputColumnsToList(inpShape, aliases, fieldOIs); + StructObjectInspector wdwOutOI = ObjectInspectorFactory.getStandardStructObjectInspector( + aliases, fieldOIs); + tFn.setWdwProcessingOutputOI(wdwOutOI); + RowResolver wdwOutRR = buildRowResolverForWindowing(wdwTFnDef, false); + ShapeDetails wdwOutShape = setupShape(wdwOutOI, null, wdwOutRR); + wdwTFnDef.setOutputFromWdwFnProcessing(wdwOutShape); + } + else { + wdwTFnDef.setOutputFromWdwFnProcessing(inpShape); + } + + /* + * process Wdw expressions + */ + ShapeDetails wdwOutShape = wdwTFnDef.getOutputFromWdwFnProcessing(); + ArrayList windowExpressions = new ArrayList(); + if ( wdwSpec.getWindowExpressions() != null ) { + for(WindowExpressionSpec expr : wdwSpec.getWindowExpressions()) { + if ( !(expr instanceof WindowFunctionSpec) ) { + try { + PTFExpressionDef eDef = buildExpressionDef(wdwOutShape, expr.getExpression()); + WindowExpressionDef wdwEDef = new WindowExpressionDef(eDef); + wdwEDef.setAlias(expr.getAlias()); + windowExpressions.add(wdwEDef); + } + catch(HiveException he) { + throw new SemanticException(he); + } + } + } + wdwTFnDef.setWindowExpressions(windowExpressions); + } + + /* + * set outputOI + */ + if ( windowExpressions.size() > 0 ) { + ArrayList aliases = new ArrayList(); + ArrayList fieldOIs = new ArrayList(); + for(WindowExpressionDef wEDef : windowExpressions) { + aliases.add(wEDef.getAlias()); + fieldOIs.add(wEDef.getOI()); + } + PTFTranslator.addInputColumnsToList(wdwOutShape, aliases, fieldOIs); + StructObjectInspector outOI = ObjectInspectorFactory.getStandardStructObjectInspector( + aliases, fieldOIs); + RowResolver outRR = buildRowResolverForWindowing(wdwTFnDef, true); + ShapeDetails outShape = setupShape(outOI, null, outRR); + wdwTFnDef.setOutputShape(outShape); + } + else { + wdwTFnDef.setOutputShape(copyShape(wdwOutShape)); + } + + /* + * translate having + */ + if ( wdwSpec.getFilterExpr() != null ) { + PTFExpressionDef hvEDef; + try { + hvEDef = buildExpressionDef(wdwOutShape, wdwSpec.getFilterExpr()); + wdwTFnDef.setHavingExpression(hvEDef); + } + catch(HiveException he) { + throw new SemanticException(he); + } + } + + tFn.setupOutputOI(); + + /* + * If we have windowExpressions or having then we convert to Std. Object to process; + * we just stream these rows; no need to put in an output Partition. + */ + if ( windowExpressions.size() > 0 || wdwSpec.getFilterExpr() != null ) { + StructObjectInspector oi = (StructObjectInspector) + ObjectInspectorUtils.getStandardObjectInspector(wdwTFnDef.getOutputShape().getOI()); + wdwTFnDef.getOutputShape().setOI(oi); + } + + return ptfDesc; + } + + private void translatePTFChain() throws SemanticException { + + Stack ptfChain = new Stack(); + PTFInputSpec currentSpec = ptfInvocation.getFunction(); + while (currentSpec != null ) { + ptfChain.push(currentSpec); + currentSpec = currentSpec.getInput(); + } + + int inputNum = 0; + PTFInputDef currentDef = null; + while ( !ptfChain.isEmpty() ) { + currentSpec = ptfChain.pop(); + + if ( currentSpec instanceof PTFQueryInputSpec) { + currentDef = translate((PTFQueryInputSpec) currentSpec, inputNum); + } + else { + currentDef = translate((PartitionedTableFunctionSpec) currentSpec, + currentDef, + inputNum); + } + inputNum++; + } + ptfDesc.setFuncDef((PartitionedTableFunctionDef)currentDef); + } + + private PTFQueryInputDef translate(PTFQueryInputSpec spec, + int inpNum) throws SemanticException + { + PTFQueryInputDef def = new PTFQueryInputDef(); + StructObjectInspector oi = PTFTranslator.getStandardStructOI(inputRR); + ShapeDetails shp = setupShape(oi, null, inputRR); + def.setOutputShape(shp); + def.setType(spec.getType()); + def.setAlias(spec.getSource() == null ? "ptf_" + inpNum : spec.getSource()); + return def; + } + + private PartitionedTableFunctionDef translate(PartitionedTableFunctionSpec spec, + PTFInputDef inpDef, + int inpNum) + throws SemanticException { + TableFunctionResolver tFn = FunctionRegistry.getTableFunctionResolver(spec.getName()); + if (tFn == null) { + throw new SemanticException(String.format("Unknown Table Function %s", + spec.getName())); + } + PartitionedTableFunctionDef def = new PartitionedTableFunctionDef(); + def.setInput(inpDef); + def.setName(spec.getName()); + def.setResolverClassName(tFn.getClass().getName()); + def.setAlias(spec.getAlias() == null ? "ptf_" + inpNum : spec.getAlias()); + def.setExpressionTreeString(spec.getAstNode().toStringTree()); + def.setTransformsRawInput(tFn.transformsRawInput()); + /* + * translate args + */ + ArrayList args = spec.getArgs(); + if (args != null) + { + for (ASTNode expr : args) + { + PTFExpressionDef argDef = null; + try { + argDef = buildExpressionDef(inpDef.getOutputShape(), expr); + } + catch(HiveException he) { + throw new SemanticException(he); + } + def.addArg(argDef); + } + } + + tFn.initialize(hCfg, ptfDesc, def); + TableFunctionEvaluator tEval = tFn.getEvaluator(); + def.setTFunction(tEval); + def.setCarryForwardNames(tFn.carryForwardNames()); + tFn.setupRawInputOI(); + + if ( tFn.transformsRawInput() ) { + StructObjectInspector rawInOutOI = tEval.getRawInputOI(); + ArrayList rawInOutColNames = tFn.getRawInputColumnNames(); + RowResolver rawInRR = buildRowResolverForPTF(def.getName(), + spec.getAlias(), + rawInOutOI, + rawInOutColNames, + inpDef.getOutputShape().getRr()); + ShapeDetails rawInpShape = setupTableFnShape(def.getName(), + inpDef.getOutputShape(), + rawInOutOI, + rawInOutColNames, + rawInRR); + def.setRawInputShape(rawInpShape); + } + else { + def.setRawInputShape(inpDef.getOutputShape()); + } + + translatePartitioning(def, spec); + tFn.setupOutputOI(); + + StructObjectInspector outputOI = tEval.getOutputOI(); + ArrayList outColNames = tFn.getOutputColumnNames(); + RowResolver outRR = buildRowResolverForPTF(def.getName(), + spec.getAlias(), + outputOI, + outColNames, + def.getRawInputShape().getRr()); + ShapeDetails outputShape = setupTableFnShape(def.getName(), + inpDef.getOutputShape(), + outputOI, + outColNames, + outRR); + def.setOutputShape(outputShape); + + return def; + } + + private WindowFunctionDef translate(WindowTableFunctionDef wdwTFnDef, + WindowFunctionSpec spec) throws SemanticException { + WindowFunctionInfo wFnInfo = FunctionRegistry.getWindowFunctionInfo(spec.getName()); + WindowFunctionDef def = new WindowFunctionDef(); + def.setName(spec.getName()); + def.setAlias(spec.getAlias()); + def.setDistinct(spec.isDistinct()); + def.setExpressionTreeString(spec.getExpression().toStringTree()); + def.setStar(spec.isStar()); + def.setPivotResult(wFnInfo.isPivotResult()); + ShapeDetails inpShape = wdwTFnDef.getRawInputShape(); + + /* + * translate args + */ + ArrayList args = spec.getArgs(); + if (args != null) + { + for (ASTNode expr : args) + { + PTFExpressionDef argDef = null; + try { + argDef = buildExpressionDef(inpShape, expr); + } + catch(HiveException he) { + throw new SemanticException(he); + } + def.addArg(argDef); + } + } + + if (RANKING_FUNCS.contains(spec.getName())) + { + setupRankingArgs(wdwTFnDef, def, spec); + } + + WindowSpec wdwSpec = spec.getWindowSpec(); + if ( wdwSpec != null ) { + String desc = spec.toString(); + + WindowFrameDef wdwFrame = translate(spec.getName(), inpShape, wdwSpec); + if (!wFnInfo.isSupportsWindow() && wdwFrame != null) + { + throw new SemanticException(String.format("Function %s doesn't support windowing", + desc)); + } + def.setWindowFrame(wdwFrame); + } + + try { + setupWdwFnEvaluator(def); + } + catch(HiveException he) { + throw new SemanticException(he); + } + + return def; + } + + private void translatePartitioning(PartitionedTableFunctionDef def, + PartitionedTableFunctionSpec spec) + throws SemanticException { + + if ( spec.getPartition() == null ) { + return; + } + PartitionDef partDef = translate(def.getRawInputShape(), spec.getPartition()); + OrderDef orderDef = translate(def.getRawInputShape(), spec.getOrder(), partDef); + def.setPartition(partDef); + def.setOrder(orderDef); + } + + private PartitionDef translate(ShapeDetails inpShape, PartitionSpec spec) + throws SemanticException + { + if (spec == null || spec.getExpressions() == null || spec.getExpressions().size() == 0) { + return null; + } + + PartitionDef pDef = new PartitionDef(); + for (PartitionExpression pExpr : spec.getExpressions()) + { + PTFExpressionDef expDef = translate(inpShape, pExpr); + pDef.addExpression(expDef); + } + return pDef; + } + + private PTFExpressionDef translate(ShapeDetails inpShape, + PartitionExpression pExpr) throws SemanticException + { + PTFExpressionDef expDef = null; + try { + expDef = buildExpressionDef(inpShape, pExpr.getExpression()); + } + catch(HiveException he) { + throw new SemanticException(he); + } + PTFTranslator.validateComparable(expDef.getOI(), + String.format("Partition Expression %s is not a comparable expression", pExpr.getExpression().toStringTree())); + return expDef; + } + + private OrderDef translate(ShapeDetails inpShape, + OrderSpec spec, + PartitionDef partitionDef) throws SemanticException { + + if (spec == null || spec.getExpressions() == null || spec.getExpressions().size() == 0) + { + if (partitionDef == null) { + return null; + } + return new OrderDef(partitionDef); + } + + if (partitionDef == null) { + throw new SemanticException(String.format( + "Cannot have an Order spec (%s) w/o a Partition spec", spec)); + } + + OrderDef def = new OrderDef(); + for (OrderExpression oExpr : spec.getExpressions()) + { + OrderExpressionDef oexpDef = translate(inpShape, oExpr); + def.addExpression(oexpDef); + } + + /* + * either all partition expressions must be in Order list or none must be specified. + * If none are specified then add them all. + */ + int numOfPartColumns = 0; + List orderExprs = def.getExpressions(); + List partExprs = partitionDef.getExpressions(); + int chkSize = partExprs.size(); + chkSize = chkSize > orderExprs.size() ? orderExprs.size() : chkSize; + for (int i = 0; i < chkSize; i++) + { + if (orderExprs.get(i).getExpressionTreeString() + .equals(partExprs.get(i).getExpressionTreeString())) + { + numOfPartColumns++; + } else { + break; + } + } + + if (numOfPartColumns != 0 && numOfPartColumns != partExprs.size()) + { + throw new SemanticException( + String.format( + "all partition columns must be in order clause(%s) or none should be specified", + spec)); + } + + if (numOfPartColumns == 0) + { + ArrayList combinedOrderCols = new ArrayList(); + for (PTFExpressionDef eDef : partExprs) + { + OrderExpressionDef oeDef = new OrderExpressionDef(eDef); + combinedOrderCols.add(oeDef); + } + combinedOrderCols.addAll(orderExprs); + def.setExpressions(combinedOrderCols); + } + + return def; + } + + private OrderExpressionDef translate(ShapeDetails inpShape, + OrderExpression oExpr) + throws SemanticException + { + OrderExpressionDef oexpDef = new OrderExpressionDef(); + oexpDef.setOrder(oExpr.getOrder()); + try { + PTFExpressionDef expDef = buildExpressionDef(inpShape, oExpr.getExpression()); + oexpDef.setExpressionTreeString(expDef.getExpressionTreeString()); + oexpDef.setExprEvaluator(expDef.getExprEvaluator()); + oexpDef.setExprNode(expDef.getExprNode()); + oexpDef.setOI(expDef.getOI()); + } + catch(HiveException he) { + throw new SemanticException(he); + } + PTFTranslator.validateComparable(oexpDef.getOI(), + String.format("Partition Expression %s is not a comparable expression", + oExpr.getExpression().toStringTree())); + return oexpDef; + } + + private WindowFrameDef translate(String wFnName, ShapeDetails inpShape, WindowSpec spec) throws SemanticException { + ArrayList sources = new ArrayList(); + fillInWindowSpec(spec.getSourceId(), spec, sources); + PartitionSpec pSpec = spec.getPartition(); + OrderSpec oSpec = spec.getOrder(); + PartitionDef pDef = translate(inpShape, pSpec); + OrderDef oDef = translate(inpShape, oSpec, pDef); + validatePartitioningForWdwFn(wFnName, pDef, oDef); + return translate(inpShape, spec.getWindowFrame()); + } + + private WindowFrameDef translate(ShapeDetails inpShape, + WindowFrameSpec spec) + throws SemanticException { + if (spec == null) + { + return null; + } + + BoundarySpec s = spec.getStart(); + BoundarySpec e = spec.getEnd(); + int cmp = s.compareTo(e); + if (cmp > 0) + { + throw new SemanticException(String.format( + "Window range invalid, start boundary is greater than end boundary: %s", spec)); + } + + WindowFrameDef wfDef = new WindowFrameDef(); + wfDef.setStart(translate(inpShape, s)); + wfDef.setEnd(translate(inpShape, e)); + return wfDef; + } + + private BoundaryDef translate(ShapeDetails inpShape, BoundarySpec bndSpec) + throws SemanticException { + if (bndSpec instanceof ValueBoundarySpec) + { + ValueBoundarySpec vBndSpec = (ValueBoundarySpec) bndSpec; + ValueBoundaryDef vbDef = new ValueBoundaryDef(); + vbDef.setAmt(vBndSpec.getAmt()); + vbDef.setDirection(vBndSpec.getDirection()); + PTFTranslator.validateNoLeadLagInValueBoundarySpec(vBndSpec.getExpression()); + PTFExpressionDef exprDef = null; + try { + exprDef = buildExpressionDef(inpShape, vBndSpec.getExpression()); + } + catch(HiveException he) { + throw new SemanticException(he); + } + PTFTranslator.validateValueBoundaryExprType(exprDef.getOI()); + vbDef.setExpressionDef(exprDef); + return vbDef; + } + else if (bndSpec instanceof RangeBoundarySpec) + { + RangeBoundarySpec rBndSpec = (RangeBoundarySpec) bndSpec; + RangeBoundaryDef rbDef = new RangeBoundaryDef(); + rbDef.setAmt(rBndSpec.getAmt()); + rbDef.setDirection(rBndSpec.getDirection()); + return rbDef; + } + else if (bndSpec instanceof CurrentRowSpec) + { + CurrentRowDef cbDef = new CurrentRowDef(); + return cbDef; + } + throw new SemanticException("Unknown Boundary: " + bndSpec); + } + + static void setupWdwFnEvaluator(WindowFunctionDef def) throws HiveException + { + ArrayList args = def.getArgs(); + ArrayList argOIs = new ArrayList(); + ObjectInspector[] funcArgOIs = null; + + if (args != null) { + for (PTFExpressionDef arg : args) { + argOIs.add(arg.getOI()); + } + funcArgOIs = new ObjectInspector[args.size()]; + funcArgOIs = argOIs.toArray(funcArgOIs); + } + + GenericUDAFEvaluator wFnEval = FunctionRegistry.getGenericWindowingEvaluator(def.getName(), argOIs, + def.isDistinct(), def.isStar()); + ObjectInspector OI = wFnEval.init(GenericUDAFEvaluator.Mode.COMPLETE, funcArgOIs); + def.setWFnEval(wFnEval); + def.setOI(OI); + } + + private void fillInWindowSpec(String sourceId, WindowSpec dest, ArrayList visited) + throws SemanticException + { + if (sourceId != null) + { + if ( visited.contains(sourceId)) { + visited.add(sourceId); + throw new SemanticException(String.format("Cycle in Window references %s", visited)); + } + WindowSpec source = windowingSpec.getWindowSpecs().get(sourceId); + if (source == null || source.equals(dest)) + { + throw new SemanticException(String.format("Window Spec %s refers to an unknown source " , + dest)); + } + + if (dest.getPartition() == null) + { + dest.setPartition(source.getPartition()); + } + + if (dest.getOrder() == null) + { + dest.setOrder(source.getOrder()); + } + + if (dest.getWindowFrame() == null) + { + dest.setWindowFrame(source.getWindowFrame()); + } + + visited.add(sourceId); + + fillInWindowSpec(source.getSourceId(), dest, visited); + } + } + + private static void validateValueBoundaryExprType(ObjectInspector OI) + throws SemanticException + { + if (!OI.getCategory().equals(Category.PRIMITIVE)) + { + throw new SemanticException( + "Value Boundary expression must be of primitve type"); + } + + PrimitiveObjectInspector pOI = (PrimitiveObjectInspector) OI; + PrimitiveCategory pC = pOI.getPrimitiveCategory(); + + switch (pC) + { + case BYTE: + case DOUBLE: + case FLOAT: + case INT: + case LONG: + case SHORT: + case DECIMAL: + case TIMESTAMP: + break; + default: + throw new SemanticException( + String.format("Primitve type %s not supported in Value Boundary expression", + pC)); + } + + } + + private void validatePartitioningForWdwFn(String wFnName, PartitionDef fPart, OrderDef fOrder) + throws SemanticException + { + PartitionDef tPart = ptfDesc.getFuncDef().getPartition(); + + if (!PTFTranslator.isCompatible(tPart, fPart)) + { + throw new SemanticException( + String.format("Window Function '%s' has an incompatible partition clause", wFnName)); + } + + OrderDef tOrder = ptfDesc.getFuncDef().getOrder(); + if (!PTFTranslator.isCompatible(tOrder, fOrder)) + { + throw new SemanticException( + String.format("Window Function '%s' has an incompatible order clause", wFnName)); + } + } + + private ShapeDetails setupTableFnShape(String fnName, ShapeDetails inpShape, StructObjectInspector OI, ArrayList columnNames, RowResolver rr) + throws SemanticException { + if (fnName.equals(FunctionRegistry.NOOP_TABLE_FUNCTION) + || fnName.equals( + FunctionRegistry.NOOP_MAP_TABLE_FUNCTION)) + { + return setupShapeForNoop(inpShape, OI, columnNames, rr); + } + return setupShape(OI, columnNames, rr); + } + + private ShapeDetails setupShape(StructObjectInspector OI, + ArrayList columnNames, + RowResolver rr) throws SemanticException { + Map serdePropsMap = new LinkedHashMap(); + SerDe serde = null; + ShapeDetails shp = new ShapeDetails(); + + try { + serde = PTFTranslator.createLazyBinarySerDe(hCfg, OI, serdePropsMap); + shp.setOI((StructObjectInspector)serde.getObjectInspector()); + } + catch(SerDeException se) { + throw new SemanticException(se); + } + + shp.setRr(rr); + shp.setSerde(serde); + shp.setSerdeClassName(serde.getClass().getName()); + shp.setSerdeProps(serdePropsMap); + shp.setColumnNames(columnNames); + + TypeCheckCtx tCtx = new TypeCheckCtx(rr); + tCtx.setUnparseTranslator(unparseT); + shp.setTypeCheckCtx(tCtx); + + return shp; + } + + private ShapeDetails copyShape(ShapeDetails src) { + ShapeDetails dest = new ShapeDetails(); + dest.setSerdeClassName(src.getSerdeClassName()); + dest.setSerdeProps(src.getSerdeProps()); + dest.setColumnNames(src.getColumnNames()); + dest.setOI(src.getOI()); + dest.setSerde(src.getSerde()); + dest.setRr(src.getRr()); + dest.setTypeCheckCtx(src.getTypeCheckCtx()); + + return dest; + } + + private ShapeDetails setupShapeForNoop(ShapeDetails inpShape, + StructObjectInspector OI, + ArrayList columnNames, + RowResolver rr) throws SemanticException { + ShapeDetails shp = new ShapeDetails(); + + shp.setRr(rr); + shp.setOI(inpShape.getOI()); + shp.setSerde(inpShape.getSerde()); + shp.setSerdeClassName(inpShape.getSerde().getClass().getName()); + shp.setSerdeProps(inpShape.getSerdeProps()); + shp.setColumnNames(columnNames); + + TypeCheckCtx tCtx = new TypeCheckCtx(rr); + tCtx.setUnparseTranslator(unparseT); + shp.setTypeCheckCtx(tCtx); + + return shp; + } + + protected static ArrayList addPartitionExpressionsToOrderList( + ArrayList partCols, + ArrayList orderCols) throws SemanticException { + int numOfPartColumns = 0; + int chkSize = partCols.size(); + + chkSize = chkSize > orderCols.size() ? orderCols.size() : chkSize; + for (int i = 0; i < chkSize; i++) { + if (orderCols.get(i).getExpression().toStringTree() + .equals(partCols.get(i).getExpression().toStringTree())) { + numOfPartColumns++; + } else { + break; + } + } + + if (numOfPartColumns != 0 && numOfPartColumns != partCols.size()) { + List partitionColumnNames = new ArrayList(); + for(PartitionExpression partitionExpression : partCols) { + ASTNode column = partitionExpression.getExpression(); + if(column != null && column.getChildCount() > 0) { + partitionColumnNames.add(column.getChild(0).getText()); + } + } + throw new SemanticException( + String.format( + "all partition columns %s must be in order clause or none should be specified", + partitionColumnNames.toString())); + } + ArrayList combinedOrdExprs = new ArrayList(); + if (numOfPartColumns == 0) + { + for (PartitionExpression partCol : partCols) + { + OrderExpression orderCol = new OrderExpression(partCol); + combinedOrdExprs.add(orderCol); + } + } + combinedOrdExprs.addAll(orderCols); + return combinedOrdExprs; + } + + + /* + * Ranking Functions helpers + */ + + protected static final ArrayList RANKING_FUNCS = new ArrayList(); + static + { + RANKING_FUNCS.add("rank"); + RANKING_FUNCS.add("dense_rank"); + RANKING_FUNCS.add("percent_rank"); + RANKING_FUNCS.add("cume_dist"); + }; + + private void setupRankingArgs(WindowTableFunctionDef wdwTFnDef, + WindowFunctionDef wFnDef, + WindowFunctionSpec wSpec) + throws SemanticException + { + if (wSpec.getArgs().size() > 0) + { + throw new SemanticException("Ranking Functions can take no arguments"); + } + + OrderDef oDef = wdwTFnDef.getOrder(); + ArrayList oExprs = oDef.getExpressions(); + for (OrderExpressionDef oExpr : oExprs) + { + wFnDef.addArg(oExpr); + } + } + + /* + * Expr translation helper methods + */ + public PTFExpressionDef buildExpressionDef(ShapeDetails inpShape, ASTNode arg) + throws HiveException + { + PTFExpressionDef argDef = new PTFExpressionDef(); + + ExprNodeDesc exprNode = semAly.genExprNodeDesc(arg, inpShape.getRr(), + inpShape.getTypeCheckCtx()); + ExprNodeEvaluator exprEval = WindowingExprNodeEvaluatorFactory.get(llInfo, exprNode); + ObjectInspector oi = initExprNodeEvaluator(exprEval, exprNode, inpShape); + + argDef.setExpressionTreeString(arg.toStringTree()); + argDef.setExprNode(exprNode); + argDef.setExprEvaluator(exprEval); + argDef.setOI(oi); + return argDef; + } + + private ObjectInspector initExprNodeEvaluator(ExprNodeEvaluator exprEval, + ExprNodeDesc exprNode, + ShapeDetails inpShape) + throws HiveException + { + ObjectInspector outOI; + outOI = exprEval.initialize(inpShape.getOI()); + + /* + * if there are any LeadLag functions in this Expression Tree: - setup a + * duplicate Evaluator for the 1st arg of the LLFuncDesc - initialize it + * using the InputInfo provided for this Expr tree - set the duplicate + * evaluator on the LLUDF instance. + */ + List llFuncExprs = llInfo.getLLFuncExprsInTopExpr(exprNode); + if (llFuncExprs != null) + { + for (ExprNodeGenericFuncDesc llFuncExpr : llFuncExprs) + { + ExprNodeDesc firstArg = llFuncExpr.getChildren().get(0); + ExprNodeEvaluator dupExprEval = WindowingExprNodeEvaluatorFactory.get(llInfo, firstArg); + dupExprEval.initialize(inpShape.getOI()); + GenericUDFLeadLag llFn = (GenericUDFLeadLag) llFuncExpr.getGenericUDF(); + llFn.setExprEvaluator(dupExprEval); + } + } + + return outOI; + } + + /* + * Partition & Order expression compatibility methods + */ + + protected static boolean isCompatible(PartitionDef p1, PartitionDef p2) { + if (p1 == null && p2 == null) { + return true; + } + if (p1 == null && p2 != null) { + return false; + } + if (p1 != null && p2 == null) { + return true; + } + + ArrayList p1Exprs = p1.getExpressions(); + ArrayList p2Exprs = p2.getExpressions(); + if (p1Exprs.size() != p2Exprs.size()) { + return false; + } + for (int i = 0; i < p1Exprs.size(); i++) + { + boolean e = isEqual(p1Exprs.get(i), p2Exprs.get(i)); + if (!e) { + return false; + } + } + return true; + } + + protected static boolean isEqual(PTFExpressionDef e1, PTFExpressionDef e2) { + if (e1 == null && e2 == null) { + return false; + } + if (e1 == null && e2 != null) { + return false; + } + if (e1 != null && e2 == null) { + return false; + } + + return e1.getExpressionTreeString().equals(e2.getExpressionTreeString()); + } + + protected static boolean isCompatible(OrderDef o1, OrderDef o2) { + if (o1 == null && o2 == null) { + return true; + } + if (o1 == null && o2 != null) { + return false; + } + if (o1 != null && o2 == null) { + return true; + } + + ArrayList o1Exprs = o1.getExpressions(); + ArrayList o2Exprs = o2.getExpressions(); + if (o1Exprs.size() != o2Exprs.size()) { + return false; + } + for (int i = 0; i < o1Exprs.size(); i++) + { + boolean e = isEqual(o1Exprs.get(i), o2Exprs.get(i)); + if (!e) { + return false; + } + } + return true; + } + + protected static boolean isEqual(OrderExpressionDef e1, OrderExpressionDef e2) { + boolean e = isEqual((PTFExpressionDef)e1, (PTFExpressionDef)e2); + if (!e) { + return false; + } + return e1.getOrder().equals(e2.getOrder()); + } + + /* + * OI & Serde helper methods + */ + + protected static SerDe createLazyBinarySerDe(Configuration cfg, + StructObjectInspector oi, Map serdePropsMap) throws SerDeException { + serdePropsMap = serdePropsMap == null ? new LinkedHashMap() : serdePropsMap; + + addOIPropertiestoSerDePropsMap(oi, serdePropsMap); + + SerDe serDe = new LazyBinarySerDe(); + Properties p = new Properties(); + p.setProperty(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS, + serdePropsMap.get(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS)); + p.setProperty( + org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES, + serdePropsMap.get(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES)); + serDe.initialize(cfg, p); + return serDe; + } + + @SuppressWarnings({"unchecked"}) + private static void addOIPropertiestoSerDePropsMap(StructObjectInspector OI, + Map serdePropsMap) { + + if ( serdePropsMap == null ) { + return; + } + + ArrayList[] tInfo = getTypeMap(OI); + + ArrayList columnNames = (ArrayList) tInfo[0]; + ArrayList fields = (ArrayList) tInfo[1]; + StringBuilder cNames = new StringBuilder(); + StringBuilder cTypes = new StringBuilder(); + + for (int i = 0; i < fields.size(); i++) + { + cNames.append(i > 0 ? "," : ""); + cTypes.append(i > 0 ? "," : ""); + cNames.append(columnNames.get(i)); + cTypes.append(fields.get(i).getTypeName()); + } + + serdePropsMap.put(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS, + cNames.toString()); + serdePropsMap.put(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES, + cTypes.toString()); + } + + private static ArrayList[] getTypeMap( + StructObjectInspector oi) { + StructTypeInfo t = (StructTypeInfo) TypeInfoUtils + .getTypeInfoFromObjectInspector(oi); + ArrayList fnames = t.getAllStructFieldNames(); + ArrayList fields = t.getAllStructFieldTypeInfos(); + return new ArrayList[] + { fnames, fields }; + } + + /** + * For each column on the input RR, construct a StructField for it + * OI is constructed using the list of input column names and + * their corresponding OIs. + * @param rr + * @return + */ + public static StructObjectInspector getStandardStructOI(RowResolver rr) { + StructObjectInspector oi; + ArrayList colLists = rr.getColumnInfos(); + ArrayList structFieldNames = new ArrayList(); + ArrayList structFieldObjectInspectors = new ArrayList(); + for (ColumnInfo columnInfo : colLists) { + String colName = columnInfo.getInternalName(); + ObjectInspector colOI = columnInfo.getObjectInspector(); + structFieldNames.add(colName); + structFieldObjectInspectors.add(colOI); + } + oi = ObjectInspectorFactory.getStandardStructObjectInspector(structFieldNames, + structFieldObjectInspectors); + return oi; + + } + + protected static void validateComparable(ObjectInspector OI, String errMsg) + throws SemanticException + { + if (!ObjectInspectorUtils.compareSupported(OI)) + { + throw new SemanticException(errMsg); + } + } + + private static void addInputColumnsToList(ShapeDetails shape, + ArrayList fieldNames, ArrayList fieldOIs) + { + StructObjectInspector OI = shape.getOI(); + for (StructField f : OI.getAllStructFieldRefs()) + { + fieldNames.add(f.getFieldName()); + fieldOIs.add(f.getFieldObjectInspector()); + } + } + + /* + * RowResolver helper methods + */ + + protected static RowResolver buildRowResolverForPTF(String tbFnName, String tabAlias, + StructObjectInspector rowObjectInspector, + ArrayList outputColNames, RowResolver inputRR) throws SemanticException { + + if ( tbFnName.equals(FunctionRegistry.NOOP_TABLE_FUNCTION) || + tbFnName.equals(FunctionRegistry.NOOP_MAP_TABLE_FUNCTION) ) { + return buildRowResolverForNoop(tabAlias, rowObjectInspector, inputRR); + } + + RowResolver rwsch = new RowResolver(); + List fields = rowObjectInspector.getAllStructFieldRefs(); + for (int i = 0; i < fields.size(); i++) + { + ColumnInfo colInfo = new ColumnInfo(fields.get(i).getFieldName(), + TypeInfoUtils.getTypeInfoFromObjectInspector(fields.get(i) + .getFieldObjectInspector()), + tabAlias, + false); + rwsch.put(tabAlias, outputColNames.get(i), colInfo); + } + return rwsch; + } + + protected RowResolver buildRowResolverForWindowing(WindowTableFunctionDef def, + boolean addWdwExprs) throws SemanticException { + RowResolver rr = new RowResolver(); + HashMap aliasToExprMap = windowingSpec.getAliasToWdwExpr(); + /* + * add Window Expressions + */ + if ( addWdwExprs ) { + for(WindowExpressionDef wEDef : def.getWindowExpressions() ) { + ASTNode ast = aliasToExprMap.get(wEDef.getAlias()).getExpression(); + ColumnInfo cInfo = new ColumnInfo(wEDef.getAlias(), + TypeInfoUtils.getTypeInfoFromObjectInspector(wEDef.getOI()), + null, + false); + rr.putExpression(ast, cInfo); + } + } + + /* + * add Window Functions + */ + for(WindowFunctionDef wFnDef : def.getWindowFunctions() ) { + ASTNode ast = aliasToExprMap.get(wFnDef.getAlias()).getExpression(); + ObjectInspector wFnOI = null; + if ( wFnDef.isPivotResult() ) { + wFnOI = ((ListObjectInspector)wFnDef.getOI()).getListElementObjectInspector(); + } + else { + wFnOI = wFnDef.getOI(); + } + ColumnInfo cInfo = new ColumnInfo(wFnDef.getAlias(), + TypeInfoUtils.getTypeInfoFromObjectInspector(wFnOI), + null, + false); + rr.putExpression(ast, cInfo); + } + + RowResolver inpRR = def.getRawInputShape().getRr(); + /* + * add columns from inpRR + */ + for (ColumnInfo inpCInfo : inputRR.getColumnInfos()) { + ColumnInfo cInfo = new ColumnInfo(inpCInfo); + String colAlias = cInfo.getAlias(); + + String[] tabColAlias = inputRR.reverseLookup(inpCInfo.getInternalName()); + if (tabColAlias != null) { + colAlias = tabColAlias[1]; + } + ASTNode inExpr = null; + inExpr = PTFTranslator.getASTNode(inpCInfo, inpRR); + if ( inExpr != null ) { + rr.putExpression(inExpr, cInfo); + } + else { + rr.put(cInfo.getTabAlias(), colAlias, cInfo); + } + } + + return rr; + } + + protected static RowResolver buildRowResolverForNoop(String tabAlias, + StructObjectInspector rowObjectInspector, + RowResolver inputRowResolver) throws SemanticException + { + LOG.info("QueryTranslationInfo::getRowResolver invoked on ObjectInspector"); + RowResolver rwsch = new RowResolver(); + List fields = rowObjectInspector.getAllStructFieldRefs(); + for (int i = 0; i < fields.size(); i++) + { + StructField field = fields.get(i); + String internalName = field.getFieldName(); + String[] tabColAlias = inputRowResolver == null ? null : inputRowResolver + .reverseLookup(internalName); + String colTabAlias = tabColAlias == null ? tabAlias : tabColAlias[0]; + String colAlias = tabColAlias == null ? null : tabColAlias[1]; + ColumnInfo inpColInfo; + ColumnInfo colInfo; + + if (tabColAlias != null) { + inpColInfo = inputRowResolver.get(colTabAlias, colAlias); + } + else { + /* + * for the Virtual columns: + * - the internalName is UPPER Case and the alias is lower case + * - since we put them in an OI, the fieldnames became lower cased. + * - so we look in the inputRR for the fieldName as an alias. + */ + inpColInfo = inputRowResolver == null ? null : inputRowResolver + .get(tabAlias, internalName); + colAlias = inpColInfo != null ? inpColInfo.getInternalName() : colAlias; + } + + if (inpColInfo != null) { + colInfo = new ColumnInfo(inpColInfo); + colInfo.setTabAlias(tabAlias); + } + else { + colInfo = new ColumnInfo(fields.get(i).getFieldName(), + TypeInfoUtils.getTypeInfoFromObjectInspector(fields.get(i) + .getFieldObjectInspector()), + tabAlias, + false); + colAlias = colInfo.getInternalName(); + } + + ASTNode expr = inputRowResolver == null ? null : PTFTranslator.getASTNode(inpColInfo, + inputRowResolver); + + if (expr != null) { + rwsch.putExpression(expr, colInfo); + } + else { + rwsch.put(tabAlias, colAlias, colInfo); + } + } + return rwsch; + } + + /* + * If the cInfo is for an ASTNode, this function returns the ASTNode that it is for. + */ + public static ASTNode getASTNode(ColumnInfo cInfo, RowResolver rr) throws SemanticException { + for(Map.Entry entry : rr.getExpressionMap().entrySet()) { + ASTNode expr = entry.getValue(); + if ( rr.getExpression(expr).equals(cInfo)) { + return expr; + } + } + return null; + } + + /* + * Utility to visit all nodes in an AST tree. + */ + public static void visit(Object t, ContextVisitor visitor) { + _visit(t, null, 0, visitor); + } + + /** Do the recursive work for visit */ + private static void _visit(Object t, Object parent, int childIndex, ContextVisitor visitor) { + if ( t==null ) { + return; + } + visitor.visit(t, parent, childIndex, null); + int n = ParseDriver.adaptor.getChildCount(t); + for (int i=0; i componentize(PTFInvocationSpec ptfInvocation) throws SemanticException { + + ArrayList componentInvocations = new ArrayList(); + + Stack ptfChain = new Stack(); + PTFInputSpec spec = ptfInvocation.getFunction(); + while (spec instanceof PartitionedTableFunctionSpec) { + ptfChain.push(spec); + spec = spec.getInput(); + } + + PartitionedTableFunctionSpec prevFn = (PartitionedTableFunctionSpec) ptfChain.pop(); + PartitionSpec partSpec = prevFn.getPartition(); + OrderSpec orderSpec = prevFn.getOrder(); + + if ( partSpec == null ) { + //oops this should have been caught before trying to componentize + throw new SemanticException( + "No Partitioning specification specified at start of a PTFChain"); + } + if ( orderSpec == null ) { + orderSpec = new OrderSpec(partSpec); + prevFn.setOrder(orderSpec); + } + + while (!ptfChain.isEmpty()) { + PartitionedTableFunctionSpec currentFn = (PartitionedTableFunctionSpec) ptfChain.pop(); + String fnName = currentFn.getName(); + if (!FunctionRegistry.isTableFunction(fnName)) { + throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(fnName)); + } + boolean transformsRawInput = FunctionRegistry.getTableFunctionResolver(fnName) + .transformsRawInput(); + + /* + * if the current table function has no partition info specified: inherit it from the PTF up + * the chain. + */ + if (currentFn.getPartition() == null) { + currentFn.setPartition(prevFn.getPartition()); + if (currentFn.getOrder() == null) { + currentFn.setOrder(prevFn.getOrder()); + } + } + /* + * If the current table function has no order info specified; + */ + if (currentFn.getOrder() == null) { + currentFn.setOrder(new OrderSpec(currentFn.getPartition())); + } + + if (!currentFn.getPartition().equals(partSpec) || + !currentFn.getOrder().equals(orderSpec) || + transformsRawInput) { + PTFInvocationSpec component = new PTFInvocationSpec(); + component.setFunction(prevFn); + componentInvocations.add(component); + PTFQueryInputSpec cQInSpec = new PTFQueryInputSpec(); + cQInSpec.setType(PTFQueryInputType.PTFCOMPONENT); + currentFn.setInput(cQInSpec); + } + + prevFn = currentFn; + partSpec = prevFn.getPartition(); + orderSpec = prevFn.getOrder(); + } + componentInvocations.add(ptfInvocation); + return componentInvocations; + } + + + /* + * Lead Lag functionality + */ + public static class LeadLagInfo { + /* + * list of LL invocations in a Query. + */ + List leadLagExprs; + /* + * map from the Select Expr Node to the LL Function invocations in it. + */ + Map> mapTopExprToLLFunExprs; + + private void addLeadLagExpr(ExprNodeGenericFuncDesc llFunc) + { + leadLagExprs = leadLagExprs == null ? new ArrayList() : leadLagExprs; + leadLagExprs.add(llFunc); + } + + public List getLeadLagExprs() + { + return leadLagExprs; + } + + public void addLLFuncExprForTopExpr(ExprNodeDesc topExpr, ExprNodeGenericFuncDesc llFuncExpr) + { + addLeadLagExpr(llFuncExpr); + mapTopExprToLLFunExprs = mapTopExprToLLFunExprs == null ? + new HashMap>() : mapTopExprToLLFunExprs; + List funcList = mapTopExprToLLFunExprs.get(topExpr); + if (funcList == null) + { + funcList = new ArrayList(); + mapTopExprToLLFunExprs.put(topExpr, funcList); + } + funcList.add(llFuncExpr); + } + + public List getLLFuncExprsInTopExpr(ExprNodeDesc topExpr) + { + if (mapTopExprToLLFunExprs == null) { + return null; + } + return mapTopExprToLLFunExprs.get(topExpr); + } + } + + public static void validateNoLeadLagInValueBoundarySpec(ASTNode node) + throws SemanticException + { + String errMsg = "Lead/Lag not allowed in ValueBoundary Spec"; + TreeWizard tw = new TreeWizard(ParseDriver.adaptor, HiveParser.tokenNames); + ValidateNoLeadLag visitor = new ValidateNoLeadLag(errMsg); + tw.visit(node, HiveParser.TOK_FUNCTION, visitor); + visitor.checkValid(); + } + + public static class ValidateNoLeadLag implements + ContextVisitor + { + String errMsg; + boolean throwError = false; + ASTNode errorNode; + + public ValidateNoLeadLag(String errMsg) + { + this.errMsg = errMsg; + } + + @SuppressWarnings("rawtypes") + @Override + public void visit(Object t, Object parent, int childIndex, Map labels) + { + ASTNode expr = (ASTNode) t; + ASTNode nameNode = (ASTNode) expr.getChild(0); + if (nameNode.getText().equals(FunctionRegistry.LEAD_FUNC_NAME) + || nameNode.getText() + .equals(FunctionRegistry.LAG_FUNC_NAME)) + { + throwError = true; + errorNode = expr; + } + } + + void checkValid() throws SemanticException + { + if (throwError) + { + throw new SemanticException(errMsg + errorNode.toStringTree()); + } + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java index 33815a8..e8bd719 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java @@ -203,4 +203,44 @@ public ASTNode parse(String command, Context ctx) throws ParseException { return (ASTNode) r.getTree(); } + + + /* + * parse a String as a Select List. This allows table functions to be passed expression Strings + * that are translated in + * the context they define at invocation time. Currently used by NPath to allow users to specify + * what output they want. + * NPath allows expressions n 'tpath' a column that represents the matched set of rows. This + * column doesn't exist in + * the input schema and hence the Result Expression cannot be analyzed by the regular Hive + * translation process. + */ + public ASTNode parseSelect(String command, Context ctx) throws ParseException { + LOG.info("Parsing command: " + command); + + HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command)); + TokenRewriteStream tokens = new TokenRewriteStream(lexer); + if (ctx != null) { + ctx.setTokenRewriteStream(tokens); + } + HiveParser parser = new HiveParser(tokens); + parser.setTreeAdaptor(adaptor); + HiveParser_SelectClauseParser.selectClause_return r = null; + try { + r = parser.selectClause(); + } catch (RecognitionException e) { + e.printStackTrace(); + throw new ParseException(parser.errors); + } + + if (lexer.getErrors().size() == 0 && parser.errors.size() == 0) { + LOG.info("Parse Completed"); + } else if (lexer.getErrors().size() != 0) { + throw new ParseException(lexer.getErrors()); + } else { + throw new ParseException(parser.errors); + } + + return (ASTNode) r.getTree(); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java index 7dd0426..26494dc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java @@ -52,6 +52,16 @@ private boolean isAnalyzeRewrite; private CreateTableDesc tblDesc = null; // table descriptor of the final + // used by PTFs + /* + * This map maintains the PTFInvocationSpec for each PTF chain invocation in this QB. + */ + private HashMap ptfNodeToSpec; + /* + * the WindowingSpec used for windowing clauses in this QB. + */ + private HashMap destToWindowingSpec; + // results public void print(String msg) { @@ -76,6 +86,8 @@ public QB(String outer_id, String alias, boolean isSubQ) { } qbp = new QBParseInfo(alias, isSubQ); qbm = new QBMetaData(); + ptfNodeToSpec = new HashMap(); + destToWindowingSpec = new HashMap(); id = getAppendedAliasFromId(outer_id, alias); } @@ -246,4 +258,35 @@ public boolean isAnalyzeRewrite() { public void setAnalyzeRewrite(boolean isAnalyzeRewrite) { this.isAnalyzeRewrite = isAnalyzeRewrite; } -} + + public PTFInvocationSpec getPTFInvocationSpec(ASTNode node) { + return ptfNodeToSpec == null ? null : ptfNodeToSpec.get(node); + } + + public void addPTFNodeToSpec(ASTNode node, PTFInvocationSpec spec) { + ptfNodeToSpec = ptfNodeToSpec == null ? new HashMap() : ptfNodeToSpec; + ptfNodeToSpec.put(node, spec); + } + + public HashMap getPTFNodeToSpec() { + return ptfNodeToSpec; + } + + public WindowingSpec getWindowingSpec(String dest) { + return destToWindowingSpec.get(dest); + } + + public void addDestToWindowingSpec(String dest, WindowingSpec windowingSpec) { + destToWindowingSpec.put(dest, windowingSpec); + } + + public boolean hasWindowingSpec(String dest) { + return destToWindowingSpec.get(dest) != null; + } + + public HashMap getAllWindowingSpecs() { + return destToWindowingSpec; + } + + +} \ No newline at end of file diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java index 691d315..185cc82 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java @@ -66,6 +66,7 @@ private String partName; // used for column statistics private boolean isTblLvl; // used for column statistics + /** * ClusterBy is a short name for both DistributeBy and SortBy. */ @@ -96,6 +97,10 @@ private final LinkedHashMap> destToAggregationExprs; private final HashMap> destToDistinctFuncExprs; + // used by Windowing + private final LinkedHashMap> destToWindowingExprs; + + @SuppressWarnings("unused") private static final Log LOG = LogFactory.getLog(QBParseInfo.class.getName()); @@ -119,6 +124,7 @@ public QBParseInfo(String alias, boolean isSubQ) { destGroupingSets = new HashSet(); destToAggregationExprs = new LinkedHashMap>(); + destToWindowingExprs = new LinkedHashMap>(); destToDistinctFuncExprs = new HashMap>(); this.alias = alias; @@ -131,6 +137,14 @@ public QBParseInfo(String alias, boolean isSubQ) { } + /* + * If a QB is such that the aggregation expressions need to be handled by + * the Windowing PTF; we invoke this function to clear the AggExprs on the dest. + */ + public void clearAggregationExprsForClause(String clause) { + destToAggregationExprs.get(clause).clear(); + } + public void setAggregationExprsForClause(String clause, LinkedHashMap aggregationTrees) { destToAggregationExprs.put(clause, aggregationTrees); @@ -158,6 +172,26 @@ public boolean isInsertIntoTable(String dbName, String table) { return destToAggregationExprs.get(clause); } + public void addWindowingExprToClause(String clause, ASTNode windowingExprNode) { + LinkedHashMap windowingExprs = destToWindowingExprs.get(clause); + if ( windowingExprs == null ) { + windowingExprs = new LinkedHashMap(); + destToWindowingExprs.put(clause, windowingExprs); + } + windowingExprs.put(windowingExprNode.toStringTree(), windowingExprNode); + } + + public HashMap getWindowingExprsForClause(String clause) { + return destToWindowingExprs.get(clause); + } + + public void clearDistinctFuncExprsForClause(String clause) { + List l = destToDistinctFuncExprs.get(clause); + if ( l != null ) { + l.clear(); + } + } + public void setDistinctFuncExprsForClause(String clause, List ast) { destToDistinctFuncExprs.put(clause, ast); } @@ -578,3 +612,5 @@ public void setNoScanAnalyzeCommand(boolean isNoScanAnalyzeCommand) { this.isNoScanAnalyzeCommand = isNoScanAnalyzeCommand; } } + + diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SelectClauseParser.g ql/src/java/org/apache/hadoop/hive/ql/parse/SelectClauseParser.g index 79494da..133e90a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SelectClauseParser.g +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SelectClauseParser.g @@ -125,7 +125,10 @@ selectItem @init { gParent.msgs.push("selection target"); } @after { gParent.msgs.pop(); } : - ( selectExpression ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))?) -> ^(TOK_SELEXPR selectExpression identifier*) + ( selectExpression + ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))? + (KW_OVER ws=window_specification )? + ) -> ^(TOK_SELEXPR selectExpression identifier* $ws?) ; trfmClause diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 26b6adf..c4e0bbe 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -29,12 +30,16 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.Stack; import java.util.TreeSet; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; +import org.antlr.runtime.CommonToken; import org.antlr.runtime.tree.BaseTree; import org.antlr.runtime.tree.Tree; +import org.antlr.runtime.tree.TreeWizard; +import org.antlr.runtime.tree.TreeWizard.ContextVisitor; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.Path; @@ -68,6 +73,7 @@ import org.apache.hadoop.hive.ql.exec.MapRedTask; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.PTFOperator; import org.apache.hadoop.hive.ql.exec.RecordReader; import org.apache.hadoop.hive.ql.exec.RecordWriter; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; @@ -116,6 +122,24 @@ import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalOptimizer; import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext; import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.tableSpec.SpecType; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderExpression; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFInputSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputType; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionExpression; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionedTableFunctionSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitioningSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.CurrentRowSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.Direction; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.RangeBoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.ValueBoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowExpressionSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFrameSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFunctionSpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowSpec; import org.apache.hadoop.hive.ql.plan.AggregationDesc; import org.apache.hadoop.hive.ql.plan.ColumnStatsDesc; import org.apache.hadoop.hive.ql.plan.ColumnStatsWork; @@ -151,6 +175,12 @@ import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.OrderExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFQueryInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.ScriptDesc; @@ -166,6 +196,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFHash; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr; import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionEvaluator; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe; @@ -328,7 +359,11 @@ public void doPhase1QBExpr(ASTNode ast, QBExpr qbexpr, String id, String alias) switch (ast.getToken().getType()) { case HiveParser.TOK_QUERY: { QB qb = new QB(id, alias, true); - doPhase1(ast, qb, initPhase1Ctx()); + Phase1Ctx ctx_1 = initPhase1Ctx(); + doPhase1(ast, qb, ctx_1); + ensureWindowingSpecOnQB(qb, ctx_1.dest); + ensureWindowingSourceHasPartitioning(qb.getWindowingSpec(ctx_1.dest), ast); + qbexpr.setOpcode(QBExpr.Opcode.NULLOP); qbexpr.setQB(qb); } @@ -354,13 +389,29 @@ public void doPhase1QBExpr(ASTNode ast, QBExpr qbexpr, String id, String alias) } private LinkedHashMap doPhase1GetAggregationsFromSelect( - ASTNode selExpr) { + ASTNode selExpr, QB qb, String dest) { + + ArrayList wdwingexprNodes = null; + if ( queryProperties.hasWindowing() && qb.getWindowingSpec(dest) != null ) { + WindowingSpec wdwSpec = qb.getWindowingSpec(dest); + HashMap aliasToWdwExprs = wdwSpec.getAliasToWdwExpr(); + wdwingexprNodes = new ArrayList(); + for (WindowExpressionSpec exprSpec : aliasToWdwExprs.values()) { + wdwingexprNodes.add(exprSpec.getExpression()); + } + } + // Iterate over the selects search for aggregation Trees. // Use String as keys to eliminate duplicate trees. LinkedHashMap aggregationTrees = new LinkedHashMap(); for (int i = 0; i < selExpr.getChildCount(); ++i) { - ASTNode sel = (ASTNode) selExpr.getChild(i).getChild(0); - doPhase1GetAllAggregations(sel, aggregationTrees); + ASTNode sel = (ASTNode) selExpr.getChild(i); + if(queryProperties.hasWindowing() && qb.getWindowingSpec(dest) != null ){ + if(wdwingexprNodes.contains(sel.getChild(0))){ + continue; + } + } + doPhase1GetAllAggregations((ASTNode) sel.getChild(0), aggregationTrees); } return aggregationTrees; } @@ -646,6 +697,17 @@ private void processJoin(QB qb, ASTNode join) throws SemanticException { processTable(qb, child); } else if (child.getToken().getType() == HiveParser.TOK_SUBQUERY) { processSubQuery(qb, child); + } else if (child.getToken().getType() == HiveParser.TOK_PTBLFUNCTION) { + queryProperties.setHasPTF(true); + processPTF(qb, child); + PTFInvocationSpec ptfInvocationSpec = qb.getPTFInvocationSpec(child); + String inputAlias = ptfInvocationSpec == null ? null : + ((PartitionedTableFunctionSpec)ptfInvocationSpec.getFunction()).getAlias();; + if ( inputAlias == null ) { + throw new SemanticException(generateErrorMessage(child, + "PTF invocation in a Join must have an alias")); + } + } else if (child.getToken().getType() == HiveParser.TOK_LATERAL_VIEW) { // SELECT * FROM src1 LATERAL VIEW udtf() AS myTable JOIN src2 ... // is not supported. Instead, the lateral view must be in a subquery @@ -737,11 +799,14 @@ public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1) qbp.setHints((ASTNode) ast.getChild(0)); } - LinkedHashMap aggregations = doPhase1GetAggregationsFromSelect(ast); + handleWindowingExprsInSelectList(qb, ctx_1.dest, ast); + + LinkedHashMap aggregations = doPhase1GetAggregationsFromSelect(ast, + qb, ctx_1.dest); doPhase1GetColumnAliasesFromSelect(ast, qbp); qbp.setAggregationExprsForClause(ctx_1.dest, aggregations); qbp.setDistinctFuncExprsForClause(ctx_1.dest, - doPhase1GetDistinctFuncExprs(aggregations)); + doPhase1GetDistinctFuncExprs(aggregations)); break; case HiveParser.TOK_WHERE: @@ -766,7 +831,6 @@ public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1) .getMsg(ast)); } } - qbp.setDestForClause(ctx_1.dest, (ASTNode) ast.getChild(0)); break; @@ -789,22 +853,37 @@ public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1) queryProperties.setHasJoin(true); processJoin(qb, frm); qbp.setJoinExpr(frm); + }else if(frm.getToken().getType() == HiveParser.TOK_PTBLFUNCTION){ + queryProperties.setHasPTF(true); + processPTF(qb, frm); } break; case HiveParser.TOK_CLUSTERBY: - // Get the clusterby aliases - these are aliased to the entries in the - // select list - queryProperties.setHasClusterBy(true); - qbp.setClusterByExprForClause(ctx_1.dest, ast); + + // if there is no GroupBy Clause, but there are still Agg Exprs remaining in the select list + // - move them to qb.windowingQSpec. + if ( qbp.getAggregationExprsForClause(ctx_1.dest) != null && + !queryProperties.hasGroupBy()) { + moveaggregationExprsToWindowingSpec(qb, ctx_1.dest); + } + + // if QB has WindowingClauses + // - associate DistributeBy with qb.windowingQSpec + if (qb.hasWindowingSpec(ctx_1.dest) ) + { + handleClusterOrDistributeByForWindowing(qb, ctx_1, ast); + } + else { + // Get the clusterby aliases - these are aliased to the entries in the + // select list + queryProperties.setHasClusterBy(true); + qbp.setClusterByExprForClause(ctx_1.dest, ast); + } break; case HiveParser.TOK_DISTRIBUTEBY: - // Get the distribute by aliases - these are aliased to the entries in - // the - // select list - queryProperties.setHasDistributeBy(true); - qbp.setDistributeByExprForClause(ctx_1.dest, ast); + if (qbp.getClusterByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, ErrorMsg.CLUSTERBY_DISTRIBUTEBY_CONFLICT.getMsg())); @@ -812,13 +891,33 @@ public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1) throw new SemanticException(generateErrorMessage(ast, ErrorMsg.ORDERBY_DISTRIBUTEBY_CONFLICT.getMsg())); } + + // if there is no GroupBy Clause, but there are still Agg Exprs remaining + // in the select list + // - move them to qb.windowingQSpec. + if ( qbp.getAggregationExprsForClause(ctx_1.dest) != null && + !queryProperties.hasGroupBy()) { + moveaggregationExprsToWindowingSpec(qb, ctx_1.dest); + } + + // if QB has WindowingClauses + // - associate DistributeBy with qb.windowingQSpec + if (qb.hasWindowingSpec(ctx_1.dest) ) + { + handleClusterOrDistributeByForWindowing(qb, ctx_1, ast); + } + else { + + // Get the distribute by aliases - these are aliased to the entries in + // the + // select list + queryProperties.setHasDistributeBy(true); + qbp.setDistributeByExprForClause(ctx_1.dest, ast); + } break; case HiveParser.TOK_SORTBY: - // Get the sort by aliases - these are aliased to the entries in the - // select list - queryProperties.setHasSortBy(true); - qbp.setSortByExprForClause(ctx_1.dest, ast); + if (qbp.getClusterByForClause(ctx_1.dest) != null) { throw new SemanticException(generateErrorMessage(ast, ErrorMsg.CLUSTERBY_SORTBY_CONFLICT.getMsg())); @@ -827,6 +926,19 @@ public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1) ErrorMsg.ORDERBY_SORTBY_CONFLICT.getMsg())); } + // if QB has WindowingClauses + // - associate SortBy with qb.windowingSpec. + if (qb.hasWindowingSpec(ctx_1.dest) ) + { + handleSortByForWindowing(qb, ctx_1, ast); + } + else { + // Get the sort by aliases - these are aliased to the entries in the + // select list + queryProperties.setHasSortBy(true); + qbp.setSortByExprForClause(ctx_1.dest, ast); + } + break; case HiveParser.TOK_ORDERBY: @@ -868,8 +980,39 @@ public boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1) break; case HiveParser.TOK_HAVING: - qbp.setHavingExprForClause(ctx_1.dest, ast); - qbp.addAggregationExprsForClause(ctx_1.dest, doPhase1GetAggregationsFromSelect(ast)); + boolean hasLLArgs = false; + TreeWizard tw = new TreeWizard(ParseDriver.adaptor, HiveParser.tokenNames); + CheckLeadLagInSelectExprs checkLLFunctions = + new CheckLeadLagInSelectExprs(qb, ctx_1.dest); + tw.visit(ast, HiveParser.TOK_FUNCTION, checkLLFunctions); + if ( checkLLFunctions.isError() ) { + throw new SemanticException(generateErrorMessage(ast, + checkLLFunctions.getErrString())); + } + hasLLArgs = checkLLFunctions.hasLeadLagExprs(); + + // if QB has WindowingClauses and no GroupBy clause + // - associate Having with qb.windowingSpec. + if (qb.hasWindowingSpec(ctx_1.dest) && qbp.getGroupByForClause(ctx_1.dest) == null) + { + handleWindowingHavingClause(qb, ctx_1, ast); + } else if(hasLLArgs){ + throw new SemanticException(generateErrorMessage(ast, + "Query has no windowing or group by clause: " + + "Unsupported place for having")); + } else{ + qbp.setHavingExprForClause(ctx_1.dest, ast); + qbp.addAggregationExprsForClause(ctx_1.dest, + doPhase1GetAggregationsFromSelect(ast, qb, ctx_1.dest)); + } + break; + + case HiveParser.KW_WINDOW: + if (!qb.hasWindowingSpec(ctx_1.dest) ) { + throw new SemanticException(generateErrorMessage(ast, + "Query has no Cluster/Distribute By; but has a Window definition")); + } + handleQueryWindowClauses(qb, ctx_1, ast); break; case HiveParser.TOK_LIMIT: @@ -2168,12 +2311,20 @@ private int setBit(int bitmap, int bitIdx) { List result = new ArrayList(selectExprs == null ? 0 : selectExprs.getChildCount()); if (selectExprs != null) { + HashMap windowingExprs = parseInfo.getWindowingExprsForClause(dest); + for (int i = 0; i < selectExprs.getChildCount(); ++i) { if (((ASTNode) selectExprs.getChild(i)).getToken().getType() == HiveParser.TOK_HINTLIST) { continue; } // table.column AS alias ASTNode grpbyExpr = (ASTNode) selectExprs.getChild(i).getChild(0); + /* + * If this is handled by Windowing then ignore it. + */ + if (windowingExprs != null && windowingExprs.containsKey(grpbyExpr.toStringTree())) { + continue; + } result.add(grpbyExpr); } } @@ -2200,7 +2351,10 @@ private int setBit(int bitmap, int bitIdx) { String tabAlias = null; String[] colRef = new String[2]; - if (selExpr.getChildCount() == 2) { + //for queries with a windowing expressions, the selexpr may have a third child + if (selExpr.getChildCount() == 2 || + (selExpr.getChildCount() == 3 && + selExpr.getChild(2).getType() == HiveParser.TOK_WINDOWSPEC)) { // return zz for "xx + yy AS zz" colAlias = unescapeIdentifier(selExpr.getChild(1).getText()); colRef[0] = tabAlias; @@ -2277,6 +2431,7 @@ private static boolean isRegex(String pattern) { return false; } + private Operator genSelectPlan(String dest, QB qb, Operator input) throws SemanticException { ASTNode selExprList = qb.getParseInfo().getSelForClause(dest); @@ -2414,11 +2569,14 @@ private static boolean isRegex(String pattern) { // child can be EXPR AS ALIAS, or EXPR. ASTNode child = (ASTNode) exprList.getChild(i); boolean hasAsClause = (!isInTransform) && (child.getChildCount() == 2); + boolean isWindowSpec = child.getChildCount() == 3 ? + (child.getChild(2).getType() == HiveParser.TOK_WINDOWSPEC) : + false; // EXPR AS (ALIAS,...) parses, but is only allowed for UDTF's // This check is not needed and invalid when there is a transform b/c the // AST's are slightly different. - if (!isInTransform && !isUDTF && child.getChildCount() > 2) { + if (!isWindowSpec && !isInTransform && !isUDTF && child.getChildCount() > 2) { throw new SemanticException(generateErrorMessage( (ASTNode) child.getChild(2), ErrorMsg.INVALID_AS.getMsg())); @@ -6251,12 +6409,18 @@ private QBJoinTree genJoinTree(QB qb, ASTNode joinParseTree, ASTNode right = (ASTNode) joinParseTree.getChild(1); if ((left.getToken().getType() == HiveParser.TOK_TABREF) - || (left.getToken().getType() == HiveParser.TOK_SUBQUERY)) { + || (left.getToken().getType() == HiveParser.TOK_SUBQUERY) + || (left.getToken().getType() == HiveParser.TOK_PTBLFUNCTION)) { String tableName = getUnescapedUnqualifiedTableName((ASTNode) left.getChild(0)) .toLowerCase(); String alias = left.getChildCount() == 1 ? tableName : unescapeIdentifier(left.getChild(left.getChildCount() - 1) - .getText().toLowerCase()); + .getText().toLowerCase()); + // ptf node form is: ^(TOK_PTBLFUNCTION $name $alias? partitionTableFunctionSource partitioningSpec? expression*) + // guranteed to have an lias here: check done in processJoin + alias = (left.getToken().getType() == HiveParser.TOK_PTBLFUNCTION) ? + unescapeIdentifier(left.getChild(1).getText().toLowerCase()) : + alias; joinTree.setLeftAlias(alias); String[] leftAliases = new String[1]; leftAliases[0] = alias; @@ -6282,12 +6446,18 @@ private QBJoinTree genJoinTree(QB qb, ASTNode joinParseTree, } if ((right.getToken().getType() == HiveParser.TOK_TABREF) - || (right.getToken().getType() == HiveParser.TOK_SUBQUERY)) { + || (right.getToken().getType() == HiveParser.TOK_SUBQUERY) + || (right.getToken().getType() == HiveParser.TOK_PTBLFUNCTION)) { String tableName = getUnescapedUnqualifiedTableName((ASTNode) right.getChild(0)) .toLowerCase(); String alias = right.getChildCount() == 1 ? tableName : unescapeIdentifier(right.getChild(right.getChildCount() - 1) - .getText().toLowerCase()); + .getText().toLowerCase()); + // ptf node form is: ^(TOK_PTBLFUNCTION $name $alias? partitionTableFunctionSource partitioningSpec? expression*) + // guranteed to have an lias here: check done in processJoin + alias = (right.getToken().getType() == HiveParser.TOK_PTBLFUNCTION) ? + unescapeIdentifier(right.getChild(1).getText().toLowerCase()) : + alias; String[] rightAliases = new String[1]; rightAliases[0] = alias; joinTree.setRightAliases(rightAliases); @@ -7070,6 +7240,11 @@ private Operator genPostGroupByBodyPlan(Operator curr, String dest, QB qb) curr = genHavingPlan(dest, qb, curr); } + + if(queryProperties.hasWindowing() && qb.getWindowingSpec(dest) != null) { + curr = genWindowingPlan(qb.getWindowingSpec(dest), curr); + } + curr = genSelectPlan(dest, qb, curr); Integer limit = qbp.getDestLimit(dest); @@ -7109,6 +7284,7 @@ private Operator genPostGroupByBodyPlan(Operator curr, String dest, QB qb) curr = genReduceSinkPlan(dest, qb, curr, numReducers); } + if (qbp.getIsSubQ()) { if (limit != null) { // In case of order by, only 1 reducer is used, so no need of @@ -7754,11 +7930,38 @@ public Operator genPlan(QB qb) throws SemanticException { aliasToOpInfo.put(alias, op); } + Operator srcOpInfo = null; + Operator lastPTFOp = null; + + if(queryProperties.hasPTF()){ + //After processing subqueries and source tables, process + // partitioned table functions + + HashMap ptfNodeToSpec = qb.getPTFNodeToSpec(); + if ( ptfNodeToSpec != null ) { + for(Entry entry : ptfNodeToSpec.entrySet()) { + ASTNode ast = entry.getKey(); + PTFInvocationSpec spec = entry.getValue(); + String inputAlias = spec.getQueryInputName(); + Operator inOp = aliasToOpInfo.get(inputAlias); + if ( inOp == null ) { + throw new SemanticException(generateErrorMessage(ast, + "Cannot resolve input Operator for PTF invocation")); + } + lastPTFOp = genPTFPlan(spec, inOp); + String ptfAlias = ((PartitionedTableFunctionSpec)spec.getFunction()).getAlias(); + if ( ptfAlias != null ) { + aliasToOpInfo.put(ptfAlias, lastPTFOp); + } + } + } + + } + // For all the source tables that have a lateral view, attach the // appropriate operators to the TS genLateralViewPlans(aliasToOpInfo, qb); - Operator srcOpInfo = null; // process join if (qb.getParseInfo().getJoinExpr() != null) { @@ -7781,6 +7984,9 @@ public Operator genPlan(QB qb) throws SemanticException { // Now if there are more than 1 sources then we have a join case // later we can extend this to the union all case as well srcOpInfo = aliasToOpInfo.values().iterator().next(); + // with ptfs, there maybe more (note for PTFChains: + // 1 ptf invocation may entail multiple PTF operators) + srcOpInfo = lastPTFOp != null ? lastPTFOp : srcOpInfo; } Operator bodyOpInfo = genBodyPlan(qb, srcOpInfo); @@ -8439,10 +8645,13 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { } // continue analyzing from the child ASTNode. - if (!doPhase1(child, qb, initPhase1Ctx())) { + Phase1Ctx ctx_1 = initPhase1Ctx(); + if (!doPhase1(child, qb, ctx_1)) { // if phase1Result false return return; } + ensureWindowingSpecOnQB(qb, ctx_1.dest); + ensureWindowingSourceHasPartitioning(qb.getWindowingSpec(ctx_1.dest), ast); LOG.info("Completed phase 1 of Semantic Analysis"); @@ -8455,6 +8664,8 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { // up with later. Operator sinkOp = genPlan(qb); + dumpOperatorChain(sinkOp, null); + resultSchema = convertRowSchemaToViewSchema(opParseCtx.get(sinkOp).getRowResolver()); @@ -9548,4 +9759,1452 @@ public QB getQB() { public void setQB(QB qb) { this.qb = qb; } + +//--------------------------- PTF handling ----------------------------------- + + /* + * - a partitionTableFunctionSource can be a tableReference, a SubQuery or another + * PTF invocation. + * - For a TABLEREF: set the source to the alias returned by processTable + * - For a SubQuery: set the source to the alias returned by processSubQuery + * - For a PTF invocation: recursively call processPTFChain. + */ + private PTFInputSpec processPTFSource(QB qb, ASTNode inputNode) throws SemanticException{ + + PTFInputSpec qInSpec = null; + int type = inputNode.getType(); + String alias; + switch(type) + { + case HiveParser.TOK_TABREF: + alias = processTable(qb, inputNode); + qInSpec = new PTFQueryInputSpec(); + ((PTFQueryInputSpec)qInSpec).setType(PTFQueryInputType.TABLE); + ((PTFQueryInputSpec)qInSpec).setSource(alias); + break; + case HiveParser.TOK_SUBQUERY: + alias = processSubQuery(qb, inputNode); + qInSpec = new PTFQueryInputSpec(); + ((PTFQueryInputSpec)qInSpec).setType(PTFQueryInputType.SUBQUERY); + ((PTFQueryInputSpec)qInSpec).setSource(alias); + break; + case HiveParser.TOK_PTBLFUNCTION: + qInSpec = processPTFChain(qb, inputNode); + break; + default: + throw new SemanticException(generateErrorMessage(inputNode, + "Unknown input type to PTF")); + } + + qInSpec.setAstNode(inputNode); + return qInSpec; + + } + + /* + * - tree form is + * ^(TOK_PTBLFUNCTION name alias? partitionTableFunctionSource partitioningSpec? arguments*) + * - a partitionTableFunctionSource can be a tableReference, a SubQuery or another + * PTF invocation. + */ + private PartitionedTableFunctionSpec processPTFChain(QB qb, ASTNode ptf) + throws SemanticException{ + int child_count = ptf.getChildCount(); + if (child_count < 2) { + throw new SemanticException(generateErrorMessage(ptf, + "Not enough Children " + child_count)); + } + + PartitionedTableFunctionSpec ptfSpec = new PartitionedTableFunctionSpec(); + ptfSpec.setAstNode(ptf); + + /* + * name + */ + ASTNode nameNode = (ASTNode) ptf.getChild(0); + ptfSpec.setName(nameNode.getText()); + + int inputIdx = 1; + + /* + * alias + */ + ASTNode secondChild = (ASTNode) ptf.getChild(1); + if ( secondChild.getType() == HiveParser.Identifier ) { + ptfSpec.setAlias(secondChild.getText()); + inputIdx++; + } + + /* + * input + */ + ASTNode inputNode = (ASTNode) ptf.getChild(inputIdx); + ptfSpec.setInput(processPTFSource(qb, inputNode)); + + int argStartIdx = inputIdx + 1; + + /* + * partitioning Spec + */ + int pSpecIdx = inputIdx + 1; + ASTNode pSpecNode = ptf.getChildCount() > inputIdx ? + (ASTNode) ptf.getChild(pSpecIdx) : null; + if (pSpecNode != null && pSpecNode.getType() == HiveParser.TOK_PARTITIONINGSPEC) + { + PartitioningSpec partitioning = processPTFPartitionSpec(pSpecNode); + ptfSpec.setPartitioning(partitioning); + argStartIdx++; + } + + /* + * arguments + */ + for(int i=argStartIdx; i < ptf.getChildCount(); i++) + { + ptfSpec.addArg((ASTNode) ptf.getChild(i)); + } + return ptfSpec; + } + + /* + * - invoked during FROM AST tree processing, on encountering a PTF invocation. + * - tree form is + * ^(TOK_PTBLFUNCTION name partitionTableFunctionSource partitioningSpec? arguments*) + * - setup a PTFInvocationSpec for this top level PTF invocation. + */ + private void processPTF(QB qb, ASTNode ptf) throws SemanticException{ + + PartitionedTableFunctionSpec ptfSpec = processPTFChain(qb, ptf); + + if ( ptfSpec.getAlias() != null ) { + qb.addAlias(ptfSpec.getAlias()); + } + + PTFInvocationSpec spec = new PTFInvocationSpec(); + spec.setFunction(ptfSpec); + ensurePTFChainHasPartitioning(spec, ptf); + qb.addPTFNodeToSpec(ptf, spec); + } + + /* + * ensure that the PTF chain has a partitioning specification associated. This method + * should be called when a PTF chain is encountered as a fromSource. + * (from the processPTF method) + */ + private void ensurePTFChainHasPartitioning(PTFInvocationSpec qSpec, ASTNode node) + throws SemanticException { + if(qSpec == null){ + return; + } + PartitionedTableFunctionSpec ptfSpec = qSpec.getStartOfChain(); + PartitionSpec pSpec = ptfSpec.getPartition(); + if ( pSpec == null ) { + throw new SemanticException(generateErrorMessage(node, + "No partition specification associated with start of PTF chain ")); + } + } + +//--------------------------- Windowing handling ----------------------------------- + + + /* + * ensure that the PTF chain has a partitioning specification associated. + * This method should be called at the end of Phase1 + * to check that the WindowingTableFunction + * is driven from a partitioning specification + * (specified in the distribute by or cluster by clauses). + */ + private void ensureWindowingSourceHasPartitioning(WindowingSpec qSpec, ASTNode node) + throws SemanticException { + if(qSpec == null){ + return; + } + PartitionSpec pSpec = qSpec.getQueryPartitionSpec(); + if ( pSpec == null ) { + throw new SemanticException(generateErrorMessage(node, + "No partition specification associated with Windowing")); + } + } + + /* + * - A Select Item form is: ^(TOK_SELEXPR selectExpression Identifier* window_specification?) + * What constitutes a Windowing Select Expression: + * 1. + * - It must be a UDAF function invocation + * - It must have a Windowing Spec + * 2. + * - It must be a Ranking or Navigation Function invocation which is also a UDAF function + * (so no lead/lag) + * 3. + * - It must be a UDAF function invocation + * - must have an argument which invokes Lead/Lag + * Error Expressions are: + * - a Window Spec without a UDAF invocation: we don't support expressions i Windowing. + */ + private boolean checkAndExtractWindowFunctionsInSelect(QB qb, ASTNode selectExpr, String dest) + throws SemanticException { + + int childCount = selectExpr.getChildCount(); + ASTNode windowSpec = (ASTNode) selectExpr.getChild(childCount - 1); + + boolean hasWindowSpec = windowSpec.getType() == HiveParser.TOK_WINDOWSPEC; + ASTNode function = (ASTNode) selectExpr.getChild(0); + boolean isFunction = ( function.getType() == HiveParser.TOK_FUNCTION || + function.getType() == HiveParser.TOK_FUNCTIONDI || + function.getType() == HiveParser.TOK_FUNCTIONSTAR ); + + if ( !isFunction ) { + return false; + } + + String fnName = function.getChild(0).getText().toLowerCase(); + if(!FunctionRegistry.isWindowFunction(fnName)) { + // Its either UDF, UDTF or lead or lag function. + if (hasWindowSpec){ + // We currently supports windowing only with UDAFs, so if there is a windowing spec + // thats the error condition. + throw new SemanticException(generateErrorMessage(selectExpr, + "Currently windowing Specification can only be associated with a UDAF " + + "invocation or navigation functions")); + } else { + return false; + } + } + + boolean hasLLArgs = false; + boolean isRankingOrNavFunction = FunctionRegistry.RANKING_FUNCTIONS.contains(fnName) || + FunctionRegistry.NAVIGATION_FUNCTIONS.contains(fnName); + + /* + * treat Lead & Lag as a UDF if there is no WindowSpec. + */ + if (!hasWindowSpec + && + (fnName.equals(FunctionRegistry.LAG_FUNC_NAME) || fnName + .equals(FunctionRegistry.LEAD_FUNC_NAME))) { + return false; + } + + /* + * If Windowing Function has LeadLag expression in its args, + * then it will be handled by WindowingTabFunc. + */ + if (!isRankingOrNavFunction ) { + // but lead/lag are not supported as an argument of ranking or navigation function. + TreeWizard tw = new TreeWizard(ParseDriver.adaptor, HiveParser.tokenNames); + CheckLeadLagInSelectExprs checkLLFunctions = new CheckLeadLagInSelectExprs(qb, dest); + for(int i=1; !hasLLArgs && i < function.getChildCount(); i++) { + ASTNode child = (ASTNode) function.getChild(i); + tw.visit(child, HiveParser.TOK_FUNCTION, checkLLFunctions); + if ( checkLLFunctions.isError() ) { + throw new SemanticException(generateErrorMessage(selectExpr, + checkLLFunctions.getErrString())); + } + hasLLArgs = checkLLFunctions.hasLeadLagExprs(); + } + } + + if (hasWindowSpec || isRankingOrNavFunction || hasLLArgs) { + /* + * @revisit: what should I do if there are more than 3 children; + * i.e. more than one Identifier. + */ + + String alias = null; + ASTNode secondChildNode = (ASTNode) selectExpr.getChild(1); + if ( childCount > 1 && secondChildNode.getType() == HiveParser.Identifier) { + alias = secondChildNode.getText(); + } + else { + /* + * generate an alias using the function AST tree. + * - calling getColAlias with a null InputResolver and null defaultName. + * - so includeFunctionName should be true. + */ + String[] colAlias = getColAlias(function, null, null, true, -1); + alias = colAlias[1]; + } + + WindowingSpec spec = qb.getWindowingSpec(dest); + if(spec == null) { + queryProperties.setHasWindowing(true); + spec = new WindowingSpec(); + qb.addDestToWindowingSpec(dest, spec); + } + WindowFunctionSpec wFnSpec = processWindowFunction(function, + hasWindowSpec ? windowSpec : null); + wFnSpec.setAlias(alias); + spec.addWindowFunction(wFnSpec); + qb.getParseInfo().addWindowingExprToClause(dest, wFnSpec.getExpression()); + return true; + } + return false; + } + + + private static class CheckLeadLagInSelectExprs implements ContextVisitor + { + QB qb; + String dest; + boolean hasLeadLagExprs = false; + boolean error = false; + String errString; + + private CheckLeadLagInSelectExprs(QB qb, String dest) { + this.qb = qb; + this.dest = dest; + } + + @Override + public void visit(Object t, Object parent, int childIndex, Map labels) + { + error = false; errString = null; + ASTNode function = (ASTNode) t; + WindowingSpec wSpec = qb.getWindowingSpec(dest); + + HashMap windowingExprs = + qb.getParseInfo().getWindowingExprsForClause(dest); + + /* + * reference to a Windowing Fns is allowed in Windowing Exprs. + */ + if ( windowingExprs != null && windowingExprs.containsKey(function.toStringTree())) { + return; + } + + String fnName = function.getChild(0).getText().toLowerCase(); + if (fnName.equals(FunctionRegistry.LEAD_FUNC_NAME) + || fnName.equals(FunctionRegistry.LAG_FUNC_NAME)) + { + hasLeadLagExprs = true; + } + else if ( FunctionRegistry.NAVIGATION_FUNCTIONS.contains(fnName)) { + error = true; + errString = "Currently you cannot use a Navigation Functions: " + + "first_value, last_value in expressions"; + } + else if ( FunctionRegistry.RANKING_FUNCTIONS.contains(fnName)) { + error = true; + errString = "Currently you cannot use a Ranking Functions: in expressions"; + } + } + + public boolean hasLeadLagExprs() { + return hasLeadLagExprs; + } + + protected boolean isError() { + return error; + } + + protected String getErrString() { + return errString; + } + } + + + /* + * - Invoked during Phase1 when a TOK_SELECT is encountered. + * - Select tree form is: ^(TOK_SELECT ^(TOK_SELECTEXPR...) ^(TOK_SELECTEXPR...) ...) + * - A Select Item form is: ^(TOK_SELEXPR selectExpression Identifier* window_specification?) + * + * We need to extract the SelectList any SelectItems that must be handled during + * Windowing processing. These are: + * - SelectItems that have a window_specification + * - SelectItems that invoke row navigation functions: Lead/Lag. + * + * Do we need to change the SelectList in any way? + * - initially we thought of replacing the selectExpressions handled by Windowing + * with a ASTNode that is of type Identfier and + * references the alias to the orginal expression. Why? + * - the output of processing the PTF Operator that handles windowing will + * contain the values of the Windowing expressions. + * - the final Select Op that is a child of the above PTF Op can get these values + * from its input by referring to the computed + * windowing expression via its alias. + * - but this is not needed. Why? + * - When transforming a AST tree to an ExprNodeDesc the TypeCheckFactory checks + * if there is a mapping from an AST tree to an output + * column in the InputResolver; if there is it uses the alias for the Output column + * - This is how values get handed from a GBy Op to the next Select Op; + * - we need the same thing to happen. + */ + private void handleWindowingExprsInSelectList(QB qb, String dest, ASTNode selectNode) + throws SemanticException { + TreeWizard tw = new TreeWizard(ParseDriver.adaptor, HiveParser.tokenNames); + CheckLeadLagInSelectExprs checkLLFunctions = new CheckLeadLagInSelectExprs(qb, dest); + + for(int i=0; i < selectNode.getChildCount(); i++) + { + ASTNode selectExpr = (ASTNode) selectNode.getChild(i); + if ( selectExpr.getType() != HiveParser.TOK_SELEXPR ) + { + continue; + } + boolean hasWindowingExprs = checkAndExtractWindowFunctionsInSelect(qb, selectExpr, dest); + + if ( !hasWindowingExprs ) { + tw.visit(selectExpr, HiveParser.TOK_FUNCTION, checkLLFunctions); + + if ( checkLLFunctions.isError() ) { + throw new SemanticException(generateErrorMessage(selectExpr, + checkLLFunctions.getErrString())); + } + + hasWindowingExprs = checkLLFunctions.hasLeadLagExprs(); + + if ( hasWindowingExprs ) + { + ASTNode expr = (ASTNode)selectExpr.getChild(0); + WindowingSpec spec = qb.getWindowingSpec(dest); + if(spec == null) { + queryProperties.setHasWindowing(true); + spec = new WindowingSpec(); + qb.addDestToWindowingSpec(dest, spec); + } + + String alias; + int childCount = selectExpr.getChildCount(); + /* + * @revisit what if there are multiple Identifiers(lateral view) + */ + if ( childCount < 2 ) { + /* + * generate an alias using the function AST tree. + * - calling getColAlias with a null InputResolver and null defaultName. + * - so includeFunctionName should be true. + */ + alias = getColAlias(expr, null, null, true, -1)[1]; + } + else { + alias = selectExpr.getChild(1).getText(); + } + spec.addExpression(expr, alias); + qb.getParseInfo().addWindowingExprToClause(dest, expr); + } + } + } + } + + /* + * this method is called in the case when the Query has Windowing Clauses and a + * Distribute/Cluster By. + * - the Distribute/Cluster By is associated with the WindowingSpec + * - If the Query has no Group by, but there are qb.destToAggExprs then + * move these to the Windowing Spec. + */ + private void handleClusterOrDistributeByForWindowing(QB qb, Phase1Ctx ctx_1, ASTNode ast) + throws SemanticException { + QBParseInfo qbp = qb.getParseInfo(); + PartitionSpec pSpec = processPartitionSpec(ast); + WindowingSpec spec = qb.getWindowingSpec(ctx_1.dest); + spec.setQueryPartitonSpec(pSpec); + } + + private void handleSortByForWindowing(QB qb, Phase1Ctx ctx_1, ASTNode ast) + throws SemanticException { + OrderSpec oSpec = processOrderSpec(ast); + WindowingSpec spec = qb.getWindowingSpec(ctx_1.dest); + spec.setQueryOrderSpec(oSpec); + } + + private void handleQueryWindowClauses(QB qb, Phase1Ctx ctx_1, ASTNode node) + throws SemanticException { + WindowingSpec spec = qb.getWindowingSpec(ctx_1.dest); + for(int i=0; i < node.getChildCount(); i++) { + processQueryWindowClause(spec, (ASTNode) node.getChild(i)); + } + } + + private void handleWindowingHavingClause(QB qb, Phase1Ctx ctx_1, ASTNode node) { + WindowingSpec spec = qb.getWindowingSpec(ctx_1.dest); + spec.setFilterExpr((ASTNode)node.getChild(0)); + } + + private PartitionSpec processPartitionSpec(ASTNode node) { + PartitionSpec pSpec = new PartitionSpec(); + int exprCnt = node.getChildCount(); + for(int i=0; i < exprCnt; i++) { + PartitionExpression exprSpec = new PartitionExpression(); + exprSpec.setExpression((ASTNode) node.getChild(i)); + pSpec.addExpression(exprSpec); + } + return pSpec; + } + + private OrderSpec processOrderSpec(ASTNode sortNode) { + OrderSpec oSpec = new OrderSpec(); + int exprCnt = sortNode.getChildCount(); + for(int i=0; i < exprCnt; i++) { + OrderExpression exprSpec = new OrderExpression(); + exprSpec.setExpression((ASTNode) sortNode.getChild(i).getChild(0)); + if ( sortNode.getChild(i).getType() == HiveParser.TOK_TABSORTCOLNAMEASC ) { + exprSpec.setOrder(org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order.ASC); + } + else { + exprSpec.setOrder(org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order.DESC); + } + oSpec.addExpression(exprSpec); + } + return oSpec; + } + + private PartitioningSpec processPTFPartitionSpec(ASTNode pSpecNode) + { + PartitioningSpec partitioning = new PartitioningSpec(); + ASTNode firstChild = (ASTNode) pSpecNode.getChild(0); + int type = firstChild.getType(); + int exprCnt; + + PartitionSpec pSpec = processPartitionSpec(firstChild); + partitioning.setPartSpec(pSpec); + + if ( type == HiveParser.TOK_DISTRIBUTEBY ) + { + ASTNode sortNode = pSpecNode.getChildCount() > 1 ? (ASTNode) pSpecNode.getChild(1) : null; + if ( sortNode != null ) + { + OrderSpec oSpec = processOrderSpec(sortNode); + partitioning.setOrderSpec(oSpec); + } + } + return partitioning; + } + + private WindowFunctionSpec processWindowFunction(ASTNode node, ASTNode wsNode) + throws SemanticException { + WindowFunctionSpec wfSpec = new WindowFunctionSpec(); + + switch(node.getType()) { + case HiveParser.TOK_FUNCTIONSTAR: + wfSpec.setStar(true); + break; + case HiveParser.TOK_FUNCTIONDI: + wfSpec.setDistinct(true); + break; + } + + if ( wfSpec.isDistinct() ) { + throw new SemanticException(generateErrorMessage(node, + "Count/Sum distinct not supported with Windowing")); + } + + wfSpec.setExpression(node); + + ASTNode nameNode = (ASTNode) node.getChild(0); + wfSpec.setName(nameNode.getText()); + + for(int i=1; i < node.getChildCount(); i++) { + ASTNode child = (ASTNode) node.getChild(i); + wfSpec.addArg(child); + } + + if ( wsNode != null ) { + WindowSpec ws = processWindowSpec(wsNode); + wfSpec.setWindowSpec(ws); + } + + /* + * In order to distinguish between different UDAF invocations on the same UDAF but different Windows + * add the WdwSpec node as a child of the Function Node. + * It is safe to do this after the function node has been converetd to a WdwFuncSpec. + */ + if ( wsNode != null ) { + node.addChild(wsNode); + } + + return wfSpec; + } + + private void processQueryWindowClause(WindowingSpec spec, ASTNode node) + throws SemanticException { + ASTNode nameNode = (ASTNode) node.getChild(0); + ASTNode wsNode = (ASTNode) node.getChild(1); + if(spec.getWindowSpecs() != null && spec.getWindowSpecs().containsKey(nameNode.getText())){ + throw new SemanticException(generateErrorMessage(nameNode, + "Duplicate definition of window " + nameNode.getText() + + " is not allowed")); + } + WindowSpec ws = processWindowSpec(wsNode); + spec.addWindowSpec(nameNode.getText(), ws); + } + + private WindowSpec processWindowSpec(ASTNode node) { + String sourceId = null; + PartitionSpec partition = null; + OrderSpec order = null; + WindowFrameSpec windowFrame = null; + + boolean hasSrcId = false, hasPartSpec = false, hasWF = false; + int srcIdIdx = -1, partIdx = -1, wfIdx = -1; + + for(int i=0; i < node.getChildCount(); i++) + { + int type = node.getChild(i).getType(); + switch(type) + { + case HiveParser.Identifier: + hasSrcId = true; srcIdIdx = i; + break; + case HiveParser.TOK_PARTITIONINGSPEC: + hasPartSpec = true; partIdx = i; + break; + case HiveParser.TOK_WINDOWRANGE: + case HiveParser.TOK_WINDOWVALUES: + hasWF = true; wfIdx = i; + break; + } + } + + WindowSpec ws = new WindowSpec(); + + if (hasSrcId) { + ASTNode nameNode = (ASTNode) node.getChild(srcIdIdx); + ws.setSourceId(nameNode.getText()); + } + + if (hasPartSpec) { + ASTNode partNode = (ASTNode) node.getChild(partIdx); + PartitioningSpec partitioning = processPTFPartitionSpec(partNode); + ws.setPartitioning(partitioning); + } else if(node.getChildCount() == 0){ + // no partition information was specified so partition by a constant + PartitioningSpec partitioningSpec = new PartitioningSpec(); + PartitionSpec partitionSpec = new PartitionSpec(); + PartitionExpression partExpr = new PartitionExpression(); + partExpr.setExpression(new ASTNode(new CommonToken(HiveParser.Number, "0"))); + partitionSpec.addExpression(partExpr); + partitioningSpec.setPartSpec(partitionSpec); + ws.setPartitioning(partitioningSpec); + } + + if ( hasWF) + { + ASTNode wfNode = (ASTNode) node.getChild(wfIdx); + WindowFrameSpec wfSpec = processWindowFrame(wfNode); + ws.setWindowFrame(wfSpec); + } + + return ws; + } + + private WindowFrameSpec processWindowFrame(ASTNode node) { + int type = node.getType(); + BoundarySpec start = null, end = null; + + switch(type) + { + case HiveParser.TOK_WINDOWRANGE: + start = processRangeBoundary((ASTNode) node.getChild(0)); + end = processRangeBoundary((ASTNode) node.getChild(1)); + break; + case HiveParser.TOK_WINDOWVALUES: + start = processValueBoundary((ASTNode) node.getChild(0)); + end = processValueBoundary((ASTNode) node.getChild(1)); + break; + } + + return new WindowFrameSpec(start, end); + } + + private BoundarySpec processRangeBoundary(ASTNode node) { + RangeBoundarySpec rbs = new RangeBoundarySpec(); + BoundarySpec bs = rbs; + int type = node.getType(); + boolean hasAmt = true; + + switch(type) + { + case HiveParser.KW_PRECEDING: + rbs.setDirection(Direction.PRECEDING); + break; + case HiveParser.KW_FOLLOWING: + rbs.setDirection(Direction.FOLLOWING); + break; + case HiveParser.KW_CURRENT: + bs = new CurrentRowSpec(); + hasAmt = false; + break; + } + + if ( hasAmt ) + { + ASTNode amtNode = (ASTNode) node.getChild(0); + if ( amtNode.getType() == HiveParser.KW_UNBOUNDED) + { + rbs.setAmt(BoundarySpec.UNBOUNDED_AMOUNT); + } + else + { + rbs.setAmt(Integer.parseInt(amtNode.getText())); + } + } + + return bs; + } + + private BoundarySpec processValueBoundary(ASTNode node) { + BoundarySpec bs = null; + int type = node.getType(); + + switch(type) + { + case HiveParser.KW_PRECEDING: + bs = new RangeBoundarySpec(Direction.PRECEDING, BoundarySpec.UNBOUNDED_AMOUNT); + break; + case HiveParser.KW_FOLLOWING: + bs = new RangeBoundarySpec(Direction.FOLLOWING, BoundarySpec.UNBOUNDED_AMOUNT); + break; + case HiveParser.KW_CURRENT: + bs = new CurrentRowSpec(); + break; + case HiveParser.KW_LESS: + bs = new ValueBoundarySpec(Direction.PRECEDING, + (ASTNode) node.getChild(0), + Integer.parseInt(node.getChild(1).getText())); + break; + case HiveParser.KW_MORE: + bs = new ValueBoundarySpec(Direction.FOLLOWING, + (ASTNode) node.getChild(0), + Integer.parseInt(node.getChild(1).getText())); + break; + } + return bs; + } + + /* + * check if a Select Expr is a constant. + * - current logic used is to look for HiveParser.TOK_TABLE_OR_COL + * - if there is none then the expression is a constant. + */ + private static class ConstantExprCheck implements ContextVisitor { + boolean isConstant = true; + + public void visit(Object t, Object parent, int childIndex, Map labels) { + if ( !isConstant ) { + return; + } + ASTNode node = (ASTNode) t; + if (ParseDriver.adaptor.getType(t) == HiveParser.TOK_TABLE_OR_COL ) { + isConstant = false; + } + } + + public void reset() { + isConstant = true; + } + + protected boolean isConstant() { + return isConstant; + } + } + + private static class AggregationExprCheck implements ContextVisitor { + HashMap destAggrExprs; + boolean isAggr = false; + + public AggregationExprCheck(HashMap destAggrExprs) { + super(); + this.destAggrExprs = destAggrExprs; + } + + public void visit(Object t, Object parent, int childIndex, Map labels) { + if ( isAggr ) { + return; + } + if ( destAggrExprs.values().contains(t)) { + isAggr = true; + } + } + + public void reset() { + isAggr = false; + } + + protected boolean isAggr() { + return isAggr; + } + } + + /* + * If we decide to create a PTFSpec for the QB (see ensurePTFSpecOnQB(...)) + * then this method is invoked. + * Constructs the function expr and alias list and invokes addWindowingFuncsToPTFSpec + */ + private void moveaggregationExprsToWindowingSpec(QB currQB, String dest) + throws SemanticException { + HashMap aggregationTree = + currQB.getParseInfo().getAggregationExprsForClause(dest); + if((aggregationTree != null) && !(aggregationTree.isEmpty()) ){ + WindowingSpec spec = currQB.getWindowingSpec(dest); + if(spec == null){ + queryProperties.setHasWindowing(true); + spec = new WindowingSpec(); + currQB.addDestToWindowingSpec(dest, spec); + } + Collection aggrExprs = aggregationTree.values(); + for (ASTNode expr : aggrExprs) { + String alias = currQB.getParseInfo().getExprToColumnAlias(expr); + if ( alias == null ) { + /* + * generate an alias using the function AST tree. + * - calling getColAlias with a null InputResolver and null defaultName. + * - so includeFunctionName should be true. + */ + String[] colAlias = getColAlias(expr, null, null, true, -1); + alias = colAlias[1]; + } + WindowFunctionSpec wFn = processWindowFunction(expr, null); + wFn.setAlias(alias); + spec.addWindowFunction(wFn); + currQB.getParseInfo().getAllExprToColumnAlias().remove(expr); + } + currQB.getParseInfo().clearAggregationExprsForClause(dest); + } + } + + /* + * Returns false if there is a SelectExpr that is not a constant or an aggr. + * + */ + private boolean isValidGroupBySelectList(QB currQB, String clause){ + ConstantExprCheck constantExprCheck = new ConstantExprCheck(); + AggregationExprCheck aggrExprCheck = new AggregationExprCheck( + currQB.getParseInfo().getAggregationExprsForClause(clause)); + + TreeWizard tw = new TreeWizard(ParseDriver.adaptor, HiveParser.tokenNames); + ASTNode selectNode = currQB.getParseInfo().getSelForClause(clause); + + /* + * for Select Distinct Queries we don't move any aggregations. + */ + if ( selectNode != null && selectNode.getType() == HiveParser.TOK_SELECTDI ) { + return true; + } + + for (int i = 0; selectNode != null && i < selectNode.getChildCount(); i++) { + ASTNode selectExpr = (ASTNode) selectNode.getChild(i); + //check for TOK_HINTLIST expressions on ast + if(selectExpr.getType() != HiveParser.TOK_SELEXPR){ + continue; + } + + constantExprCheck.reset(); + PTFTranslator.visit(selectExpr.getChild(0), constantExprCheck); + + if ( !constantExprCheck.isConstant() ) { + aggrExprCheck.reset(); + PTFTranslator.visit(selectExpr.getChild(0), aggrExprCheck); + if (!aggrExprCheck.isAggr() ) { + return false; + } + } + + } + return true; + } + + /** + * If the query has no group by and has aggregations, then we treat this as a + * windowing query. + * There is one caveat: if the select list has only aggregations, then + * this is not handled by windowing. + */ + private void ensureWindowingSpecOnQB(QB currQB, String clause) throws SemanticException{ + if( currQB.getParseInfo().getAggregationExprsForClause(clause) != null + && !queryProperties.hasGroupBy()){ + boolean isValid = isValidGroupBySelectList(currQB, clause); + if(!isValid) { + moveaggregationExprsToWindowingSpec(currQB, clause); + } + } + } + + +//--------------------------- PTF handling: PTFInvocationSpec to PTFDesc -------------------------- + + private PTFDesc translatePTFInvocationSpec(PTFInvocationSpec ptfQSpec, RowResolver inputRR) + throws SemanticException{ + PTFDesc ptfDesc = null; + PTFTranslator translator = new PTFTranslator(); + ptfDesc = translator.translate(ptfQSpec, this, conf, inputRR, unparseTranslator); + return ptfDesc; + } + + Operator genPTFPlan(PTFInvocationSpec ptfQSpec, Operator input) throws SemanticException { + ArrayList componentQueries = PTFTranslator.componentize(ptfQSpec); + for (PTFInvocationSpec ptfSpec : componentQueries) { + input = genPTFPlanForComponentQuery(ptfSpec, input); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Created PTF Plan "); + } + return input; + } + + + /** + * Construct the data structures containing ExprNodeDesc for partition + * columns and order columns. Use the input definition to construct the list + * of output columns for the ReduceSinkOperator + * + * @throws SemanticException + */ + void buildPTFReduceSinkDetails(PartitionedTableFunctionDef tabDef, + RowResolver inputRR, + ArrayList partCols, + ArrayList valueCols, + ArrayList orderCols, + Map colExprMap, + List outputColumnNames, + StringBuilder orderString, + RowResolver extractRR) throws SemanticException { + + ArrayList partColList = tabDef.getPartition().getExpressions(); + + for (PTFExpressionDef colDef : partColList) { + partCols.add(colDef.getExprNode()); + } + + /* + * Order columns are used as key columns for constructing + * the ReduceSinkOperator + * Since we do not explicitly add these to outputColumnNames, + * we need to set includeKeyCols = false while creating the + * ReduceSinkDesc + */ + ArrayList orderColList = tabDef.getOrder().getExpressions(); + for (int i = 0; i < orderColList.size(); i++) { + OrderExpressionDef colDef = orderColList.get(i); + org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order order = colDef.getOrder(); + if (order.name().equals("ASC")) { + orderString.append('+'); + } else { + orderString.append('-'); + } + orderCols.add(colDef.getExprNode()); + } + + /* + * We add the column to value columns or output column names + * only if it is not a virtual column + */ + ArrayList colInfoList = inputRR.getColumnInfos(); + LinkedHashMap colsAddedByHaving = + new LinkedHashMap(); + int pos = 0; + for (ColumnInfo colInfo : colInfoList) { + if (!colInfo.isHiddenVirtualCol()) { + ExprNodeDesc valueColExpr = new ExprNodeColumnDesc(colInfo.getType(), colInfo + .getInternalName(), colInfo.getTabAlias(), colInfo + .getIsVirtualCol()); + valueCols.add(valueColExpr); + colExprMap.put(colInfo.getInternalName(), valueColExpr); + String outColName = SemanticAnalyzer.getColumnInternalName(pos++); + outputColumnNames.add(outColName); + + String[] alias = inputRR.reverseLookup(colInfo.getInternalName()); + /* + * if we have already encountered this colInfo internalName. + * We encounter it again because it must be put for the Having clause. + * We will add these entries in the end; in a loop on colsAddedByHaving. See below. + */ + if ( colsAddedByHaving.containsKey(alias)) { + continue; + } + ASTNode astNode = PTFTranslator.getASTNode(colInfo, inputRR); + ColumnInfo eColInfo = new ColumnInfo( + outColName, colInfo.getType(), alias[0], + colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()); + + if ( astNode == null ) { + extractRR.put(alias[0], alias[1], eColInfo); + } + else { + /* + * in case having clause refers to this column may have been added twice; + * once with the ASTNode.toStringTree as the alias + * and then with the real alias. + */ + extractRR.putExpression(astNode, eColInfo); + if ( !astNode.toStringTree().toLowerCase().equals(alias[1]) ) { + colsAddedByHaving.put(alias, eColInfo); + } + } + } + } + + for(Map.Entry columnAddedByHaving : colsAddedByHaving.entrySet() ) { + String[] alias = columnAddedByHaving.getKey(); + ColumnInfo eColInfo = columnAddedByHaving.getValue(); + extractRR.put(alias[0], alias[1], eColInfo); + } + } + + private Operator genPTFPlanForComponentQuery(PTFInvocationSpec ptfQSpec, Operator input) + throws SemanticException { + /* + * 1. Create the PTFDesc from the Qspec attached to this QB. + */ + RowResolver rr = opParseCtx.get(input).getRowResolver(); + PTFDesc ptfDesc = translatePTFInvocationSpec(ptfQSpec, rr); + /* + * Build an RR for the Extract Op from the ResuceSink Op's RR. + * Why? + * We need to remove the Virtual Columns present in the RS's RR. The OI + * that gets passed to Extract at runtime doesn't contain the Virtual Columns. + * So internal names get changed. Consider testCase testJoinWithLeadLag, + * which is a self join on part and also has a Windowing expression. + * The RR of the RS op at transaltion time looks something like this: + * (_co1,_col2,..,_col7, _col8(vc=true),_col9(vc=true), + * _col10,_col11,.._col15(vc=true),_col16(vc=true),..) + * At runtime the Virtual columns are removed and all the columns after _col7 + * are shifted 1 or 2 positions. + * So in child Operators ColumnExprNodeDesc's are no longer referring to the right columns. + * + * So we build a new RR for the Extract Op, with the Virtual Columns removed. + * We hand this to the PTFTranslator as the + * starting RR to use to translate a PTF Chain. + */ + RowResolver extractOpRR = new RowResolver(); + + /* + * 2. build Map-side Op Graph. Graph template is either: + * Input -> PTF_map -> ReduceSink + * or + * Input -> ReduceSink + * + * Here the ExprNodeDescriptors in the QueryDef are based on the Input Operator's RR. + */ + { + PartitionedTableFunctionDef tabDef = ptfDesc.getStartOfChain(); + + /* + * a. add Map-side PTF Operator if needed + */ + if (tabDef.isTransformsRawInput() ) + { + RowResolver ptfMapRR = tabDef.getRawInputShape().getRr(); + + input = putOpInsertMap(OperatorFactory.getAndMakeChild(ptfDesc, + new RowSchema(ptfMapRR.getColumnInfos()), + input), ptfMapRR); + rr = opParseCtx.get(input).getRowResolver(); + } + + /* + * b. Build Reduce Sink Details (keyCols, valueCols, outColNames etc.) for this ptfDesc. + */ + + ArrayList partCols = new ArrayList(); + ArrayList valueCols = new ArrayList(); + ArrayList orderCols = new ArrayList(); + Map colExprMap = new HashMap(); + List outputColumnNames = new ArrayList(); + StringBuilder orderString = new StringBuilder(); + + /* + * Use the input RR of TableScanOperator in case there is no map-side + * reshape of input. + * If the parent of ReduceSinkOperator is PTFOperator, use it's + * output RR. + */ + buildPTFReduceSinkDetails(tabDef, + rr, + partCols, + valueCols, + orderCols, + colExprMap, + outputColumnNames, + orderString, + extractOpRR); + + input = putOpInsertMap(OperatorFactory.getAndMakeChild(PlanUtils + .getReduceSinkDesc(orderCols, + valueCols, outputColumnNames, false, + -1, partCols, orderString.toString(), -1), + new RowSchema(rr.getColumnInfos()), input), rr); + input.setColumnExprMap(colExprMap); + } + + /* + * 3. build Reduce-side Op Graph + */ + { + /* + * b. Construct Extract Operator. + */ + input = putOpInsertMap(OperatorFactory.getAndMakeChild( + new ExtractDesc( + new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, + Utilities.ReduceField.VALUE + .toString(), "", false)), + new RowSchema(extractOpRR.getColumnInfos()), + input), extractOpRR); + + /* + * c. Rebuilt the QueryDef. + * Why? + * - so that the ExprNodeDescriptors in the QueryDef are based on the + * Extract Operator's RowResolver + */ + rr = opParseCtx.get(input).getRowResolver(); + ptfDesc = translatePTFInvocationSpec(ptfQSpec, rr); + + /* + * d. Construct PTF Operator. + */ + RowResolver ptfOpRR = ptfDesc.getFuncDef().getOutputShape().getRr(); + input = putOpInsertMap(OperatorFactory.getAndMakeChild(ptfDesc, + new RowSchema(ptfOpRR.getColumnInfos()), + input), ptfOpRR); + + } + + return input; + + } + +//--------------------------- Windowing handling: PTFInvocationSpec to PTFDesc -------------------- + + private PTFDesc translateWindowingSpec(WindowingSpec wSpec, RowResolver inputRR) + throws SemanticException{ + PTFDesc ptfDesc = null; + PTFTranslator translator = new PTFTranslator(); + ptfDesc = translator.translate(wSpec, this, conf, inputRR, unparseTranslator); + return ptfDesc; + } + + Operator genWindowingPlan(WindowingSpec wSpec, Operator input) throws SemanticException { + + RowResolver rr = opParseCtx.get(input).getRowResolver(); + input = genReduceSinkPlanForWindowing(wSpec, rr, input); + + rr = opParseCtx.get(input).getRowResolver(); + PTFDesc ptfDesc = translateWindowingSpec(wSpec, rr); + + /* + * d. Construct PTF Operator. + */ + RowResolver ptfOpRR = ptfDesc.getFuncDef().getOutputShape().getRr(); + input = putOpInsertMap(OperatorFactory.getAndMakeChild(ptfDesc, + new RowSchema(ptfOpRR.getColumnInfos()), + input), ptfOpRR); + + dumpOperatorChain(input, null); + + return input; + } + + private Operator genReduceSinkPlanForWindowing(WindowingSpec spec, + RowResolver inputRR, + Operator input) throws SemanticException{ + ArrayList partCols = new ArrayList(); + ArrayList valueCols = new ArrayList(); + ArrayList orderCols = new ArrayList(); + Map colExprMap = new HashMap(); + List outputColumnNames = new ArrayList(); + StringBuilder orderString = new StringBuilder(); + + ArrayList partColList = spec.getQueryPartitionSpec().getExpressions(); + for (PartitionExpression partCol : partColList) { + ExprNodeDesc partExpr = genExprNodeDesc(partCol.getExpression(), inputRR); + partCols.add(partExpr); + } + + ArrayList orderColList = spec.getQueryOrderSpec() == null ? + new ArrayList() : + spec.getQueryOrderSpec().getExpressions(); + ArrayList combinedOrderList = + PTFTranslator.addPartitionExpressionsToOrderList(partColList, orderColList); + for (int i = 0; i < combinedOrderList.size(); i++) { + OrderExpression orderCol = combinedOrderList.get(i); + org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order order = orderCol.getOrder(); + if (order.name().equals("ASC")) { + orderString.append('+'); + } else { + orderString.append('-'); + } + ExprNodeDesc orderExpr = genExprNodeDesc(orderCol.getExpression(), inputRR); + orderCols.add(orderExpr); + } + + ArrayList colInfoList = inputRR.getColumnInfos(); + RowResolver rsNewRR = new RowResolver(); + int pos = 0; + for (ColumnInfo colInfo : colInfoList) { + ExprNodeDesc valueColExpr = new ExprNodeColumnDesc(colInfo.getType(), colInfo + .getInternalName(), colInfo.getTabAlias(), colInfo + .getIsVirtualCol()); + valueCols.add(valueColExpr); + colExprMap.put(colInfo.getInternalName(), valueColExpr); + String outColName = SemanticAnalyzer.getColumnInternalName(pos++); + outputColumnNames.add(outColName); + + String[] alias = inputRR.reverseLookup(colInfo.getInternalName()); + ColumnInfo newColInfo = new ColumnInfo( + outColName, colInfo.getType(), alias[0], + colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()); + rsNewRR.put(alias[0], alias[1], newColInfo); + + } + + input = putOpInsertMap(OperatorFactory.getAndMakeChild(PlanUtils + .getReduceSinkDesc(orderCols, + valueCols, outputColumnNames, false, + -1, partCols, orderString.toString(), -1), + new RowSchema(inputRR.getColumnInfos()), input), rsNewRR); + input.setColumnExprMap(colExprMap); + + + // Construct the RR for extract operator + RowResolver extractRR = new RowResolver(); + LinkedHashMap colsAddedByHaving = + new LinkedHashMap(); + pos = 0; + + for (ColumnInfo colInfo : colInfoList) { + String[] alias = inputRR.reverseLookup(colInfo.getInternalName()); + /* + * if we have already encountered this colInfo internalName. + * We encounter it again because it must be put for the Having clause. + * We will add these entries in the end; in a loop on colsAddedByHaving. See below. + */ + if ( colsAddedByHaving.containsKey(alias)) { + continue; + } + ASTNode astNode = PTFTranslator.getASTNode(colInfo, inputRR); + ColumnInfo eColInfo = new ColumnInfo( + SemanticAnalyzer.getColumnInternalName(pos++), colInfo.getType(), alias[0], + colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()); + + if ( astNode == null ) { + extractRR.put(alias[0], alias[1], eColInfo); + } + else { + /* + * in case having clause refers to this column may have been added twice; + * once with the ASTNode.toStringTree as the alias + * and then with the real alias. + */ + extractRR.putExpression(astNode, eColInfo); + if ( !astNode.toStringTree().toLowerCase().equals(alias[1]) ) { + colsAddedByHaving.put(alias, eColInfo); + } + } + } + + for(Map.Entry columnAddedByHaving : colsAddedByHaving.entrySet() ) { + String[] alias = columnAddedByHaving.getKey(); + ColumnInfo eColInfo = columnAddedByHaving.getValue(); + extractRR.put(alias[0], alias[1], eColInfo); + } + + /* + * b. Construct Extract Operator. + */ + input = putOpInsertMap(OperatorFactory.getAndMakeChild( + new ExtractDesc( + new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, + Utilities.ReduceField.VALUE + .toString(), "", false)), + new RowSchema(inputRR.getColumnInfos()), + input), extractRR); + + + return input; + } + + + public static ArrayList parseSelect(String selectExprStr) + throws SemanticException + { + ASTNode selNode = null; + try { + ParseDriver pd = new ParseDriver(); + selNode = pd.parseSelect(selectExprStr, null); + } catch (ParseException pe) { + throw new SemanticException(pe); + } + + ArrayList selSpec = new ArrayList(); + int childCount = selNode.getChildCount(); + for (int i = 0; i < childCount; i++) { + ASTNode selExpr = (ASTNode) selNode.getChild(i); + if (selExpr.getType() != HiveParser.TOK_SELEXPR) { + throw new SemanticException(String.format( + "Only Select expressions supported in dynamic select list: %s", selectExprStr)); + } + ASTNode expr = (ASTNode) selExpr.getChild(0); + if (expr.getType() == HiveParser.TOK_ALLCOLREF) { + throw new SemanticException( + String.format("'%s' column not allowed in dynamic select list", selectExprStr)); + } + ASTNode aliasNode = selExpr.getChildCount() > 1 + && selExpr.getChild(1).getType() == HiveParser.Identifier ? + (ASTNode) selExpr.getChild(1) : null; + String alias = null; + if ( aliasNode != null ) { + alias = aliasNode.getText(); + } + else { + String[] tabColAlias = getColAlias(selExpr, null, null, true, -1); + alias = tabColAlias[1]; + } + WindowExpressionSpec exprSpec = new WindowExpressionSpec(); + exprSpec.setAlias(alias); + exprSpec.setExpression(expr); + selSpec.add(exprSpec); + } + + return selSpec; + } + + // debug methods + void dumpOperatorChain(Operator sinkOp, PTFDesc ptfDesc) { + Stack stack = new Stack(); + Operator op = sinkOp; + while(op != null ) { + stack.push(op); + List parentOps =op.getParentOperators(); + if (parentOps != null ) { + op = parentOps.get(0); + } + else { + op = null; + } + } + + int opNum = 1; + StringBuilder buf = new StringBuilder(); + while(!stack.isEmpty()) { + op = stack.pop(); + buf.append("\n").append(opNum).append("."); + buf.append(op.getName()); + buf.append(" :\n"); + RowResolver rr = opParseCtx.get(op).getRowResolver(); + dumpRowResolver(buf, rr); + if ( op instanceof PTFOperator && ptfDesc != null ) { + /* + * 1/21 hb: this is no longer correct; in a chain containing multiple PTFOps, + * every PTFOp dump prints the info from the + * last PTFDef + */ + dump(buf, ptfDesc); + } + opNum++; + } + System.out.println(buf); + } + + static void dumpRowResolver(StringBuilder buf, RowResolver rr) { + buf.append("RowResolver::\n"); + buf.append("\tcolumns:["); + boolean first = true; + for(ColumnInfo cInfo : rr.getRowSchema().getSignature()) { + String tabalias = cInfo.getTabAlias(); + String cname = cInfo.getInternalName(); + if (!first) { + buf.append(", "); + } else { + first = false; + } + buf.append(tabalias != null ? tabalias : ""); + buf.append("."); + buf.append(cname); + } + buf.append("]\n"); + buf.append("\tAliases:["); + for(Map.Entry> entry : rr.getRslvMap().entrySet() ) { + String tabalias = entry.getKey(); + buf.append("\n\t\t"); + buf.append(tabalias != null ? tabalias : ""); + buf.append(":["); + LinkedHashMap colAliases = entry.getValue(); + first = true; + for(Map.Entry column: colAliases.entrySet()) { + if (!first) { + buf.append(", "); + } else { + first = false; + } + buf.append(column.getKey()).append(" -> ").append(column.getValue().getInternalName()); + } + } + buf.append("\n\t]\n"); + buf.append("\tcolumns mapped to expressions:["); + first = true; + for(Map.Entry exprs : rr.getExpressionMap().entrySet()) { + if (!first) { + buf.append(", "); + } else { + first = false; + } + buf.append("\n\t\t"); + buf.append(exprs.getKey()); + buf.append(" -> "); + buf.append(exprs.getValue().toStringTree()); + } + buf.append("\n\t]\n"); + } + + private static void dump(StringBuilder buf, PTFDesc ptfDesc) { + Stack ptfChain = new Stack(); + PTFInputDef currentDef = ptfDesc.getFuncDef(); + while(currentDef != null ) { + ptfChain.push(currentDef); + currentDef = currentDef.getInput(); + } + + while(!ptfChain.isEmpty() ) { + PTFInputDef iDef = ptfChain.pop(); + if ( iDef instanceof PTFQueryInputDef ) { + dump(buf, (PTFQueryInputDef) iDef); + }else { + dump(buf, (PartitionedTableFunctionDef) iDef); + } + } + + } + + private static void dump(StringBuilder buf, PartitionedTableFunctionDef tFnDef) { + buf.append("\n").append(tFnDef.getName()).append(":"); + dump(buf, (PTFInputDef)tFnDef); + TableFunctionEvaluator tFn = tFnDef.getTFunction(); + + if ( tFn.isTransformsRawInput() ) { + buf.append("\nEvaluator RawInput ObjectInspector:["); + dump(buf, tFn.getRawInputOI()); + buf.append("]"); + } + + buf.append("\nEvaluator Output ObjectInspector:["); + dump(buf, tFn.getOutputOI()); + buf.append("]"); + + } + private static void dump(StringBuilder buf, PTFQueryInputDef htblDef) { + buf.append("\n").append(htblDef.getDestination()).append(":"); + dump(buf, (PTFInputDef)htblDef); + } + + private static void dump(StringBuilder buf, PTFInputDef qInDef) { + StructObjectInspector OI = (StructObjectInspector) qInDef.getOutputShape().getOI(); + buf.append("\nDef ObjectInspector:["); + dump(buf, OI); + buf.append("]\nSerDe:").append(qInDef.getOutputShape().getSerde().getClass().getName()); + } + + private static void dump(StringBuilder buf, StructObjectInspector OI) { + boolean first = true; + for(StructField field : OI.getAllStructFieldRefs() ) { + if (!first) { + buf.append(", "); + } else { + first = false; + } + buf.append(field.getFieldName()); + } + } + } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/WindowingExprNodeEvaluatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/parse/WindowingExprNodeEvaluatorFactory.java new file mode 100644 index 0000000..f14a603 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/WindowingExprNodeEvaluatorFactory.java @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse; + +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.PTFTranslator.LeadLagInfo; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag.GenericUDFLag; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag.GenericUDFLead; + +/* + * When constructing the Evaluator Tree from an ExprNode Tree + * - look for any descendant LeadLag Function Expressions + * - if they are found: + * - add them to the LLInfo.leadLagExprs and + * - add a mapping from the Expr Tree root to the LLFunc Expr in LLInfo.mapTopExprToLLFunExprs + */ +public class WindowingExprNodeEvaluatorFactory { + + public static ExprNodeEvaluator get(LeadLagInfo llInfo, ExprNodeDesc desc) throws HiveException + { + FindLeadLagFuncExprs visitor = new FindLeadLagFuncExprs(llInfo, desc); + new ExprNodeWalker(visitor).walk(desc); + return ExprNodeEvaluatorFactory.get(desc); + } + + public static class FindLeadLagFuncExprs + { + ExprNodeDesc topExpr; + LeadLagInfo llInfo; + + FindLeadLagFuncExprs(LeadLagInfo llInfo, ExprNodeDesc topExpr) + { + this.llInfo = llInfo; + this.topExpr = topExpr; + } + + public void visit(ExprNodeGenericFuncDesc fnExpr) throws HiveException + { + GenericUDF fn = fnExpr.getGenericUDF(); + if (fn instanceof GenericUDFLead || fn instanceof GenericUDFLag ) + { + llInfo.addLLFuncExprForTopExpr(topExpr, fnExpr); + } + } + } + + static class ExprNodeWalker + { + FindLeadLagFuncExprs visitor; + + public ExprNodeWalker(FindLeadLagFuncExprs visitor) + { + super(); + this.visitor = visitor; + } + + public void walk(ExprNodeDesc e) throws HiveException + { + if ( e == null ) { + return; + } + List children = e.getChildren(); + if ( children != null ) + { + for(ExprNodeDesc child : children) + { + walk(child); + } + } + + if ( e instanceof ExprNodeGenericFuncDesc) + { + visitor.visit((ExprNodeGenericFuncDesc)e); + } + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/WindowingSpec.java ql/src/java/org/apache/hadoop/hive/ql/parse/WindowingSpec.java new file mode 100644 index 0000000..1318747 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/WindowingSpec.java @@ -0,0 +1,608 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse; + +import java.util.ArrayList; +import java.util.HashMap; + +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitionSpec; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PartitioningSpec; + +/* + * Captures the Window processing specified in a Query. A Query may + * contain: + * - UDAF invocations on a Window. + * - Lead/Lag function invocations that can only be evaluated in a + * Partition. + * - For Queries that don't have a Group By all UDAF invocations are + * treated as Window Function invocations. + * - For Queries that don't have a Group By, the Having condition is + * handled as a post processing on the rows output by Windowing + * processing. + * Windowing is a container of all the Select Expressions that are + * to be handled by Windowing. These are held in 2 lists: the functions + * list holds WindowFunction invocations; the expressions list holds + * Select Expressions having Lead/Lag function calls. It may also + * contain an ASTNode representing the post filter to apply on the + * output of Window Functions. + * Windowing also contains all the Windows defined in the Query. One of + * the Windows is designated as the 'default' Window. If the Query has a + * Distribute By/Cluster By clause; then the information in these + * clauses is captured as a Partitioning and used as the default Window + * for the Query. Otherwise the first Window specified is treated as the + * default. + * Finally Windowing maintains a Map from an 'alias' to the ASTNode that + * represents the Select Expression that was translated to a Window + * Function invocation or a Window Expression. This is used when + * building RowResolvers. + */ +public class WindowingSpec { + HashMap aliasToWdwExpr; + ASTNode filterExpr; + HashMap windowSpecs; + ArrayList windowExpressions; + + /* + * this is specified explicitly on the Distribute/Cluster by specified at the Query level + * or it is inferred from a WindowingFunction or Window Clause as a convenience. + * As a further convenience functions w/o a + * Partitioning Spec can inherit this from the default. + * The rules for setting the default are the following: + * 1. On encountering the first UDAF with a WindowingSpec we set this. + * 2. On encountering a Distribute/Cluster clause we set this. + * 3. On encountering a Sort clause: + * a. we set the Partitioning's OrderSpec. + * 4. On encountering the first Window Clause, if there is no PartitionSpec we set it from + * the Window Clause. + * If there is no OrderSpec, we set it from the Window Clause. + * If there is a OrderSpec and the Window Clause doesn't then we set the OrderSpec on the + * Window Clause. + */ + PartitioningSpec queryPartitioningSpec; + /* + * set if Query level partitioning is set using this WindowSpec. + */ + WindowSpec sourceOfQueryPartitoningSpec; + + + public void addWindowSpec(String name, WindowSpec wdwSpec) { + windowSpecs = windowSpecs == null ? new HashMap() : windowSpecs; + windowSpecs.put(name, wdwSpec); + inferDefaultWindowingSpec(wdwSpec); + } + + public void addExpression(ASTNode expr, String alias) { + windowExpressions = windowExpressions == null ? + new ArrayList() : windowExpressions; + aliasToWdwExpr = aliasToWdwExpr == null ? + new HashMap() : aliasToWdwExpr; + WindowExpressionSpec wExprSpec = new WindowExpressionSpec(); + wExprSpec.setAlias(alias); + wExprSpec.setExpression(expr); + + windowExpressions.add(wExprSpec); + aliasToWdwExpr.put(alias, wExprSpec); + } + + public void addWindowFunction(WindowFunctionSpec wFn) { + windowExpressions = windowExpressions == null ? + new ArrayList() : windowExpressions; + aliasToWdwExpr = aliasToWdwExpr == null ? + new HashMap() : aliasToWdwExpr; + windowExpressions.add(wFn); + aliasToWdwExpr.put(wFn.getAlias(), wFn); + inferDefaultWindowingSpec(wFn); + } + + public HashMap getAliasToWdwExpr() { + return aliasToWdwExpr; + } + + public void setAliasToWdwExpr(HashMap aliasToWdwExpr) { + this.aliasToWdwExpr = aliasToWdwExpr; + } + + public ASTNode getFilterExpr() { + return filterExpr; + } + + public void setFilterExpr(ASTNode filterExpr) { + this.filterExpr = filterExpr; + } + + public HashMap getWindowSpecs() { + return windowSpecs; + } + + public void setWindowSpecs(HashMap windowSpecs) { + this.windowSpecs = windowSpecs; + } + + public ArrayList getWindowExpressions() { + return windowExpressions; + } + + public void setWindowExpressions(ArrayList windowExpressions) { + this.windowExpressions = windowExpressions; + } + + public WindowSpec getSourceOfQueryPartitoningSpec() { + return sourceOfQueryPartitoningSpec; + } + + public void setSourceOfQueryPartitoningSpec(WindowSpec sourceOfQueryPartitoningSpec) { + this.sourceOfQueryPartitoningSpec = sourceOfQueryPartitoningSpec; + } + + public PartitioningSpec getQueryPartitioningSpec() { + return queryPartitioningSpec; + } + + public PartitionSpec getQueryPartitionSpec() { + return queryPartitioningSpec == null ? null : queryPartitioningSpec.getPartSpec(); + } + + public void setQueryPartitonSpec(PartitionSpec partSpec) { + if ( queryPartitioningSpec == null ) { + queryPartitioningSpec = new PartitioningSpec(); + } + queryPartitioningSpec.setPartSpec(partSpec); + setSourceOfQueryPartitoningSpec(null); + } + + public OrderSpec getQueryOrderSpec() { + return queryPartitioningSpec == null ? null : queryPartitioningSpec.getOrderSpec(); + } + + public void setQueryOrderSpec(OrderSpec orderSpec) { + if ( queryPartitioningSpec == null ) { + queryPartitioningSpec = new PartitioningSpec(); + } + queryPartitioningSpec.setOrderSpec(orderSpec); + if ( getSourceOfQueryPartitoningSpec() != null ) { + WindowSpec dws = getSourceOfQueryPartitoningSpec(); + if ( dws.getOrder() == null ) { + dws.setOrder(orderSpec); + } + } + } + + private void inferDefaultWindowingSpec(WindowFunctionSpec wFnSpec) { + WindowSpec wSpec = wFnSpec == null ? null : wFnSpec.getWindowSpec(); + if ( getSourceOfQueryPartitoningSpec() == null && wSpec != null && + wSpec.getPartition() != null ) { + setQueryPartitonSpec(wSpec.getPartition()); + setQueryOrderSpec(wSpec.getOrder()); + setSourceOfQueryPartitoningSpec(wSpec); + } + } + + private void inferDefaultWindowingSpec(WindowSpec wSpec) { + if ( wSpec == null + || wSpec.getPartition() == null + || getSourceOfQueryPartitoningSpec() != null + || this.getQueryPartitionSpec() != null) { + return; + } + + setQueryPartitonSpec(wSpec.getPartition()); + + if ( getQueryOrderSpec() == null ) { + setQueryOrderSpec(wSpec.getOrder()); + } + else { + if ( wSpec.getOrder() == null ) { + wSpec.setOrder(getQueryOrderSpec()); + } + } + setSourceOfQueryPartitoningSpec(wSpec); + } + + /* + * Represents a Select Expression in the context of Windowing. These can + * refer to the output of Windowing Functions and can navigate the + * Partition using Lead/Lag functions. + */ + public static class WindowExpressionSpec { + String alias; + ASTNode expression; + public String getAlias() { + return alias; + } + public void setAlias(String alias) { + this.alias = alias; + } + public ASTNode getExpression() { + return expression; + } + public void setExpression(ASTNode expression) { + this.expression = expression; + } + } + + /* + * Represents a UDAF invocation in the context of a Window Frame. As + * explained above sometimes UDAFs will be handled as Window Functions + * even w/o an explicit Window specification. This is to support Queries + * that have no Group By clause. A Window Function invocation captures: + * - the ASTNode that represents this invocation + * - its name + * - whether it is star/distinct invocation. + * - its alias + * - and an optional Window specification + */ + public static class WindowFunctionSpec extends WindowExpressionSpec + { + String name; + boolean isStar; + boolean isDistinct; + ArrayList args; + WindowSpec windowSpec; + + public String getName() { + return name; + } + public void setName(String name) { + this.name = name; + } + public boolean isStar() { + return isStar; + } + public void setStar(boolean isStar) { + this.isStar = isStar; + } + public boolean isDistinct() { + return isDistinct; + } + public void setDistinct(boolean isDistinct) { + this.isDistinct = isDistinct; + } + public ArrayList getArgs() { + args = args == null ? new ArrayList() : args; + return args; + } + public void setArgs(ArrayList args) { + this.args = args; + } + public void addArg(ASTNode arg) { + args = args == null ? new ArrayList() : args; + args.add((ASTNode)arg); + } + public WindowSpec getWindowSpec() { + return windowSpec; + } + public void setWindowSpec(WindowSpec windowSpec) { + this.windowSpec = windowSpec; + } + @Override + public String toString() { + StringBuilder buf = new StringBuilder(); + buf.append(name).append("("); + if (isStar ) + { + buf.append("*"); + } + else + { + if ( isDistinct ) + { + buf.append("distinct "); + } + if ( args != null ) + { + boolean first = true; + for(ASTNode arg : args) + { + if ( first) { + first = false; + } else { + buf.append(", "); + } + buf.append(arg.toStringTree()); + } + } + } + + buf.append(")"); + + if ( windowSpec != null ) + { + buf.append(" ").append(windowSpec.toString()); + } + + if ( alias != null ) + { + buf.append(" as ").append(alias); + } + + return buf.toString(); + } + + } + + /* + * It represents a WindowFrame applied to a Partitioning. A Window can + * refer to a source Window by name. The source Window provides the + * basis for this Window definition. This Window specification + * extends/overrides the source Window definition. In our e.g. the + * Select Expression $sum(p_retailprice) over (w1)$ is translated into a + * WindowFunction instance that has a Window specification that refers + * to the global Window Specification 'w1'. The Function's specification + * has no content, but inherits all its attributes from 'w1' during + * subsequent phases of translation. + */ + public static class WindowSpec + { + String sourceId; + PartitioningSpec partitioning; + WindowFrameSpec windowFrame; + public String getSourceId() { + return sourceId; + } + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + public PartitioningSpec getPartitioning() { + return partitioning; + } + public void setPartitioning(PartitioningSpec partitioning) { + this.partitioning = partitioning; + } + public WindowFrameSpec getWindowFrame() { + return windowFrame; + } + public void setWindowFrame(WindowFrameSpec windowFrame) { + this.windowFrame = windowFrame; + } + public PartitionSpec getPartition() { + return getPartitioning() == null ? null : getPartitioning().getPartSpec(); + } + public void setPartition(PartitionSpec partSpec) { + partitioning = partitioning == null ? new PartitioningSpec() : partitioning; + partitioning.setPartSpec(partSpec); + } + public OrderSpec getOrder() { + return getPartitioning() == null ? null : getPartitioning().getOrderSpec(); + } + public void setOrder(OrderSpec orderSpec) { + partitioning = partitioning == null ? new PartitioningSpec() : partitioning; + partitioning.setOrderSpec(orderSpec); + } + }; + + /* + * A WindowFrame specifies the Range on which a Window Function should + * be applied for the 'current' row. Its is specified by a start and + * end Boundary. + */ + public static class WindowFrameSpec + { + BoundarySpec start; + BoundarySpec end; + + public WindowFrameSpec() { + } + + public WindowFrameSpec(BoundarySpec start, BoundarySpec end) + { + super(); + this.start = start; + this.end = end; + } + + public BoundarySpec getStart() + { + return start; + } + + public void setStart(BoundarySpec start) + { + this.start = start; + } + + public BoundarySpec getEnd() + { + return end; + } + + public void setEnd(BoundarySpec end) + { + this.end = end; + } + + @Override + public String toString() + { + return String.format("window(start=%s, end=%s)", start, end); + } + + } + + public static enum Direction + { + PRECEDING, + CURRENT, + FOLLOWING + }; + + /* + * A Boundary specifies how many rows back/forward a WindowFrame extends from the + * current row. A Boundary is specified as: + * - Range Boundary :: as the number of rows to go forward or back from + the Current Row. + * - Current Row :: which implies the Boundary is at the current row. + * - Value Boundary :: which is specified as the amount the value of an + Expression must decrease/increase + */ + public abstract static class BoundarySpec implements Comparable + { + public static int UNBOUNDED_AMOUNT = Integer.MAX_VALUE; + + public abstract Direction getDirection(); + + } + + public static class RangeBoundarySpec extends BoundarySpec + { + + Direction direction; + int amt; + + public RangeBoundarySpec() { + } + + public RangeBoundarySpec(Direction direction, int amt) + { + super(); + this.direction = direction; + this.amt = amt; + } + + @Override + public Direction getDirection() + { + return direction; + } + + public void setDirection(Direction direction) + { + this.direction = direction; + } + + public int getAmt() + { + return amt; + } + + public void setAmt(int amt) + { + this.amt = amt; + } + + @Override + public String toString() + { + return String.format("range(%s %s)", (amt == UNBOUNDED_AMOUNT ? "Unbounded" : amt), + direction); + } + + public int compareTo(BoundarySpec other) + { + int c = direction.compareTo(other.getDirection()); + if (c != 0) { + return c; + } + RangeBoundarySpec rb = (RangeBoundarySpec) other; + return amt - rb.amt; + } + + } + + public static class CurrentRowSpec extends BoundarySpec + { + public CurrentRowSpec() { + } + + @Override + public String toString() + { + return "currentRow"; + } + + @Override + public Direction getDirection() { + return Direction.CURRENT; + } + + public int compareTo(BoundarySpec other) + { + return getDirection().compareTo(other.getDirection()); + } + + } + + public static class ValueBoundarySpec extends BoundarySpec + { + Direction direction; + ASTNode expression; + int amt; + + public ValueBoundarySpec() { + } + + public ValueBoundarySpec(Direction direction, ASTNode expression, + int amt) + { + super(); + this.direction = direction; + this.expression = (ASTNode) expression; + this.amt = amt; + } + + @Override + public Direction getDirection() + { + return direction; + } + + public void setDirection(Direction direction) + { + this.direction = direction; + } + + public ASTNode getExpression() + { + return expression; + } + + public void setExpression(ASTNode expression) + { + this.expression = expression; + } + + public int getAmt() + { + return amt; + } + + public void setAmt(int amt) + { + this.amt = amt; + } + + @Override + public String toString() + { + return String.format("value(%s %s %s)", expression.toStringTree(), amt, direction); + } + + public int compareTo(BoundarySpec other) + { + int c = direction.compareTo(other.getDirection()); + if (c != 0) { + return c; + } + ValueBoundarySpec vb = (ValueBoundarySpec) other; + return amt - vb.amt; + } + + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDesc.java new file mode 100644 index 0000000..a0b86e6 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDesc.java @@ -0,0 +1,636 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.util.ArrayList; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.PTFUtils; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order; +import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.PTFQueryInputType; +import org.apache.hadoop.hive.ql.parse.PTFTranslator.LeadLagInfo; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.TypeCheckCtx; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.Direction; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionEvaluator; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; + +@Explain(displayName = "PTF Operator") +public class PTFDesc extends AbstractOperatorDesc +{ + private static final long serialVersionUID = 1L; + @SuppressWarnings("unused") + private static final Log LOG = LogFactory.getLog(PTFDesc.class.getName()); + + PartitionedTableFunctionDef funcDef; + LeadLagInfo llInfo; + + static{ + PTFUtils.makeTransient(PTFDesc.class, "llInfo"); + } + + public PartitionedTableFunctionDef getFuncDef() { + return funcDef; + } + + public void setFuncDef(PartitionedTableFunctionDef funcDef) { + this.funcDef = funcDef; + } + + public PartitionedTableFunctionDef getStartOfChain() { + return funcDef == null ? null : funcDef.getStartOfChain(); + } + + public LeadLagInfo getLlInfo() { + return llInfo; + } + + public void setLlInfo(LeadLagInfo llInfo) { + this.llInfo = llInfo; + } + + public boolean forWindowing() { + return funcDef != null && (funcDef instanceof WindowTableFunctionDef); + } + + public abstract static class PTFInputDef { + String expressionTreeString; + ShapeDetails outputShape; + String alias; + + public String getExpressionTreeString() { + return expressionTreeString; + } + + public void setExpressionTreeString(String expressionTreeString) { + this.expressionTreeString = expressionTreeString; + } + + public ShapeDetails getOutputShape() { + return outputShape; + } + + public void setOutputShape(ShapeDetails outputShape) { + this.outputShape = outputShape; + } + public String getAlias() { + return alias; + } + public void setAlias(String alias) { + this.alias = alias; + } + + public abstract PTFInputDef getInput(); + } + + public static class PTFQueryInputDef extends PTFInputDef { + String destination; + PTFQueryInputType type; + public String getDestination() { + return destination; + } + public void setDestination(String destination) { + this.destination = destination; + } + public PTFQueryInputType getType() { + return type; + } + public void setType(PTFQueryInputType type) { + this.type = type; + } + + @Override + public PTFInputDef getInput() { + return null; + } + } + + public static class PartitionedTableFunctionDef extends PTFInputDef { + String name; + String resolverClassName; + ShapeDetails rawInputShape; + boolean carryForwardNames; + PTFInputDef input; + ArrayList args; + PartitionDef partition; + OrderDef order; + TableFunctionEvaluator tFunction; + boolean transformsRawInput; + public String getName() { + return name; + } + public void setName(String name) { + this.name = name; + } + public ShapeDetails getRawInputShape() { + return rawInputShape; + } + public void setRawInputShape(ShapeDetails rawInputShape) { + this.rawInputShape = rawInputShape; + } + public boolean isCarryForwardNames() { + return carryForwardNames; + } + public void setCarryForwardNames(boolean carryForwardNames) { + this.carryForwardNames = carryForwardNames; + } + @Override + public PTFInputDef getInput() { + return input; + } + public void setInput(PTFInputDef input) { + this.input = input; + } + public PartitionDef getPartition() { + return partition; + } + public void setPartition(PartitionDef partition) { + this.partition = partition; + } + public OrderDef getOrder() { + return order; + } + public void setOrder(OrderDef order) { + this.order = order; + } + public TableFunctionEvaluator getTFunction() { + return tFunction; + } + public void setTFunction(TableFunctionEvaluator tFunction) { + this.tFunction = tFunction; + } + public ArrayList getArgs() { + return args; + } + + public void setArgs(ArrayList args) { + this.args = args; + } + + public void addArg(PTFExpressionDef arg) { + args = args == null ? new ArrayList() : args; + args.add(arg); + } + + public PartitionedTableFunctionDef getStartOfChain() { + if (input instanceof PartitionedTableFunctionDef ) { + return ((PartitionedTableFunctionDef)input).getStartOfChain(); + } + return this; + } + public boolean isTransformsRawInput() { + return transformsRawInput; + } + public void setTransformsRawInput(boolean transformsRawInput) { + this.transformsRawInput = transformsRawInput; + } + public String getResolverClassName() { + return resolverClassName; + } + public void setResolverClassName(String resolverClassName) { + this.resolverClassName = resolverClassName; + } + } + + public static class WindowTableFunctionDef extends PartitionedTableFunctionDef { + PTFExpressionDef havingExpression; + ArrayList windowExpressions; + ArrayList windowFunctions; + /* + * this shape omits the non WdwFunction Expressions. Expr Evaluators for the Window Expressions is based on this + * shape, so they can refer to the Wdw Function values. + * @note: this will eventually be removed, as plan is to push Wdw expression processing to separate Select Op after + * PTF Op. + */ + ShapeDetails outputFromWdwFnProcessing; + + public PTFExpressionDef getHavingExpression() { + return havingExpression; + } + public void setHavingExpression(PTFExpressionDef havingExpression) { + this.havingExpression = havingExpression; + } + public ArrayList getWindowExpressions() { + return windowExpressions; + } + public void setWindowExpressions(ArrayList windowExpressions) { + this.windowExpressions = windowExpressions; + } + public ShapeDetails getOutputFromWdwFnProcessing() { + return outputFromWdwFnProcessing; + } + public void setOutputFromWdwFnProcessing(ShapeDetails outputFromWdwFnProcessing) { + this.outputFromWdwFnProcessing = outputFromWdwFnProcessing; + } + public ArrayList getWindowFunctions() { + return windowFunctions; + } + public void setWindowFunctions(ArrayList windowFunctions) { + this.windowFunctions = windowFunctions; + } + } + + public static class ShapeDetails { + String serdeClassName; + Map serdeProps; + ArrayList columnNames; + transient StructObjectInspector OI; + transient SerDe serde; + transient RowResolver rr; + transient TypeCheckCtx typeCheckCtx; + + static{ + PTFUtils.makeTransient(ShapeDetails.class, "serde"); + PTFUtils.makeTransient(ShapeDetails.class, "OI"); + PTFUtils.makeTransient(ShapeDetails.class, "rr"); + PTFUtils.makeTransient(ShapeDetails.class, "typeCheckCtx"); + } + + public String getSerdeClassName() { + return serdeClassName; + } + + public void setSerdeClassName(String serdeClassName) { + this.serdeClassName = serdeClassName; + } + + public Map getSerdeProps() { + return serdeProps; + } + + public void setSerdeProps(Map serdeProps) { + this.serdeProps = serdeProps; + } + + public ArrayList getColumnNames() { + return columnNames; + } + + public void setColumnNames(ArrayList columnNames) { + this.columnNames = columnNames; + } + + public StructObjectInspector getOI() { + return OI; + } + + public void setOI(StructObjectInspector oI) { + OI = oI; + } + + public SerDe getSerde() { + return serde; + } + + public void setSerde(SerDe serde) { + this.serde = serde; + } + + public RowResolver getRr() { + return rr; + } + + public void setRr(RowResolver rr) { + this.rr = rr; + } + + public TypeCheckCtx getTypeCheckCtx() { + return typeCheckCtx; + } + + public void setTypeCheckCtx(TypeCheckCtx typeCheckCtx) { + this.typeCheckCtx = typeCheckCtx; + } + } + + public static class PartitionDef { + ArrayList expressions; + + public ArrayList getExpressions() { + return expressions; + } + + public void setExpressions(ArrayList expressions) { + this.expressions = expressions; + } + public void addExpression(PTFExpressionDef e) { + expressions = expressions == null ? new ArrayList() : expressions; + expressions.add(e); + } + } + + public static class OrderDef { + ArrayList expressions; + + public OrderDef() {} + + public OrderDef(PartitionDef pDef) { + for(PTFExpressionDef eDef : pDef.getExpressions()) + { + addExpression(new OrderExpressionDef(eDef)); + } + } + + public ArrayList getExpressions() { + return expressions; + } + + public void setExpressions(ArrayList expressions) { + this.expressions = expressions; + } + public void addExpression(OrderExpressionDef e) { + expressions = expressions == null ? new ArrayList() : expressions; + expressions.add(e); + } + } + + public static class OrderExpressionDef extends PTFExpressionDef { + Order order; + + public OrderExpressionDef() {} + public OrderExpressionDef(PTFExpressionDef e) { + super(e); + order = Order.ASC; + } + + public Order getOrder() { + return order; + } + + public void setOrder(Order order) { + this.order = order; + } + } + + public static class WindowExpressionDef extends PTFExpressionDef { + String alias; + + public WindowExpressionDef() {} + public WindowExpressionDef(PTFExpressionDef eDef) { + super(eDef); + } + public String getAlias() { + return alias; + } + + public void setAlias(String alias) { + this.alias = alias; + } + } + + public static class WindowFunctionDef extends WindowExpressionDef + { + String name; + boolean isStar; + boolean isDistinct; + ArrayList args; + WindowFrameDef windowFrame; + GenericUDAFEvaluator wFnEval; + boolean pivotResult; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public boolean isStar() { + return isStar; + } + + public void setStar(boolean isStar) { + this.isStar = isStar; + } + + public boolean isDistinct() { + return isDistinct; + } + + public void setDistinct(boolean isDistinct) { + this.isDistinct = isDistinct; + } + + public ArrayList getArgs() { + return args; + } + + public void setArgs(ArrayList args) { + this.args = args; + } + + public void addArg(PTFExpressionDef arg) { + args = args == null ? new ArrayList() : args; + args.add(arg); + } + + public WindowFrameDef getWindowFrame() { + return windowFrame; + } + + public void setWindowFrame(WindowFrameDef windowFrame) { + this.windowFrame = windowFrame; + } + + public GenericUDAFEvaluator getWFnEval() { + return wFnEval; + } + + public void setWFnEval(GenericUDAFEvaluator wFnEval) { + this.wFnEval = wFnEval; + } + + public boolean isPivotResult() { + return pivotResult; + } + + public void setPivotResult(boolean pivotResult) { + this.pivotResult = pivotResult; + } + + } + + public static class WindowFrameDef + { + BoundaryDef start; + BoundaryDef end; + public BoundaryDef getStart() { + return start; + } + public void setStart(BoundaryDef start) { + this.start = start; + } + public BoundaryDef getEnd() { + return end; + } + public void setEnd(BoundaryDef end) { + this.end = end; + } + } + + public static abstract class BoundaryDef { + Direction direction; + + public Direction getDirection() { + return direction; + } + + public void setDirection(Direction direction) { + this.direction = direction; + } + } + + public static class RangeBoundaryDef extends BoundaryDef { + int amt; + + public int compareTo(BoundaryDef other) + { + int c = getDirection().compareTo(other.getDirection()); + if ( c != 0) { + return c; + } + RangeBoundaryDef rb = (RangeBoundaryDef) other; + return getAmt() - rb.getAmt(); + } + + public int getAmt() { + return amt; + } + + public void setAmt(int amt) { + this.amt = amt; + } + } + + public static class CurrentRowDef extends BoundaryDef + { + public int compareTo(BoundaryDef other) + { + return getDirection().compareTo(other.getDirection()); + } + @Override + public Direction getDirection() { + return Direction.CURRENT; + } + } + + public static class ValueBoundaryDef extends BoundaryDef + { + PTFExpressionDef expressionDef; + int amt; + + public int compareTo(BoundaryDef other) { + int c = getDirection().compareTo(other.getDirection()); + if ( c != 0) { + return c; + } + ValueBoundaryDef vb = (ValueBoundaryDef) other; + return getAmt() - vb.getAmt(); + } + + public PTFExpressionDef getExpressionDef() { + return expressionDef; + } + + public void setExpressionDef(PTFExpressionDef expressionDef) { + this.expressionDef = expressionDef; + } + + public ExprNodeDesc getExprNode() { + return expressionDef == null ? null : expressionDef.getExprNode(); + } + + public ExprNodeEvaluator getExprEvaluator() { + return expressionDef == null ? null : expressionDef.getExprEvaluator(); + } + + public ObjectInspector getOI() { + return expressionDef == null ? null : expressionDef.getOI(); + } + + public int getAmt() { + return amt; + } + + public void setAmt(int amt) { + this.amt = amt; + } + } + + public static class PTFExpressionDef + { + String expressionTreeString; + ExprNodeDesc exprNode; + transient ExprNodeEvaluator exprEvaluator; + transient ObjectInspector OI; + + static{ + PTFUtils.makeTransient(PTFExpressionDef.class, "exprEvaluator"); + PTFUtils.makeTransient(PTFExpressionDef.class, "OI"); + } + + public PTFExpressionDef() {} + public PTFExpressionDef(PTFExpressionDef e) { + expressionTreeString = e.getExpressionTreeString(); + exprNode = e.getExprNode(); + exprEvaluator = e.getExprEvaluator(); + OI = e.getOI(); + } + + public String getExpressionTreeString() { + return expressionTreeString; + } + + public void setExpressionTreeString(String expressionTreeString) { + this.expressionTreeString = expressionTreeString; + } + + public ExprNodeDesc getExprNode() { + return exprNode; + } + + public void setExprNode(ExprNodeDesc exprNode) { + this.exprNode = exprNode; + } + + public ExprNodeEvaluator getExprEvaluator() { + return exprEvaluator; + } + + public void setExprEvaluator(ExprNodeEvaluator exprEvaluator) { + this.exprEvaluator = exprEvaluator; + } + + public ObjectInspector getOI() { + return OI; + } + + public void setOI(ObjectInspector oI) { + OI = oI; + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDeserializer.java ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDeserializer.java new file mode 100644 index 0000000..2aef84e --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDeserializer.java @@ -0,0 +1,335 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Stack; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.PTFTranslator.LeadLagInfo; +import org.apache.hadoop.hive.ql.parse.WindowingExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.plan.PTFDesc.BoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFQueryInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.ShapeDetails; +import org.apache.hadoop.hive.ql.plan.PTFDesc.ValueBoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowFrameDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowTableFunctionDef; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLeadLag; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionEvaluator; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver; +import org.apache.hadoop.hive.ql.udf.ptf.WindowingTableFunction.WindowingTableFunctionResolver; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.util.ReflectionUtils; + +public class PTFDeserializer { + + PTFDesc ptfDesc; + StructObjectInspector inputOI; + HiveConf hConf; + LeadLagInfo llInfo; + + public PTFDeserializer(PTFDesc ptfDesc, StructObjectInspector inputOI, HiveConf hConf) { + super(); + this.ptfDesc = ptfDesc; + this.inputOI = inputOI; + this.hConf = hConf; + llInfo = new LeadLagInfo(); + ptfDesc.setLlInfo(llInfo); + } + + public void initializePTFChain(PartitionedTableFunctionDef tblFnDef) throws HiveException { + Stack ptfChain = new Stack(); + PTFInputDef currentDef = tblFnDef; + while (currentDef != null ) { + ptfChain.push(currentDef); + currentDef = currentDef.getInput(); + } + + while ( !ptfChain.isEmpty() ) { + currentDef = ptfChain.pop(); + if ( currentDef instanceof PTFQueryInputDef) { + initialize((PTFQueryInputDef)currentDef); + } + else if ( currentDef instanceof WindowTableFunctionDef) { + initializeWindowing((WindowTableFunctionDef)currentDef); + } + else { + initialize((PartitionedTableFunctionDef)currentDef); + } + } + } + + public void initializeWindowing(WindowTableFunctionDef def) throws HiveException { + ShapeDetails inpShape = def.getInput().getOutputShape(); + + /* + * 1. setup resolve, make connections + */ + TableFunctionEvaluator tEval = def.getTFunction(); + /*WindowingTableFunctionResolver tResolver = (WindowingTableFunctionResolver) + FunctionRegistry.getTableFunctionResolver(def.getName());*/ + WindowingTableFunctionResolver tResolver = + (WindowingTableFunctionResolver) constructResolver(def.getResolverClassName()); + tResolver.initialize(ptfDesc, def, tEval); + + + /* + * 2. initialize WFns. + */ + if ( def.getWindowFunctions() != null ) { + for(WindowFunctionDef wFnDef : def.getWindowFunctions() ) { + + if ( wFnDef.getArgs() != null ) { + for(PTFExpressionDef arg : wFnDef.getArgs()) { + initialize(arg, inpShape); + } + } + + if ( wFnDef.getWindowFrame() != null ) { + WindowFrameDef wFrmDef = wFnDef.getWindowFrame(); + initialize(wFrmDef.getStart(), inpShape); + initialize(wFrmDef.getEnd(), inpShape); + } + setupWdwFnEvaluator(wFnDef); + } + ArrayList aliases = new ArrayList(); + ArrayList fieldOIs = new ArrayList(); + for(WindowFunctionDef wFnDef : def.getWindowFunctions()) { + aliases.add(wFnDef.getAlias()); + if ( wFnDef.isPivotResult() ) { + fieldOIs.add(((ListObjectInspector)wFnDef.getOI()).getListElementObjectInspector()); + } else { + fieldOIs.add(wFnDef.getOI()); + } + } + PTFDeserializer.addInputColumnsToList(inpShape, aliases, fieldOIs); + StructObjectInspector wdwOutOI = ObjectInspectorFactory.getStandardStructObjectInspector( + aliases, fieldOIs); + tResolver.setWdwProcessingOutputOI(wdwOutOI); + initialize(def.getOutputFromWdwFnProcessing()); + } + else { + def.setOutputFromWdwFnProcessing(inpShape); + } + + inpShape = def.getOutputFromWdwFnProcessing(); + + /* + * 3. initialize WExprs. + having clause + */ + if ( def.getWindowExpressions() != null ) { + for(WindowExpressionDef wEDef : def.getWindowExpressions()) { + initialize(wEDef, inpShape); + } + } + + if ( def.getHavingExpression() != null ) { + initialize(def.getHavingExpression(), inpShape); + } + + /* + * 4. give Evaluator chance to setup for Output execution; setup Output shape. + */ + initialize(def.getOutputShape()); + tResolver.initializeOutputOI(); + + /* + * If we have windowExpressions or having then we convert to Std. Object to process; + * we just stream these rows; no need to put in an output Partition. + */ + if ( def.getWindowExpressions().size() > 0 || def.getHavingExpression() != null ) { + StructObjectInspector oi = (StructObjectInspector) + ObjectInspectorUtils.getStandardObjectInspector(def.getOutputShape().getOI()); + def.getOutputShape().setOI(oi); + } + } + + protected void initialize(PTFQueryInputDef def) throws HiveException { + ShapeDetails outShape = def.getOutputShape(); + initialize(outShape); + } + + protected void initialize(PartitionedTableFunctionDef def) throws HiveException { + ShapeDetails inpShape = def.getInput().getOutputShape(); + + /* + * 1. initialize args + */ + if (def.getArgs() != null ) { + for(PTFExpressionDef arg : def.getArgs()) { + initialize(arg, inpShape); + } + } + + /* + * 2. setup resolve, make connections + */ + TableFunctionEvaluator tEval = def.getTFunction(); + //TableFunctionResolver tResolver = FunctionRegistry.getTableFunctionResolver(def.getName()); + TableFunctionResolver tResolver = constructResolver(def.getResolverClassName()); + tResolver.initialize(ptfDesc, def, tEval); + + /* + * 3. give Evaluator chance to setup for RawInput execution; setup RawInput shape + */ + if (tEval.isTransformsRawInput()) + { + tResolver.initializeRawInputOI(); + initialize(def.getRawInputShape()); + } + else { + def.setRawInputShape(inpShape); + } + + inpShape = def.getRawInputShape(); + + /* + * 4. give Evaluator chance to setup for Output execution; setup Output shape. + */ + tResolver.initializeOutputOI(); + initialize(def.getOutputShape()); + } + + static void setupWdwFnEvaluator(WindowFunctionDef def) throws HiveException + { + ArrayList args = def.getArgs(); + ArrayList argOIs = new ArrayList(); + ObjectInspector[] funcArgOIs = null; + + if (args != null) { + for (PTFExpressionDef arg : args) { + argOIs.add(arg.getOI()); + } + funcArgOIs = new ObjectInspector[args.size()]; + funcArgOIs = argOIs.toArray(funcArgOIs); + } + + GenericUDAFEvaluator wFnEval = def.getWFnEval(); + ObjectInspector OI = wFnEval.init(GenericUDAFEvaluator.Mode.COMPLETE, funcArgOIs); + def.setWFnEval(wFnEval); + def.setOI(OI); + } + + protected void initialize(BoundaryDef def, ShapeDetails inpShape) throws HiveException { + if ( def instanceof ValueBoundaryDef ) { + ValueBoundaryDef vDef = (ValueBoundaryDef) def; + initialize(vDef.getExpressionDef(), inpShape); + } + } + + protected void initialize(PTFExpressionDef eDef, ShapeDetails inpShape) throws HiveException { + ExprNodeDesc exprNode = eDef.getExprNode(); + ExprNodeEvaluator exprEval = WindowingExprNodeEvaluatorFactory.get(llInfo, exprNode); + ObjectInspector oi = initExprNodeEvaluator(exprEval, exprNode, inpShape); + eDef.setExprEvaluator(exprEval); + eDef.setOI(oi); + } + + private ObjectInspector initExprNodeEvaluator(ExprNodeEvaluator exprEval, + ExprNodeDesc exprNode, + ShapeDetails inpShape) + throws HiveException + { + ObjectInspector outOI; + outOI = exprEval.initialize(inpShape.getOI()); + + /* + * if there are any LeadLag functions in this Expression Tree: - setup a + * duplicate Evaluator for the 1st arg of the LLFuncDesc - initialize it + * using the InputInfo provided for this Expr tree - set the duplicate + * evaluator on the LLUDF instance. + */ + List llFuncExprs = llInfo.getLLFuncExprsInTopExpr(exprNode); + if (llFuncExprs != null) + { + for (ExprNodeGenericFuncDesc llFuncExpr : llFuncExprs) + { + ExprNodeDesc firstArg = llFuncExpr.getChildren().get(0); + ExprNodeEvaluator dupExprEval = WindowingExprNodeEvaluatorFactory.get(llInfo, firstArg); + dupExprEval.initialize(inpShape.getOI()); + GenericUDFLeadLag llFn = (GenericUDFLeadLag) llFuncExpr.getGenericUDF(); + llFn.setExprEvaluator(dupExprEval); + } + } + + return outOI; + } + + protected void initialize(ShapeDetails shp) throws HiveException { + String serdeClassName = shp.getSerdeClassName(); + Properties serDeProps = new Properties(); + Map serdePropsMap = shp.getSerdeProps(); + for (String serdeName : serdePropsMap.keySet()) { + serDeProps.setProperty(serdeName, serdePropsMap.get(serdeName)); + } + try { + SerDe serDe = (SerDe) SerDeUtils.lookupDeserializer(serdeClassName); + serDe.initialize(hConf, serDeProps); + shp.setSerde(serDe); + shp.setOI((StructObjectInspector) serDe.getObjectInspector()); + } + catch (SerDeException se) + { + throw new HiveException(se); + } + } + + private static void addInputColumnsToList(ShapeDetails shape, + ArrayList fieldNames, ArrayList fieldOIs) + { + StructObjectInspector OI = shape.getOI(); + for (StructField f : OI.getAllStructFieldRefs()) + { + fieldNames.add(f.getFieldName()); + fieldOIs.add(f.getFieldObjectInspector()); + } + } + + private TableFunctionResolver constructResolver(String className) throws HiveException { + try { + @SuppressWarnings("unchecked") + Class rCls = (Class) + Class.forName(className); + return (TableFunctionResolver) ReflectionUtils.newInstance(rCls, null); + } + catch(Exception e) { + throw new HiveException(e); + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java new file mode 100644 index 0000000..bb134c2 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.IntWritable; + +@WindowFunctionDescription +( + description = @Description( + name = "cume_dist", + value = "_FUNC_(x) - The CUME_DIST function (defined as the inverse of percentile in some " + + "statistical books) computes the position of a specified value relative to a set of values. " + + "To compute the CUME_DIST of a value x in a set S of size N, you use the formula: " + + "CUME_DIST(x) = number of values in S coming before " + + " and including x in the specified order/ N" + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFCumeDist extends GenericUDAFRank +{ + + static final Log LOG = LogFactory.getLog(GenericUDAFCumeDist.class.getName()); + + @Override + protected GenericUDAFRankEvaluator createEvaluator() + { + return new GenericUDAFCumeDistEvaluator(); + } + + public static class GenericUDAFCumeDistEvaluator extends GenericUDAFRankEvaluator + { + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException + { + super.init(m, parameters); + return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + ArrayList ranks = ((RankBuffer) agg).rowNums; + double sz = ranks.size(); + ArrayList pranks = new ArrayList(ranks.size()); + + for(IntWritable i : ranks) + { + double pr = ((double)i.get())/sz; + pranks.add(new DoubleWritable(pr)); + } + + return pranks; + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java new file mode 100644 index 0000000..c7c7b0c --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; + +@WindowFunctionDescription +( + description = @Description( + name = "dense_rank", + value = "_FUNC_(x) The difference between RANK and DENSE_RANK is that DENSE_RANK leaves no " + + "gaps in ranking sequence when there are ties. That is, if you were " + + "ranking a competition using DENSE_RANK and had three people tie for " + + "second place, you would say that all three were in second place and " + + "that the next person came in third." + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFDenseRank extends GenericUDAFRank +{ + static final Log LOG = LogFactory.getLog(GenericUDAFDenseRank.class.getName()); + + @Override + protected GenericUDAFRankEvaluator createEvaluator() + { + return new GenericUDAFDenseRankEvaluator(); + } + + public static class GenericUDAFDenseRankEvaluator extends GenericUDAFRankEvaluator + { + /* + * Called when the value in the partition has changed. Update the currentRank + */ + @Override + protected void nextRank(RankBuffer rb) + { + rb.currentRank++; + } + } +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFFirstValue.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFFirstValue.java new file mode 100644 index 0000000..5516a63 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFFirstValue.java @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +@WindowFunctionDescription +( + description = @Description( + name = "first_value", + value = "_FUNC_(x)" + ), + supportsWindow = true, + pivotResult = false +) +public class GenericUDAFFirstValue extends AbstractGenericUDAFResolver +{ + static final Log LOG = LogFactory.getLog(GenericUDAFFirstValue.class.getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException + { + if (parameters.length > 2) + { + throw new UDFArgumentTypeException(2, "At most 2 arguments expected"); + } + if ( parameters.length > 1 && !parameters[1].equals(TypeInfoFactory.booleanTypeInfo) ) + { + throw new UDFArgumentTypeException(1, "second argument must be a boolean expression"); + } + return createEvaluator(); + } + + protected GenericUDAFFirstValueEvaluator createEvaluator() + { + return new GenericUDAFFirstValueEvaluator(); + } + + static class FirstValueBuffer implements AggregationBuffer + { + Object val; + boolean valSet; + boolean firstRow; + boolean skipNulls; + + FirstValueBuffer() + { + init(); + } + + void init() + { + val = null; + valSet = false; + firstRow = true; + skipNulls = false; + } + + } + + public static class GenericUDAFFirstValueEvaluator extends GenericUDAFEvaluator + { + ObjectInspector inputOI; + ObjectInspector outputOI; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException + { + super.init(m, parameters); + if (m != Mode.COMPLETE) + { + throw new HiveException( + "Only COMPLETE mode supported for Rank function"); + } + inputOI = parameters[0]; + outputOI = ObjectInspectorUtils.getStandardObjectInspector(inputOI, ObjectInspectorCopyOption.WRITABLE); + return outputOI; + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException + { + return new FirstValueBuffer(); + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException + { + ((FirstValueBuffer) agg).init(); + } + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException + { + FirstValueBuffer fb = (FirstValueBuffer) agg; + + if (fb.firstRow ) + { + fb.firstRow = false; + if ( parameters.length == 2 ) + { + fb.skipNulls = PrimitiveObjectInspectorUtils.getBoolean( + parameters[1], + PrimitiveObjectInspectorFactory.writableBooleanObjectInspector); + } + } + + if ( !fb.valSet ) + { + fb.val = ObjectInspectorUtils.copyToStandardObject(parameters[0], inputOI, ObjectInspectorCopyOption.WRITABLE); + if ( !fb.skipNulls || fb.val != null ) + { + fb.valSet = true; + } + } + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException + { + throw new HiveException("terminatePartial not supported"); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException + { + throw new HiveException("merge not supported"); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + return ((FirstValueBuffer) agg).val; + } + + } +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLag.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLag.java new file mode 100644 index 0000000..47f5b31 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLag.java @@ -0,0 +1,92 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; + +@WindowFunctionDescription +( + description = @Description( + name = "lag", + value = "_FUNC_(expr, amt, default)" + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFLag extends GenericUDAFLeadLag { + + static final Log LOG = LogFactory.getLog(GenericUDAFLag.class.getName()); + + + @Override + protected String functionName() { + return "Lag"; + } + + @Override + protected GenericUDAFLeadLagEvaluator createLLEvaluator() { + return new GenericUDAFLagEvaluator(); + } + + public static class GenericUDAFLagEvaluator extends GenericUDAFLeadLagEvaluator { + + @Override + protected LeadLagBuffer getNewLLBuffer() throws HiveException { + return new LagBuffer(); + } + } + + static class LagBuffer implements LeadLagBuffer { + ArrayList values; + int lagAmt; + ArrayList lagValues; + int lastRowIdx; + + public void initialize(int lagAmt) { + this.lagAmt = lagAmt; + lagValues = new ArrayList(lagAmt); + values = new ArrayList(); + lastRowIdx = -1; + } + + public void addRow(Object currValue, Object defaultValue) { + int row = lastRowIdx + 1; + if ( row < lagAmt) { + lagValues.add(defaultValue); + } + values.add(currValue); + lastRowIdx++; + } + + public Object terminate() { + int lastIdx = values.size() - 1; + for(int i = 0; i < lagAmt; i++) { + values.remove(lastIdx - i); + } + values.addAll(0, lagValues); + return values; + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java new file mode 100644 index 0000000..d46cb0d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLastValue.java @@ -0,0 +1,164 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +@WindowFunctionDescription(description = @Description(name = "last_value", value = "_FUNC_(x)"), supportsWindow = true, pivotResult = false) +public class GenericUDAFLastValue extends AbstractGenericUDAFResolver +{ + static final Log LOG = LogFactory.getLog(GenericUDAFLastValue.class + .getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) + throws SemanticException + { + if (parameters.length > 2) + { + throw new UDFArgumentTypeException(2, "At most 2 arguments expected"); + } + if ( parameters.length > 1 && !parameters[1].equals(TypeInfoFactory.booleanTypeInfo) ) + { + throw new UDFArgumentTypeException(1, "second argument must be a boolean expression"); + } + return createEvaluator(); + } + + protected GenericUDAFLastValueEvaluator createEvaluator() + { + return new GenericUDAFLastValueEvaluator(); + } + + static class LastValueBuffer implements AggregationBuffer + { + Object val; + boolean firstRow; + boolean skipNulls; + + LastValueBuffer() + { + init(); + } + + void init() + { + val = null; + firstRow = true; + skipNulls = false; + } + + } + + public static class GenericUDAFLastValueEvaluator extends + GenericUDAFEvaluator + { + ObjectInspector inputOI; + ObjectInspector outputOI; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) + throws HiveException + { + super.init(m, parameters); + if (m != Mode.COMPLETE) + { + throw new HiveException( + "Only COMPLETE mode supported for Rank function"); + } + inputOI = parameters[0]; + outputOI = ObjectInspectorUtils.getStandardObjectInspector(inputOI, + ObjectInspectorCopyOption.WRITABLE); + return outputOI; + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException + { + return new LastValueBuffer(); + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException + { + ((LastValueBuffer) agg).init(); + } + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) + throws HiveException + { + LastValueBuffer lb = (LastValueBuffer) agg; + if (lb.firstRow ) + { + lb.firstRow = false; + if ( parameters.length == 2 ) + { + lb.skipNulls = PrimitiveObjectInspectorUtils.getBoolean( + parameters[1], + PrimitiveObjectInspectorFactory.writableBooleanObjectInspector); + } + } + + if ( !lb.skipNulls || lb.val != null ) + { + lb.val = parameters[0]; + } + } + + @Override + public Object terminatePartial(AggregationBuffer agg) + throws HiveException + { + throw new HiveException("terminatePartial not supported"); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) + throws HiveException + { + throw new HiveException("merge not supported"); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + LastValueBuffer lb = (LastValueBuffer) agg; + return ObjectInspectorUtils.copyToStandardObject(lb.val, inputOI, + ObjectInspectorCopyOption.WRITABLE); + + } + } + +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLead.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLead.java new file mode 100644 index 0000000..679e605 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLead.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; + +@WindowFunctionDescription +( + description = @Description( + name = "lead", + value = "_FUNC_(expr, amt, default)" + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFLead extends GenericUDAFLeadLag { + + static final Log LOG = LogFactory.getLog(GenericUDAFLead.class.getName()); + + + @Override + protected String functionName() { + return "Lead"; + } + + @Override + protected GenericUDAFLeadLagEvaluator createLLEvaluator() { + return new GenericUDAFLeadEvaluator(); + } + + public static class GenericUDAFLeadEvaluator extends GenericUDAFLeadLagEvaluator { + + @Override + protected LeadLagBuffer getNewLLBuffer() throws HiveException { + return new LeadBuffer(); + } + + } + + static class LeadBuffer implements LeadLagBuffer { + ArrayList values; + int leadAmt; + Object[] leadWindow; + int nextPosInWindow; + int lastRowIdx; + + public void initialize(int leadAmt) { + this.leadAmt = leadAmt; + values = new ArrayList(); + leadWindow = new Object[leadAmt]; + nextPosInWindow = 0; + lastRowIdx = -1; + } + + public void addRow(Object leadExprValue, Object defaultValue) { + int row = lastRowIdx + 1; + int leadRow = row - leadAmt; + if ( leadRow >= 0 ) { + values.add(leadExprValue); + } + leadWindow[nextPosInWindow] = defaultValue; + nextPosInWindow = (nextPosInWindow + 1) % leadAmt; + lastRowIdx++; + } + + public Object terminate() { + /* + * if there are fewer than leadAmt values in leadWindow; start reading from the first position. + * Otherwise the window starts from nextPosInWindow. + */ + if ( lastRowIdx < leadAmt ) { + nextPosInWindow = 0; + } + for(int i=0; i < leadAmt; i++) { + values.add(leadWindow[nextPosInWindow]); + nextPosInWindow = (nextPosInWindow + 1) % leadAmt; + } + return values; + } + + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLeadLag.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLeadLag.java new file mode 100644 index 0000000..8b4d7c1 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFLeadLag.java @@ -0,0 +1,180 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.io.IntWritable; + +/** + * abstract class for Lead & lag UDAFs + * GenericUDAFLeadLag. + * + */ +public abstract class GenericUDAFLeadLag extends AbstractGenericUDAFResolver { + + static final Log LOG = LogFactory.getLog(GenericUDAFLead.class.getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo parameters) + throws SemanticException { + + ObjectInspector[] paramOIs = parameters.getParameterObjectInspectors(); + String fNm = functionName(); + + if (!(paramOIs.length >= 1 && paramOIs.length <= 3) ) { + throw new UDFArgumentTypeException(paramOIs.length - 1, + "Incorrect invocation of " + fNm + ": _FUNC_(expr, amt, default)"); + } + + int amt = 1; + if ( paramOIs.length > 1 ) { + ObjectInspector amtOI = paramOIs[1]; + + if ( !ObjectInspectorUtils.isConstantObjectInspector(amtOI) || + (amtOI.getCategory() != ObjectInspector.Category.PRIMITIVE) || + ((PrimitiveObjectInspector)amtOI).getPrimitiveCategory() != + PrimitiveObjectInspector.PrimitiveCategory.INT ) + { + throw new UDFArgumentTypeException(0, + fNm + " amount must be a integer value " + + amtOI.getTypeName() + " was passed as parameter 1."); + } + Object o = ((ConstantObjectInspector)amtOI). + getWritableConstantValue(); + amt = ((IntWritable)o).get(); + } + + if (paramOIs.length == 3) { + ObjectInspectorConverters.getConverter(paramOIs[2], paramOIs[0]); + } + + GenericUDAFLeadLagEvaluator eval = createLLEvaluator(); + eval.setAmt(amt); + return eval; + } + + protected abstract String functionName(); + + protected abstract GenericUDAFLeadLagEvaluator createLLEvaluator(); + + static interface LeadLagBuffer extends AggregationBuffer { + void initialize(int leadAmt); + void addRow(Object leadExprValue, Object defaultValue) ; + Object terminate(); + + } + + public static abstract class GenericUDAFLeadLagEvaluator extends GenericUDAFEvaluator { + + ObjectInspector[] inputOI; + int amt; + String fnName; + Converter defaultValueConverter; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException + { + super.init(m, parameters); + if (m != Mode.COMPLETE) + { + throw new HiveException( + "Only COMPLETE mode supported for " + fnName + " function"); + } + + inputOI = parameters; + + if (parameters.length == 3) { + defaultValueConverter = ObjectInspectorConverters.getConverter(parameters[2], parameters[0]); + } + + return ObjectInspectorFactory.getStandardListObjectInspector( + ObjectInspectorUtils + .getStandardObjectInspector(parameters[0])); + } + + public int getAmt() { + return amt; + } + + public void setAmt(int amt) { + this.amt = amt; + } + + public String getFnName() { + return fnName; + } + + public void setFnName(String fnName) { + this.fnName = fnName; + } + + protected abstract LeadLagBuffer getNewLLBuffer() throws HiveException; + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException { + LeadLagBuffer lb = getNewLLBuffer(); + lb.initialize(amt); + return lb; + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException { + ((LeadLagBuffer)agg).initialize(amt); + } + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException { + Object rowExprVal = ObjectInspectorUtils.copyToStandardObject(parameters[0], + inputOI[0]); + Object defaultVal = parameters.length > 2 ? ObjectInspectorUtils.copyToStandardObject( + defaultValueConverter.convert(parameters[2]), + inputOI[0]) : null; + ((LeadLagBuffer)agg).addRow(rowExprVal, defaultVal); + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException { + throw new HiveException("terminatePartial not supported"); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException { + throw new HiveException("merge not supported"); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException { + return ((LeadLagBuffer)agg).terminate(); + } + + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java new file mode 100644 index 0000000..75d608f --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFNTile.java @@ -0,0 +1,181 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.IntWritable; + +@WindowFunctionDescription +( + description = @Description( + name = "rank", + value = "_FUNC_(x) NTILE allows easy calculation of tertiles, quartiles, deciles and other " + + "common summary statistics. This function divides an ordered partition into a specified " + + "number of groups called buckets and assigns a bucket number to each row in the partition." + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFNTile extends AbstractGenericUDAFResolver +{ + static final Log LOG = LogFactory.getLog(GenericUDAFNTile.class.getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException + { + if (parameters.length != 1) + { + throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected."); + } + ObjectInspector oi = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(parameters[0]); + + boolean c = ObjectInspectorUtils.compareTypes(oi, PrimitiveObjectInspectorFactory.writableIntObjectInspector); + if (!c) + { + throw new UDFArgumentTypeException(0, "Number of tiles must be an int expression"); + } + + return new GenericUDAFNTileEvaluator(); + } + + static class NTileBuffer implements AggregationBuffer + { + Integer numBuckets; + int numRows; + + void init() + { + numBuckets = null; + numRows = 0; + } + + NTileBuffer() + { + init(); + } + } + + public static class GenericUDAFNTileEvaluator extends GenericUDAFEvaluator + { + PrimitiveObjectInspector inputOI; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException + { + assert (parameters.length == 1); + super.init(m, parameters); + if (m != Mode.COMPLETE) + { + throw new HiveException( + "Only COMPLETE mode supported for NTile function"); + } + inputOI = (PrimitiveObjectInspector) parameters[0]; + return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableIntObjectInspector); + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException + { + return new NTileBuffer(); + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException + { + ((NTileBuffer) agg).init(); + } + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException + { + NTileBuffer rb = (NTileBuffer) agg; + if ( rb.numBuckets == null) + { + rb.numBuckets = PrimitiveObjectInspectorUtils.getInt(parameters[0], inputOI); + } + rb.numRows++; + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException + { + throw new HiveException("terminatePartial not supported"); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException + { + throw new HiveException("merge not supported"); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + NTileBuffer rb = (NTileBuffer) agg; + ArrayList res = new ArrayList(rb.numRows); + + /* + * if there is a remainder from numRows/numBuckets; then distribute increase the size of the first 'rem' buckets by 1. + */ + + int bucketsz = rb.numRows / rb.numBuckets; + int rem = rb.numRows % rb.numBuckets; + int start = 0; + int bucket = 1; + while ( start < rb.numRows) + { + int end = start + bucketsz; + if (rem > 0) + { + end++; rem--; + } + end = Math.min(rb.numRows, end); + for(int i = start; i < end; i++) + { + res.add(new IntWritable(bucket)); + } + start = end; + bucket++; + } + + return res; + } + + } + + +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java new file mode 100644 index 0000000..7282872 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentRank.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.IntWritable; + +@WindowFunctionDescription +( + description = @Description( + name = "percent_rank", + value = "_FUNC_(x) PERCENT_RANK is similar to CUME_DIST, but it uses rank values rather " + + "than row counts in its numerator. PERCENT_RANK of a row is calculated as: " + + "(rank of row in its partition - 1) / (number of rows in the partition - 1)" + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFPercentRank extends GenericUDAFRank +{ + static final Log LOG = LogFactory.getLog(GenericUDAFPercentRank.class.getName()); + + @Override + protected GenericUDAFRankEvaluator createEvaluator() + { + return new GenericUDAFPercentRankEvaluator(); + } + + public static class GenericUDAFPercentRankEvaluator extends GenericUDAFRankEvaluator + { + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException + { + super.init(m, parameters); + return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + ArrayList ranks = ((RankBuffer) agg).rowNums; + double sz = ranks.size(); + if ( sz > 1 ) { + sz = sz - 1; + } + ArrayList pranks = new ArrayList(ranks.size()); + + for(IntWritable i : ranks) + { + double pr = ((double)i.get() - 1)/sz; + pranks.add(new DoubleWritable(pr)); + } + + return pranks; + } + } +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java new file mode 100644 index 0000000..e59e076 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRank.java @@ -0,0 +1,212 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.IntWritable; + +@WindowFunctionDescription +( + description = @Description( + name = "rank", + value = "_FUNC_(x)" + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFRank extends AbstractGenericUDAFResolver +{ + static final Log LOG = LogFactory.getLog(GenericUDAFRank.class.getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException + { + if (parameters.length < 1) + { + throw new UDFArgumentTypeException(parameters.length - 1, "One or more arguments are expected."); + } + for(int i=0; i type or complex type containing map<>."); + } + } + return createEvaluator(); + } + + protected GenericUDAFRankEvaluator createEvaluator() + { + return new GenericUDAFRankEvaluator(); + } + + static class RankBuffer implements AggregationBuffer + { + ArrayList rowNums; + int currentRowNum; + Object[] currVal; + int currentRank; + int numParams; + + RankBuffer(int numParams) + { + this.numParams = numParams; + init(); + } + + void init() + { + rowNums = new ArrayList(); + currentRowNum = 0; + currentRank = 0; + currVal = new Object[numParams]; + } + + void incrRowNum() { currentRowNum++; } + + void addRank() + { + rowNums.add(new IntWritable(currentRank)); + } + } + + public static class GenericUDAFRankEvaluator extends GenericUDAFEvaluator + { + ObjectInspector[] inputOI; + ObjectInspector[] outputOI; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException + { + super.init(m, parameters); + if (m != Mode.COMPLETE) + { + throw new HiveException( + "Only COMPLETE mode supported for Rank function"); + } + inputOI = parameters; + outputOI = new ObjectInspector[inputOI.length]; + for(int i=0; i < inputOI.length; i++) + { + outputOI[i] = ObjectInspectorUtils.getStandardObjectInspector(inputOI[i], ObjectInspectorCopyOption.JAVA); + } + return ObjectInspectorFactory.getStandardListObjectInspector(PrimitiveObjectInspectorFactory.writableIntObjectInspector); + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException + { + return new RankBuffer(inputOI.length); + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException + { + ((RankBuffer) agg).init(); + } + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException + { + RankBuffer rb = (RankBuffer) agg; + int c = GenericUDAFRank.compare(rb.currVal, outputOI, parameters, inputOI); + rb.incrRowNum(); + if ( rb.currentRowNum == 1 || c != 0 ) + { + nextRank(rb); + rb.currVal = GenericUDAFRank.copyToStandardObject(parameters, inputOI, ObjectInspectorCopyOption.JAVA); + } + rb.addRank(); + } + + /* + * Called when the value in the partition has changed. Update the currentRank + */ + protected void nextRank(RankBuffer rb) + { + rb.currentRank = rb.currentRowNum; + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException + { + throw new HiveException("terminatePartial not supported"); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException + { + throw new HiveException("merge not supported"); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + return ((RankBuffer) agg).rowNums; + } + + } + + public static int compare(Object[] o1, ObjectInspector[] oi1, Object[] o2, + ObjectInspector[] oi2) + { + int c = 0; + for (int i = 0; i < oi1.length; i++) + { + c = ObjectInspectorUtils.compare(o1[i], oi1[i], o2[i], oi2[i]); + if (c != 0) { + return c; + } + } + return c; + } + + public static Object[] copyToStandardObject(Object[] o, + ObjectInspector[] oi, + ObjectInspectorCopyOption objectInspectorOption) + { + Object[] out = new Object[o.length]; + for (int i = 0; i < oi.length; i++) + { + out[i] = ObjectInspectorUtils.copyToStandardObject(o[i], oi[i], + objectInspectorOption); + } + return out; + } + +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java new file mode 100644 index 0000000..987da3d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFRowNumber.java @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.io.IntWritable; + +@WindowFunctionDescription +( + description = @Description( + name = "row_number", + value = "_FUNC_() - The ROW_NUMBER function assigns a unique number (sequentially, starting from 1, as defined by ORDER BY) to each row within the partition." + ), + supportsWindow = false, + pivotResult = true +) +public class GenericUDAFRowNumber extends AbstractGenericUDAFResolver +{ + static final Log LOG = LogFactory.getLog(GenericUDAFRowNumber.class.getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) + throws SemanticException + { + if (parameters.length != 0) + { + throw new UDFArgumentTypeException(parameters.length - 1, + "No argument is expected."); + } + return new GenericUDAFRowNumberEvaluator(); + } + + static class RowNumberBuffer implements AggregationBuffer + { + ArrayList rowNums; + int nextRow; + + void init() + { + rowNums = new ArrayList(); + } + + RowNumberBuffer() + { + init(); + nextRow = 1; + } + + void incr() + { + rowNums.add(new IntWritable(nextRow++)); + } + } + + public static class GenericUDAFRowNumberEvaluator extends + GenericUDAFEvaluator + { + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) + throws HiveException + { + super.init(m, parameters); + if (m != Mode.COMPLETE) + { + throw new HiveException("Only COMPLETE mode supported for row_number function"); + } + + return ObjectInspectorFactory.getStandardListObjectInspector( + PrimitiveObjectInspectorFactory.writableIntObjectInspector); + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException + { + return new RowNumberBuffer(); + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException + { + ((RowNumberBuffer) agg).init(); + } + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) + throws HiveException + { + ((RowNumberBuffer) agg).incr(); + } + + @Override + public Object terminatePartial(AggregationBuffer agg) + throws HiveException + { + throw new HiveException("terminatePartial not supported"); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) + throws HiveException + { + throw new HiveException("merge not supported"); + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException + { + return ((RowNumberBuffer) agg).rowNums; + } + + } +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLeadLag.java ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLeadLag.java new file mode 100644 index 0000000..5c09c15 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFLeadLag.java @@ -0,0 +1,177 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.exec.PTFUtils; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; + +public abstract class GenericUDFLeadLag extends GenericUDF +{ + transient ExprNodeEvaluator exprEvaluator; + transient PTFPartitionIterator pItr; + ObjectInspector firstArgOI; + + private PrimitiveObjectInspector amtOI; + + static{ + PTFUtils.makeTransient(GenericUDFLeadLag.class, "exprEvaluator"); + PTFUtils.makeTransient(GenericUDFLeadLag.class, "pItr"); + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException + { + DeferredObject amt = arguments[1]; + int intAmt = 0; + try + { + intAmt = PrimitiveObjectInspectorUtils.getInt(amt.get(), amtOI); + } + catch (NullPointerException e) + { + intAmt = Integer.MAX_VALUE; + } + catch (NumberFormatException e) + { + intAmt = Integer.MAX_VALUE; + } + + int idx = pItr.getIndex() - 1; + try + { + Object row = getRow(intAmt); + Object ret = exprEvaluator.evaluate(row); + ret = ObjectInspectorUtils.copyToStandardObject(ret, firstArgOI, ObjectInspectorCopyOption.WRITABLE); + return ret; + } + finally + { + Object currRow = pItr.resetToIndex(idx); + // reevaluate expression on current Row, to trigger the Lazy object + // caches to be reset to the current row. + exprEvaluator.evaluate(currRow); + } + + } + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) + throws UDFArgumentException + { + // index has to be a primitive + if (arguments[1] instanceof PrimitiveObjectInspector) + { + amtOI = (PrimitiveObjectInspector) arguments[1]; + } + else + { + throw new UDFArgumentTypeException(1, + "Primitive Type is expected but " + + arguments[1].getTypeName() + "\" is found"); + } + + firstArgOI = arguments[0]; + return ObjectInspectorUtils.getStandardObjectInspector(firstArgOI, + ObjectInspectorCopyOption.WRITABLE); + } + + + + public ExprNodeEvaluator getExprEvaluator() + { + return exprEvaluator; + } + + public void setExprEvaluator(ExprNodeEvaluator exprEvaluator) + { + this.exprEvaluator = exprEvaluator; + } + + public PTFPartitionIterator getpItr() + { + return pItr; + } + + public void setpItr(PTFPartitionIterator pItr) + { + this.pItr = pItr; + } + + @Override + public String getDisplayString(String[] children) + { + assert (children.length == 2); + StringBuilder sb = new StringBuilder(); + sb.append(_getFnName()); + sb.append("("); + sb.append(children[0]); + sb.append(", "); + sb.append(children[1]); + sb.append(")"); + return sb.toString(); + } + + protected abstract String _getFnName(); + + protected abstract Object getRow(int amt); + + public static class GenericUDFLead extends GenericUDFLeadLag + { + + @Override + protected String _getFnName() + { + return "lead"; + } + + @Override + protected Object getRow(int amt) + { + return pItr.lead(amt - 1); + } + + } + + public static class GenericUDFLag extends GenericUDFLeadLag + { + @Override + protected String _getFnName() + { + return "lag"; + } + + @Override + protected Object getRow(int amt) + { + return pItr.lag(amt + 1); + } + + } + +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NPath.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NPath.java new file mode 100644 index 0000000..eb68339 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NPath.java @@ -0,0 +1,936 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.ptf; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.exec.PTFUtils; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.PTFTranslator; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.TypeCheckCtx; +import org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowExpressionSpec; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFInputDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StandardListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; + +/** + * return rows that meet a specified pattern. Use symbols to specify a list of expressions + * to match. + * Pattern is used to specify a Path. The results list can contain expressions based on + * the input columns and also the matched Path. + *
    + *
  1. pattern: pattern for the Path. Path is 'dot' separated list of symbols. + * Each element is treated as a symbol. Elements that end in '*' or '+' are interpreted with + * the usual meaning of zero or more, one or more respectively. For e.g. + * "LATE.EARLY*.ONTIMEOREARLY" implies a sequence of flights + * where the first occurrence was LATE, followed by zero or more EARLY flights, + * followed by a ONTIME or EARLY flight. + *
  2. symbols specify a list of name, expression pairs. For e.g. + * 'LATE', arrival_delay > 0, 'EARLY', arrival_delay < 0 , 'ONTIME', arrival_delay == 0. + * These symbols can be used in the Pattern defined above. + *
  3. resultSelectList specified as a select list. + * The expressions in the selectList are evaluated in the context where all the + * input columns are available, plus the attribute + * "tpath" is available. Path is a collection of rows that represents the matching Path. + *
+ */ +public class NPath extends TableFunctionEvaluator +{ + private transient String patternStr; + private transient SymbolsInfo symInfo; + private transient String resultExprStr; + private transient SymbolFunction syFn; + private ResultExprInfo resultExprInfo; + /* + * the names of the Columns of the input to NPath. Used to setup the tpath Struct column. + */ + private ArrayList inputColumnNames; + private ArrayList selectListNames; + + @Override + public void execute(PTFPartitionIterator pItr, PTFPartition outP) throws HiveException + { + while (pItr.hasNext()) + { + Object iRow = pItr.next(); + + SymbolFunctionResult syFnRes = SymbolFunction.match(syFn, iRow, pItr); + if (syFnRes.matches ) + { + int sz = syFnRes.nextRow - (pItr.getIndex() - 1); + Object selectListInput = NPath.getSelectListInput(iRow, + tDef.getInput().getOutputShape().getOI(), pItr, sz); + ArrayList oRow = new ArrayList(); + for(ExprNodeEvaluator resExprEval : resultExprInfo.resultExprEvals) + { + oRow.add(resExprEval.evaluate(selectListInput)); + } + outP.append(oRow); + } + } + } + + static void throwErrorWithSignature(String message) throws SemanticException + { + throw new SemanticException(String.format( + "NPath signature is: SymbolPattern, one or more SymbolName, " + + "expression pairs, the result expression as a select list. Error %s", + message)); + } + + public ArrayList getInputColumnNames() { + return inputColumnNames; + } + + public void setInputColumnNames(ArrayList inputColumnNames) { + this.inputColumnNames = inputColumnNames; + } + + public ArrayList getSelectListNames() { + return selectListNames; + } + + public void setSelectListNames(ArrayList selectListNames) { + this.selectListNames = selectListNames; + } + + public static class NPathResolver extends TableFunctionResolver + { + + @Override + protected TableFunctionEvaluator createEvaluator(PTFDesc ptfDesc, + PartitionedTableFunctionDef tDef) + { + + return new NPath(); + } + + /** + *
    + *
  • check structure of Arguments: + *
      + *
    1. First arg should be a String + *
    2. then there should be an even number of Arguments: + * String, expression; expression should be Convertible to Boolean. + *
    3. finally there should be a String. + *
    + *
  • convert pattern into a NNode chain. + *
  • convert symbol args into a Symbol Map. + *
  • parse selectList into SelectList struct. The inputOI used to translate + * these expressions should be based on the + * columns in the Input, the 'path.attr' + *
+ */ + @Override + public void setupOutputOI() throws SemanticException + { + NPath evaluator = (NPath) getEvaluator(); + PartitionedTableFunctionDef tDef = evaluator.getTableDef(); + + ArrayList args = tDef.getArgs(); + int argsNum = args == null ? 0 : args.size(); + + if ( argsNum < 4 ) + { + throwErrorWithSignature("at least 4 arguments required"); + } + + validateAndSetupPatternStr(evaluator, args); + validateAndSetupSymbolInfo(evaluator, args, argsNum); + validateAndSetupResultExprStr(evaluator, args, argsNum); + setupSymbolFunctionChain(evaluator); + + /* + * setup OI for input to resultExpr select list + */ + RowResolver selectListInputRR = NPath.createSelectListRR(evaluator, tDef.getInput()); + + /* + * parse ResultExpr Str and setup OI. + */ + ResultExpressionParser resultExprParser = + new ResultExpressionParser(evaluator.resultExprStr, selectListInputRR); + try { + resultExprParser.translate(); + } + catch(HiveException he) { + throw new SemanticException(he); + } + evaluator.resultExprInfo = resultExprParser.getResultExprInfo(); + StructObjectInspector OI = evaluator.resultExprInfo.resultOI; + evaluator.selectListNames = new ArrayList(); + extractOIColumnNames(resultExprParser.selectListInputOI, evaluator.selectListNames); + + setOutputOI(OI); + } + /* + * validate and setup patternStr + */ + private void validateAndSetupPatternStr(NPath evaluator, + ArrayList args) throws SemanticException { + PTFExpressionDef symboPatternArg = args.get(0); + ObjectInspector symbolPatternArgOI = symboPatternArg.getOI(); + + if ( !ObjectInspectorUtils.isConstantObjectInspector(symbolPatternArgOI) || + (symbolPatternArgOI.getCategory() != ObjectInspector.Category.PRIMITIVE) || + ((PrimitiveObjectInspector)symbolPatternArgOI).getPrimitiveCategory() != + PrimitiveObjectInspector.PrimitiveCategory.STRING ) + { + throwErrorWithSignature("Currently the symbol Pattern must be a Constant String."); + } + + evaluator.patternStr = ((ConstantObjectInspector)symbolPatternArgOI). + getWritableConstantValue().toString(); + } + + /* + * validate and setup SymbolInfo + */ + private void validateAndSetupSymbolInfo(NPath evaluator, + ArrayList args, + int argsNum) throws SemanticException { + int symbolArgsSz = argsNum - 2; + if ( symbolArgsSz % 2 != 0) + { + throwErrorWithSignature("Symbol Name, Expression need to be specified in pairs: " + + "there are odd number of symbol args"); + } + + evaluator.symInfo = new SymbolsInfo(symbolArgsSz/2); + for(int i=1; i <= symbolArgsSz; i += 2) + { + PTFExpressionDef symbolNameArg = args.get(i); + ObjectInspector symbolNameArgOI = symbolNameArg.getOI(); + + if ( !ObjectInspectorUtils.isConstantObjectInspector(symbolNameArgOI) || + (symbolNameArgOI.getCategory() != ObjectInspector.Category.PRIMITIVE) || + ((PrimitiveObjectInspector)symbolNameArgOI).getPrimitiveCategory() != + PrimitiveObjectInspector.PrimitiveCategory.STRING ) + { + throwErrorWithSignature( + String.format("Currently a Symbol Name(%s) must be a Constant String", + symbolNameArg.getExpressionTreeString())); + } + String symbolName = ((ConstantObjectInspector)symbolNameArgOI). + getWritableConstantValue().toString(); + + PTFExpressionDef symolExprArg = args.get(i+1); + ObjectInspector symolExprArgOI = symolExprArg.getOI(); + if ( (symolExprArgOI.getCategory() != ObjectInspector.Category.PRIMITIVE) || + ((PrimitiveObjectInspector)symolExprArgOI).getPrimitiveCategory() != + PrimitiveObjectInspector.PrimitiveCategory.BOOLEAN ) + { + throwErrorWithSignature(String.format("Currently a Symbol Expression(%s) " + + "must be a boolean expression", symolExprArg.getExpressionTreeString())); + } + evaluator.symInfo.add(symbolName, symolExprArg); + } + } + + /* + * validate and setup resultExprStr + */ + private void validateAndSetupResultExprStr(NPath evaluator, + ArrayList args, + int argsNum) throws SemanticException { + PTFExpressionDef resultExprArg = args.get(argsNum - 1); + ObjectInspector resultExprArgOI = resultExprArg.getOI(); + + if ( !ObjectInspectorUtils.isConstantObjectInspector(resultExprArgOI) || + (resultExprArgOI.getCategory() != ObjectInspector.Category.PRIMITIVE) || + ((PrimitiveObjectInspector)resultExprArgOI).getPrimitiveCategory() != + PrimitiveObjectInspector.PrimitiveCategory.STRING ) + { + throwErrorWithSignature("Currently the result Expr parameter must be a Constant String."); + } + + evaluator.resultExprStr = ((ConstantObjectInspector)resultExprArgOI). + getWritableConstantValue().toString(); + } + + /* + * setup SymbolFunction chain. + */ + private void setupSymbolFunctionChain(NPath evaluator) throws SemanticException { + SymbolParser syP = new SymbolParser(evaluator.patternStr, + evaluator.symInfo.symbolExprsNames, + evaluator.symInfo.symbolExprsEvaluators, evaluator.symInfo.symbolExprsOIs); + syP.parse(); + evaluator.syFn = syP.getSymbolFunction(); + } + + @Override + public boolean transformsRawInput() + { + return false; + } + + @Override + public void initializeOutputOI() throws HiveException { + try { + NPath evaluator = (NPath) getEvaluator(); + PartitionedTableFunctionDef tDef = evaluator.getTableDef(); + + ArrayList args = tDef.getArgs(); + int argsNum = args.size(); + + validateAndSetupPatternStr(evaluator, args); + validateAndSetupSymbolInfo(evaluator, args, argsNum); + validateAndSetupResultExprStr(evaluator, args, argsNum); + setupSymbolFunctionChain(evaluator); + + /* + * setup OI for input to resultExpr select list + */ + StructObjectInspector selectListInputOI = NPath.createSelectListOI( evaluator, + tDef.getInput()); + ResultExprInfo resultExprInfo = evaluator.resultExprInfo; + ArrayList selectListExprOIs = new ArrayList(); + resultExprInfo.resultExprEvals = new ArrayList(); + + for(int i=0 ; i < resultExprInfo.resultExprNodes.size(); i++) { + ExprNodeDesc selectColumnExprNode =resultExprInfo.resultExprNodes.get(i); + ExprNodeEvaluator selectColumnExprEval = + ExprNodeEvaluatorFactory.get(selectColumnExprNode); + ObjectInspector selectColumnOI = selectColumnExprEval.initialize(selectListInputOI); + resultExprInfo.resultExprEvals.add(selectColumnExprEval); + selectListExprOIs.add(selectColumnOI); + } + + resultExprInfo.resultOI = ObjectInspectorFactory.getStandardStructObjectInspector( + resultExprInfo.resultExprNames, selectListExprOIs); + setOutputOI(resultExprInfo.resultOI); + } + catch(SemanticException se) { + throw new HiveException(se); + } + } + + @Override + public ArrayList getOutputColumnNames() { + NPath evaluator = (NPath) getEvaluator(); + return evaluator.resultExprInfo.getResultExprNames(); + } + + + + private static void extractOIColumnNames(StructObjectInspector OI, + ArrayList oiColumnNames) { + StructTypeInfo t = (StructTypeInfo) TypeInfoUtils.getTypeInfoFromObjectInspector(OI); + ArrayList fnames = t.getAllStructFieldNames(); + oiColumnNames.addAll(fnames); + } + + } + + public ResultExprInfo getResultExprInfo() { + return resultExprInfo; + } + + public void setResultExprInfo(ResultExprInfo resultExprInfo) { + this.resultExprInfo = resultExprInfo; + } + + static class SymbolsInfo { + int sz; + ArrayList symbolExprsEvaluators; + ArrayList symbolExprsOIs; + ArrayList symbolExprsNames; + + SymbolsInfo(int sz) + { + this.sz = sz; + symbolExprsEvaluators = new ArrayList(sz); + symbolExprsOIs = new ArrayList(sz); + symbolExprsNames = new ArrayList(sz); + } + + void add(String name, PTFExpressionDef arg) + { + symbolExprsNames.add(name); + symbolExprsEvaluators.add(arg.getExprEvaluator()); + symbolExprsOIs.add(arg.getOI()); + } + } + + public static class ResultExprInfo { + ArrayList resultExprNames; + ArrayList resultExprNodes; + private transient ArrayList resultExprEvals; + private transient StructObjectInspector resultOI; + + public ArrayList getResultExprNames() { + return resultExprNames; + } + public void setResultExprNames(ArrayList resultExprNames) { + this.resultExprNames = resultExprNames; + } + public ArrayList getResultExprNodes() { + return resultExprNodes; + } + public void setResultExprNodes(ArrayList resultExprNodes) { + this.resultExprNodes = resultExprNodes; + } + } + + public static abstract class SymbolFunction + { + SymbolFunctionResult result; + + public SymbolFunction() + { + result = new SymbolFunctionResult(); + } + + public static SymbolFunctionResult match(SymbolFunction syFn, Object row, + PTFPartitionIterator pItr) throws HiveException + { + int resetToIdx = pItr.getIndex() - 1; + try + { + return syFn.match(row, pItr); + } finally + { + pItr.resetToIndex(resetToIdx); + } + } + + protected abstract SymbolFunctionResult match(Object row, PTFPartitionIterator pItr) + throws HiveException; + + protected abstract boolean isOptional(); + } + + public static class Symbol extends SymbolFunction { + ExprNodeEvaluator symbolExprEval; + Converter converter; + + public Symbol(ExprNodeEvaluator symbolExprEval, ObjectInspector symbolOI) + { + this.symbolExprEval = symbolExprEval; + converter = ObjectInspectorConverters.getConverter( + symbolOI, + PrimitiveObjectInspectorFactory.javaBooleanObjectInspector); + } + + @Override + protected SymbolFunctionResult match(Object row, PTFPartitionIterator pItr) + throws HiveException + { + Object val = null; + val = symbolExprEval.evaluate(row); + val = converter.convert(val); + result.matches = ((Boolean) val).booleanValue(); + result.nextRow = pItr.getIndex(); + + return result; + } + + @Override + protected boolean isOptional() + { + return false; + } + } + + public static class Star extends SymbolFunction { + SymbolFunction symbolFn; + + public Star(SymbolFunction symbolFn) + { + this.symbolFn = symbolFn; + } + + @Override + protected SymbolFunctionResult match(Object row, PTFPartitionIterator pItr) + throws HiveException + { + result.matches = true; + SymbolFunctionResult rowResult = symbolFn.match(row, pItr); + + while (rowResult.matches && pItr.hasNext()) + { + row = pItr.next(); + rowResult = symbolFn.match(row, pItr); + } + + result.nextRow = pItr.getIndex() - 1; + return result; + } + + @Override + protected boolean isOptional() + { + return true; + } + } + + public static class Plus extends SymbolFunction { + SymbolFunction symbolFn; + + public Plus(SymbolFunction symbolFn) + { + this.symbolFn = symbolFn; + } + + @Override + protected SymbolFunctionResult match(Object row, PTFPartitionIterator pItr) + throws HiveException + { + SymbolFunctionResult rowResult = symbolFn.match(row, pItr); + + if (!rowResult.matches) + { + result.matches = false; + result.nextRow = pItr.getIndex() - 1; + return result; + } + + result.matches = true; + while (rowResult.matches && pItr.hasNext()) + { + row = pItr.next(); + rowResult = symbolFn.match(row, pItr); + } + + result.nextRow = pItr.getIndex() - 1; + return result; + } + + @Override + protected boolean isOptional() + { + return false; + } + } + + public static class Chain extends SymbolFunction + { + ArrayList components; + + public Chain(ArrayList components) + { + this.components = components; + } + + /* + * Iterate over the Symbol Functions in the Chain: + * - If we are not at the end of the Iterator (i.e. row != null ) + * - match the current componentFn + * - if it returns false, then return false + * - otherwise set row to the next row from the Iterator. + * - if we are at the end of the Iterator + * - skip any optional Symbol Fns (star patterns) at the end. + * - but if we come to a non optional Symbol Fn, return false. + * - if we match all Fns in the chain return true. + */ + @Override + protected SymbolFunctionResult match(Object row, PTFPartitionIterator pItr) + throws HiveException + { + SymbolFunctionResult componentResult = null; + for (SymbolFunction sFn : components) + { + if (row != null) + { + componentResult = sFn.match(row, pItr); + if (!componentResult.matches) + { + result.matches = false; + result.nextRow = componentResult.nextRow; + return result; + } + row = pItr.resetToIndex(componentResult.nextRow); + } + else + { + if (!sFn.isOptional()) + { + result.matches = false; + result.nextRow = componentResult.nextRow; + return result; + } + } + } + + result.matches = true; + result.nextRow = componentResult.nextRow; + return result; + } + + @Override + protected boolean isOptional() + { + return false; + } + } + + + public static class SymbolFunctionResult + { + /* + * does the row match the pattern represented by this SymbolFunction + */ + public boolean matches; + /* + * what is the index of the row beyond the set of rows that match this pattern. + */ + public int nextRow; + } + + public static class SymbolParser + { + String patternStr; + String[] symbols; + HashMap symbolExprEvalMap; + ArrayList symbolFunctions; + Chain symbolFnChain; + + + public SymbolParser(String patternStr, ArrayList symbolNames, + ArrayList symbolExprEvals, ArrayList symbolExprOIs) + { + super(); + this.patternStr = patternStr; + symbolExprEvalMap = new HashMap(); + int sz = symbolNames.size(); + for(int i=0; i < sz; i++) + { + String symbolName = symbolNames.get(i); + ExprNodeEvaluator symbolExprEval = symbolExprEvals.get(i); + ObjectInspector symbolExprOI = symbolExprOIs.get(i); + symbolExprEvalMap.put(symbolName.toLowerCase(), + new Object[] {symbolExprEval, symbolExprOI}); + } + } + + public SymbolFunction getSymbolFunction() + { + return symbolFnChain; + } + + public void parse() throws SemanticException + { + symbols = patternStr.split("\\."); + symbolFunctions = new ArrayList(); + + for(String symbol : symbols) + { + boolean isStar = symbol.endsWith("*"); + boolean isPlus = symbol.endsWith("+"); + + symbol = (isStar || isPlus) ? symbol.substring(0, symbol.length() - 1) : symbol; + Object[] symbolDetails = symbolExprEvalMap.get(symbol.toLowerCase()); + if ( symbolDetails == null ) + { + throw new SemanticException(String.format("Unknown Symbol %s", symbol)); + } + + ExprNodeEvaluator symbolExprEval = (ExprNodeEvaluator) symbolDetails[0]; + ObjectInspector symbolExprOI = (ObjectInspector) symbolDetails[1]; + SymbolFunction sFn = new Symbol(symbolExprEval, symbolExprOI); + + if ( isStar ) + { + sFn = new Star(sFn); + } + else if ( isPlus ) + { + sFn = new Plus(sFn); + } + symbolFunctions.add(sFn); + } + symbolFnChain = new Chain(symbolFunctions); + } + } + + /* + * ResultExpression is a Select List with the following variation: + * - the select keyword is optional. The parser checks if the expression doesn't start with + * select; if not it prefixes it. + * - Window Fn clauses are not permitted. + * - expressions can operate on the input columns plus the psuedo column 'path' + * which is array of + * structs. The shape of the struct is + * the same as the input. + */ + public static class ResultExpressionParser { + String resultExprString; + + RowResolver selectListInputRowResolver; + TypeCheckCtx selectListInputTypeCheckCtx; + StructObjectInspector selectListInputOI; + + ArrayList selectSpec; + + ResultExprInfo resultExprInfo; + + public ResultExpressionParser(String resultExprString, + RowResolver selectListInputRowResolver) + { + this.resultExprString = resultExprString; + this.selectListInputRowResolver = selectListInputRowResolver; + } + + public void translate() throws SemanticException, HiveException + { + setupSelectListInputInfo(); + fixResultExprString(); + parse(); + validateSelectExpr(); + buildSelectListEvaluators(); + } + + public ResultExprInfo getResultExprInfo() { + return resultExprInfo; + } + + private void buildSelectListEvaluators() throws SemanticException, HiveException + { + resultExprInfo = new ResultExprInfo(); + resultExprInfo.resultExprEvals = new ArrayList(); + resultExprInfo.resultExprNames = new ArrayList(); + resultExprInfo.resultExprNodes = new ArrayList(); + //result + ArrayList selectListExprOIs = new ArrayList(); + int i = 0; + for(WindowExpressionSpec expr : selectSpec) + { + String selectColName = expr.getAlias(); + ASTNode selectColumnNode = expr.getExpression(); + ExprNodeDesc selectColumnExprNode = + ResultExpressionParser.buildExprNode(selectColumnNode, + selectListInputTypeCheckCtx); + ExprNodeEvaluator selectColumnExprEval = + ExprNodeEvaluatorFactory.get(selectColumnExprNode); + ObjectInspector selectColumnOI = null; + selectColumnOI = selectColumnExprEval.initialize(selectListInputOI); + + selectColName = getColumnName(selectColName, selectColumnExprNode, i); + + resultExprInfo.resultExprEvals.add(selectColumnExprEval); + selectListExprOIs.add(selectColumnOI); + resultExprInfo.resultExprNodes.add(selectColumnExprNode); + resultExprInfo.resultExprNames.add(selectColName); + i++; + } + + resultExprInfo.resultOI = ObjectInspectorFactory.getStandardStructObjectInspector( + resultExprInfo.resultExprNames, selectListExprOIs); + } + + private void setupSelectListInputInfo() throws SemanticException + { + selectListInputTypeCheckCtx = new TypeCheckCtx(selectListInputRowResolver); + selectListInputTypeCheckCtx.setUnparseTranslator(null); + /* + * create SelectListOI + */ + selectListInputOI = (StructObjectInspector) + PTFTranslator.getStandardStructOI(selectListInputRowResolver); + } + + private void fixResultExprString() + { + String r = resultExprString.trim(); + String prefix = r.substring(0, 6); + if (!prefix.toLowerCase().equals("select")) + { + r = "select " + r; + } + resultExprString = r; + } + + private void parse() throws SemanticException + { + selectSpec = SemanticAnalyzer.parseSelect(resultExprString); + } + + private void validateSelectExpr() throws SemanticException + { + for (WindowExpressionSpec expr : selectSpec) + { + PTFTranslator.validateNoLeadLagInValueBoundarySpec(expr.getExpression()); + } + } + + private String getColumnName(String alias, ExprNodeDesc exprNode, int colIdx) + { + if (alias != null) + { + return alias; + } + else if (exprNode instanceof ExprNodeColumnDesc) + { + ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc) exprNode; + return colDesc.getColumn(); + } + return "npath_col_" + colIdx; + } + + public static ExprNodeDesc buildExprNode(ASTNode expr, + TypeCheckCtx typeCheckCtx) throws SemanticException + { + // todo: use SemanticAnalyzer::genExprNodeDesc + // currently SA not available to PTFTranslator. + Map map = TypeCheckProcFactory + .genExprNode(expr, typeCheckCtx); + ExprNodeDesc desc = map.get(expr); + if (desc == null) { + String errMsg = typeCheckCtx.getError(); + if ( errMsg == null) { + errMsg = "Error in parsing "; + } + throw new SemanticException(errMsg); + } + return desc; + } + } + + public static final String PATHATTR_NAME = "tpath"; + + /* + * add array to the list of columns + */ + protected static RowResolver createSelectListRR(NPath evaluator, + PTFInputDef inpDef) throws SemanticException { + RowResolver rr = new RowResolver(); + RowResolver inputRR = inpDef.getOutputShape().getRr(); + boolean inputColNamesKnown = evaluator.inputColumnNames != null; + + if ( !inputColNamesKnown ) { + evaluator.inputColumnNames = new ArrayList(); + } + + ArrayList inpColOIs = new ArrayList(); + + for (ColumnInfo inpCInfo : inputRR.getColumnInfos()) { + ColumnInfo cInfo = new ColumnInfo(inpCInfo); + String colAlias = cInfo.getAlias(); + + String[] tabColAlias = inputRR.reverseLookup(inpCInfo.getInternalName()); + if (tabColAlias != null) { + colAlias = tabColAlias[1]; + } + ASTNode inExpr = null; + inExpr = PTFTranslator.getASTNode(inpCInfo, inputRR); + if ( inExpr != null ) { + rr.putExpression(inExpr, cInfo); + } + else { + colAlias = colAlias == null ? cInfo.getInternalName() : colAlias; + rr.put(cInfo.getTabAlias(), colAlias, cInfo); + } + + if ( !inputColNamesKnown ) { + evaluator.inputColumnNames.add(colAlias); + } + inpColOIs.add(cInfo.getObjectInspector()); + } + + StandardListObjectInspector pathAttrOI = + ObjectInspectorFactory.getStandardListObjectInspector( + ObjectInspectorFactory.getStandardStructObjectInspector(evaluator.inputColumnNames, + inpColOIs)); + + ColumnInfo pathColumn = new ColumnInfo(PATHATTR_NAME, + TypeInfoUtils.getTypeInfoFromObjectInspector(pathAttrOI), + null, + false, false); + rr.put(null, PATHATTR_NAME, pathColumn); + + return rr; + } + + protected static StructObjectInspector createSelectListOI(NPath evaluator, PTFInputDef inpDef) { + StructObjectInspector inOI = inpDef.getOutputShape().getOI(); + ArrayList fieldOIs = new ArrayList(); + for(StructField f : inOI.getAllStructFieldRefs()) { + fieldOIs.add(f.getFieldObjectInspector()); + } + + StandardListObjectInspector pathAttrOI = + ObjectInspectorFactory.getStandardListObjectInspector( + ObjectInspectorFactory.getStandardStructObjectInspector(evaluator.inputColumnNames, + fieldOIs)); + + ArrayList selectFieldOIs = new ArrayList(); + selectFieldOIs.addAll(fieldOIs); + selectFieldOIs.add(pathAttrOI); + return ObjectInspectorFactory.getStandardStructObjectInspector( + evaluator.selectListNames, selectFieldOIs); + } + + public static Object getSelectListInput(Object currRow, ObjectInspector rowOI, + PTFPartitionIterator pItr, int sz) { + ArrayList oRow = new ArrayList(); + List currRowAsStdObject = (List) ObjectInspectorUtils + .copyToStandardObject(currRow, rowOI); + oRow.addAll(currRowAsStdObject); + oRow.add(getPath(currRow, rowOI, pItr, sz)); + return oRow; + } + + public static ArrayList getPath(Object currRow, ObjectInspector rowOI, + PTFPartitionIterator pItr, int sz) { + int idx = pItr.getIndex() - 1; + ArrayList path = new ArrayList(); + path.add(ObjectInspectorUtils.copyToStandardObject(currRow, rowOI)); + int pSz = 1; + + while (pSz < sz && pItr.hasNext()) + { + currRow = pItr.next(); + path.add(ObjectInspectorUtils.copyToStandardObject(currRow, rowOI)); + pSz++; + } + pItr.resetToIndex(idx); + return path; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java new file mode 100644 index 0000000..727195a --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.ptf; + +import java.util.ArrayList; + +import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; + +public class Noop extends TableFunctionEvaluator +{ + + @Override + public PTFPartition execute(PTFPartition iPart) throws HiveException + { + return iPart; + } + + @Override + protected void execute(PTFPartitionIterator pItr, PTFPartition oPart) + { + throw new UnsupportedOperationException(); + } + + public static class NoopResolver extends TableFunctionResolver + { + + @Override + protected TableFunctionEvaluator createEvaluator(PTFDesc ptfDesc, PartitionedTableFunctionDef tDef) + { + return new Noop(); + } + + @Override + public void setupOutputOI() throws SemanticException + { + StructObjectInspector OI = getEvaluator().getTableDef().getInput().getOutputShape().getOI(); + setOutputOI(OI); + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#carryForwardNames() + * Setting to true is correct only for special internal Functions. + */ + @Override + public boolean carryForwardNames() { + return true; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#getOutputNames() + * Set to null only because carryForwardNames is true. + */ + @Override + public ArrayList getOutputColumnNames() { + return null; + } + + @Override + public boolean transformsRawInput() + { + return false; + } + + @Override + public void initializeOutputOI() throws HiveException { + setupOutputOI(); + + } + + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java new file mode 100644 index 0000000..8cbb030 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.ptf; + +import java.util.ArrayList; + +import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; + +public class NoopWithMap extends Noop +{ + @Override + public PTFPartition execute(PTFPartition iPart) throws HiveException + { + return iPart; + } + + @Override + protected PTFPartition _transformRawInput(PTFPartition iPart) throws HiveException + { + return iPart; + } + + public static class NoopWithMapResolver extends TableFunctionResolver + { + + @Override + protected TableFunctionEvaluator createEvaluator(PTFDesc ptfDesc, PartitionedTableFunctionDef tDef) + { + return new NoopWithMap(); + } + + @Override + public void setupOutputOI() throws SemanticException + { + StructObjectInspector OI = getEvaluator().getTableDef().getInput().getOutputShape().getOI(); + setOutputOI(OI); + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#carryForwardNames() + * Setting to true is correct only for special internal Functions. + */ + @Override + public boolean carryForwardNames() { + return true; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#getOutputNames() + * Set to null only because carryForwardNames is true. + */ + @Override + public ArrayList getOutputColumnNames() { + return null; + } + + @Override + public void setupRawInputOI() throws SemanticException + { + StructObjectInspector OI = getEvaluator().getTableDef().getInput().getOutputShape().getOI(); + setRawInputOI(OI); + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#getOutputNames() + * Set to null only because carryForwardNames is true. + */ + @Override + public ArrayList getRawInputColumnNames() throws SemanticException { + return null; + } + + @Override + public boolean transformsRawInput() + { + return true; + } + + @Override + public void initializeOutputOI() throws HiveException { + setupOutputOI(); + } + + @Override + public void initializeRawInputOI() throws HiveException { + setupRawInputOI(); + } + + } + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionEvaluator.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionEvaluator.java new file mode 100644 index 0000000..fa72117 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionEvaluator.java @@ -0,0 +1,169 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.ptf; + +import org.apache.hadoop.hive.ql.exec.PTFOperator; +import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.exec.PTFUtils; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; + +/** + * Based on Hive {@link GenericUDAFEvaluator}. Break up the responsibility of the old AsbtractTableFunction + * class into a Resolver and Evaluator. + *

+ * The Evaluator also holds onto the {@link TableFunctionDef}. This provides information + * about the arguments to the function, the shape of the Input partition and the Partitioning details. + * The Evaluator is responsible for providing the 2 execute methods: + *

    + *
  1. execute: which is invoked after the input is partitioned; the contract + * is, it is given an input Partition and must return an output Partition. The shape of the output + * Partition is obtained from the getOutputOI call. + *
  2. transformRawInput: In the case where this function indicates that it will transform the raw input + * before it is fed through the partitioning mechanics, this function is called. Again the contract is + * t is given an input Partition and must return an Partition. The shape of the output Partition is + * obtained from getRawInputOI() call. + *
+ * + */ +public abstract class TableFunctionEvaluator +{ + /* + * how is this different from the OutpuShape set on the TableDef. + * This is the OI of the object coming out of the PTF. + * It is put in an output Partition whose Serde is usually LazyBinarySerde. + * So the next PTF (or Operator) in the chain gets a LazyBinaryStruct. + */ + transient protected StructObjectInspector OI; + /* + * same comment as OI applies here. + */ + transient protected StructObjectInspector rawInputOI; + protected PartitionedTableFunctionDef tDef; + protected PTFDesc ptfDesc; + String partitionClass; + int partitionMemSize; + boolean transformsRawInput; + + static{ + PTFUtils.makeTransient(TableFunctionEvaluator.class, "OI"); + PTFUtils.makeTransient(TableFunctionEvaluator.class, "rawInputOI"); + } + + + public StructObjectInspector getOutputOI() + { + return OI; + } + + protected void setOutputOI(StructObjectInspector outputOI) + { + OI = outputOI; + } + + public PartitionedTableFunctionDef getTableDef() + { + return tDef; + } + + public void setTableDef(PartitionedTableFunctionDef tDef) + { + this.tDef = tDef; + } + + protected PTFDesc getQueryDef() + { + return ptfDesc; + } + + protected void setQueryDef(PTFDesc ptfDesc) + { + this.ptfDesc = ptfDesc; + } + + public String getPartitionClass() + { + return partitionClass; + } + + public void setPartitionClass(String partitionClass) + { + this.partitionClass = partitionClass; + } + + public int getPartitionMemSize() + { + return partitionMemSize; + } + + public void setPartitionMemSize(int partitionMemSize) + { + this.partitionMemSize = partitionMemSize; + } + + public StructObjectInspector getRawInputOI() + { + return rawInputOI; + } + + protected void setRawInputOI(StructObjectInspector rawInputOI) + { + this.rawInputOI = rawInputOI; + } + + public boolean isTransformsRawInput() { + return transformsRawInput; + } + + public void setTransformsRawInput(boolean transformsRawInput) { + this.transformsRawInput = transformsRawInput; + } + + public PTFPartition execute(PTFPartition iPart) + throws HiveException + { + PTFPartitionIterator pItr = iPart.iterator(); + PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, pItr); + PTFPartition outP = new PTFPartition(getPartitionClass(), + getPartitionMemSize(), tDef.getOutputShape().getSerde(), OI); + execute(pItr, outP); + return outP; + } + + protected abstract void execute(PTFPartitionIterator pItr, PTFPartition oPart) throws HiveException; + + public PTFPartition transformRawInput(PTFPartition iPart) throws HiveException + { + if ( !isTransformsRawInput()) + { + throw new HiveException(String.format("Internal Error: mapExecute called on function (%s)that has no Map Phase", tDef.getName())); + } + return _transformRawInput(iPart); + } + + protected PTFPartition _transformRawInput(PTFPartition iPart) throws HiveException + { + return null; + } +} + diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionResolver.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionResolver.java new file mode 100644 index 0000000..8194b66 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionResolver.java @@ -0,0 +1,214 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.ptf; + +import java.util.ArrayList; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; + +/** + * Based on Hive {@link GenericUDAFResolver}. Break up the responsibility of the + * old AbstractTableFunction class into a Resolver and Evaluator. + * The Resolver is responsible for: + *
    + *
  1. setting up the {@link tableFunctionEvaluator} + *
  2. Setting up the The raw and output ObjectInspectors of the Evaluator. + *
  3. The Evaluator also holds onto the {@link TableFunctionDef}. This provides information + * about the arguments to the function, the shape of the Input partition and the Partitioning details. + *
+ * The Resolver for a function is obtained from the {@link FunctionRegistry}. The Resolver is initialized + * by the following 4 step process: + *
    + *
  1. The initialize method is called; which is passed the {@link PTFDesc} and the {@link TableFunctionDef}. + *
  2. The resolver is then asked to setup the Raw ObjectInspector. This is only required if the Function reshapes + * the raw input. + *
  3. Once the Resolver has had a chance to compute the shape of the Raw Input that is fed to the partitioning + * machinery; the translator sets up the partitioning details on the tableFuncDef. + *
  4. finally the resolver is asked to setup the output ObjectInspector. + *
+ */ +@SuppressWarnings("deprecation") +public abstract class TableFunctionResolver +{ + TableFunctionEvaluator evaluator; + PTFDesc ptfDesc; + + /* + * - called during translation. + * - invokes createEvaluator which must be implemented by a subclass + * - sets up the evaluator with references to the TableDef, PartitionClass, PartitonMemsize and + * the transformsRawInput boolean. + */ + public void initialize(HiveConf cfg, PTFDesc ptfDesc, PartitionedTableFunctionDef tDef) + throws SemanticException + { + this.ptfDesc = ptfDesc; + String partitionClass = HiveConf.getVar(cfg, ConfVars.HIVE_PTF_PARTITION_PERSISTENCE_CLASS); + int partitionMemSize = HiveConf.getIntVar(cfg, ConfVars.HIVE_PTF_PARTITION_PERSISTENT_SIZE); + + evaluator = createEvaluator(ptfDesc, tDef); + evaluator.setTransformsRawInput(transformsRawInput()); + evaluator.setTableDef(tDef); + evaluator.setQueryDef(ptfDesc); + evaluator.setPartitionClass(partitionClass); + evaluator.setPartitionMemSize(partitionMemSize); + + } + + /* + * called during deserialization of a QueryDef during runtime. + */ + public void initialize(PTFDesc ptfDesc, PartitionedTableFunctionDef tDef, TableFunctionEvaluator evaluator) + throws HiveException + { + this.evaluator = evaluator; + this.ptfDesc = ptfDesc; + evaluator.setTableDef(tDef); + evaluator.setQueryDef(ptfDesc); + } + + public TableFunctionEvaluator getEvaluator() + { + return evaluator; + } + + /* + * - a subclass must provide this method. + * - this method is invoked during translation and also when the Operator is initialized during runtime. + * - a subclass must use this call to setup the shape of its output. + * - subsequent to this call, a call to getOutputOI call on the {@link TableFunctionEvaluator} must return the OI + * of the output of this function. + */ + public abstract void setupOutputOI() throws SemanticException; + + /* + * A PTF Function must provide the 'external' names of the columns in its Output. + * + */ + public abstract ArrayList getOutputColumnNames() throws SemanticException; + + + /** + * This method is invoked during runtime(during deserialization of theQueryDef). + * At this point the TableFunction can assume that the {@link ExprNodeDesc Expression Nodes} + * exist for all the Def (ArgDef, ColumnDef, WindowDef..). It is the responsibility of + * the TableFunction to construct the {@link ExprNodeEvaluator evaluators} and setup the OI. + * + * @param tblFuncDef + * @param ptfDesc + * @throws HiveException + */ + public abstract void initializeOutputOI() throws HiveException; + + /* + * - Called on functions that transform the raw input. + * - this method is invoked during translation and also when the Operator is initialized during runtime. + * - a subclass must use this call to setup the shape of the raw input, that is fed to the partitioning mechanics. + * - subsequent to this call, a call to getRawInputOI call on the {@link TableFunctionEvaluator} must return the OI + * of the output of this function. + */ + public void setupRawInputOI() throws SemanticException + { + if (!transformsRawInput()) + { + return; + } + throw new SemanticException( + "Function has map phase, must extend setupMapOI"); + } + + /* + * A PTF Function must provide the 'external' names of the columns in the transformed Raw Input. + * + */ + public ArrayList getRawInputColumnNames() throws SemanticException { + if (!transformsRawInput()) + { + return null; + } + throw new SemanticException( + "Function transforms Raw Input; must extend getRawColumnInputNames"); + } + + /* + * Same responsibility as initializeOI, but for the RawInput. + */ + public void initializeRawInputOI() throws HiveException + { + if (!transformsRawInput()) + { + return; + } + throw new HiveException( + "Function has map phase, must extend initializeRawInputOI"); + } + + /* + * callback method used by subclasses to set the RawInputOI on the Evaluator. + */ + protected void setRawInputOI(StructObjectInspector rawInputOI) + { + evaluator.setRawInputOI(rawInputOI); + } + + /* + * callback method used by subclasses to set the OutputOI on the Evaluator. + */ + protected void setOutputOI(StructObjectInspector outputOI) + { + evaluator.setOutputOI(outputOI); + } + + public PTFDesc getPtfDesc() + { + return ptfDesc; + } + + /* + * This is used during translation to decide if the internalName -> alias mapping from the Input to the PTF is carried + * forward when building the Output RR for this PTF. + * This is used by internal PTFs: NOOP, WindowingTableFunction to make names in its input available in the Output. + * In general this should be false; and the names used for the Output Columns must be provided by the PTF Writer in the + * function getOutputNames. + */ + public boolean carryForwardNames() { + return false; + } + + /* + * a subclass must indicate whether it will transform the raw input before it is fed through the + * partitioning mechanics. + */ + public abstract boolean transformsRawInput(); + + /* + * a subclass must provide the {@link TableFunctionEvaluator} instance. + */ + protected abstract TableFunctionEvaluator createEvaluator(PTFDesc ptfDesc, PartitionedTableFunctionDef tDef); +} diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java new file mode 100644 index 0000000..562ab19 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java @@ -0,0 +1,467 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.ptf; + +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.PTFOperator; +import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.Direction; +import org.apache.hadoop.hive.ql.plan.PTFDesc; +import org.apache.hadoop.hive.ql.plan.PTFDesc.BoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.CurrentRowDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PTFExpressionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.PartitionedTableFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.RangeBoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.ValueBoundaryDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowFunctionDef; +import org.apache.hadoop.hive.ql.plan.PTFDesc.WindowTableFunctionDef; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; + +public class WindowingTableFunction extends TableFunctionEvaluator +{ + + @Override + public PTFPartition execute(PTFPartition iPart) + throws HiveException + { + WindowTableFunctionDef wFnDef = (WindowTableFunctionDef) getTableDef(); + PTFPartitionIterator pItr = iPart.iterator(); + PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, pItr); + PTFPartition outP = new PTFPartition(getPartitionClass(), + getPartitionMemSize(), wFnDef.getOutputFromWdwFnProcessing().getSerde(), OI); + execute(pItr, outP); + return outP; + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + @Override + public void execute(PTFPartitionIterator pItr, PTFPartition outP) throws HiveException + { + ArrayList> oColumns = new ArrayList>(); + PTFPartition iPart = pItr.getPartition(); + StructObjectInspector inputOI; + try { + inputOI = (StructObjectInspector) iPart.getSerDe().getObjectInspector(); + } catch (SerDeException se) { + throw new HiveException(se); + } + + WindowTableFunctionDef wTFnDef = (WindowTableFunctionDef) getTableDef(); + + for(WindowFunctionDef wFn : wTFnDef.getWindowFunctions()) + { + boolean processWindow = wFn.getWindowFrame() != null; + pItr.reset(); + if ( !processWindow ) + { + GenericUDAFEvaluator fEval = wFn.getWFnEval(); + Object[] args = new Object[wFn.getArgs() == null ? 0 : wFn.getArgs().size()]; + AggregationBuffer aggBuffer = fEval.getNewAggregationBuffer(); + while(pItr.hasNext()) + { + Object row = pItr.next(); + int i =0; + if ( wFn.getArgs() != null ) { + for(PTFExpressionDef arg : wFn.getArgs()) + { + args[i++] = arg.getExprEvaluator().evaluate(row); + } + } + fEval.aggregate(aggBuffer, args); + } + Object out = fEval.evaluate(aggBuffer); + if ( !wFn.isPivotResult()) + { + out = new SameList(iPart.size(), out); + } + oColumns.add((List)out); + } + else + { + oColumns.add(executeFnwithWindow(getQueryDef(), wFn, iPart)); + } + } + + /* + * Output Columns in the following order + * - the columns representing the output from Window Fns + * - the input Rows columns + */ + + for(int i=0; i < iPart.size(); i++) + { + ArrayList oRow = new ArrayList(); + Object iRow = iPart.getAt(i); + + for(int j=0; j < oColumns.size(); j++) + { + oRow.add(oColumns.get(j).get(i)); + } + + for(StructField f : inputOI.getAllStructFieldRefs()) + { + oRow.add(inputOI.getStructFieldData(iRow, f)); + } + + outP.append(oRow); + } + } + + public static class WindowingTableFunctionResolver extends TableFunctionResolver + { + /* + * OI of object constructed from output of Wdw Fns; before it is put + * in the Wdw Processing Partition. Set by Translator/Deserializer. + */ + private transient StructObjectInspector wdwProcessingOutputOI; + + public StructObjectInspector getWdwProcessingOutputOI() { + return wdwProcessingOutputOI; + } + + public void setWdwProcessingOutputOI(StructObjectInspector wdwProcessingOutputOI) { + this.wdwProcessingOutputOI = wdwProcessingOutputOI; + } + + @Override + protected TableFunctionEvaluator createEvaluator(PTFDesc ptfDesc, PartitionedTableFunctionDef tDef) + { + + return new WindowingTableFunction(); + } + + @Override + public void setupOutputOI() throws SemanticException { + setOutputOI(wdwProcessingOutputOI); + } + + /* + * Setup the OI based on the: + * - Input TableDef's columns + * - the Window Functions. + */ + @Override + public void initializeOutputOI() throws HiveException + { + setupOutputOI(); + } + + + @Override + public boolean transformsRawInput() + { + return false; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#carryForwardNames() + * Setting to true is correct only for special internal Functions. + */ + @Override + public boolean carryForwardNames() { + return true; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver#getOutputNames() + * Set to null only because carryForwardNames is true. + */ + @Override + public ArrayList getOutputColumnNames() { + return null; + } + + } + + static ArrayList executeFnwithWindow(PTFDesc ptfDesc, WindowFunctionDef wFnDef, PTFPartition iPart) + throws HiveException + { + ArrayList vals = new ArrayList(); + + GenericUDAFEvaluator fEval = wFnDef.getWFnEval(); + Object[] args = new Object[wFnDef.getArgs().size()]; + for(int i=0; i < iPart.size(); i++) + { + AggregationBuffer aggBuffer = fEval.getNewAggregationBuffer(); + Range rng = getRange(wFnDef, i, iPart); + PTFPartitionIterator rItr = rng.iterator(); + PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, rItr); + while(rItr.hasNext()) + { + Object row = rItr.next(); + int j = 0; + for(PTFExpressionDef arg : wFnDef.getArgs()) + { + args[j++] = arg.getExprEvaluator().evaluate(row); + } + fEval.aggregate(aggBuffer, args); + } + Object out = fEval.evaluate(aggBuffer); + out = ObjectInspectorUtils.copyToStandardObject(out, wFnDef.getOI()); + vals.add(out); + } + return vals; + } + + static Range getRange(WindowFunctionDef wFnDef, int currRow, PTFPartition p) throws HiveException + { + BoundaryDef startB = wFnDef.getWindowFrame().getStart(); + BoundaryDef endB = wFnDef.getWindowFrame().getEnd(); + + int start = getIndex(startB, currRow, p, false); + int end = getIndex(endB, currRow, p, true); + + return new Range(start, end, p); + } + + static int getIndex(BoundaryDef bDef, int currRow, PTFPartition p, boolean end) throws HiveException + { + if ( bDef instanceof CurrentRowDef) + { + return currRow + (end ? 1 : 0); + } + else if ( bDef instanceof RangeBoundaryDef) + { + RangeBoundaryDef rbDef = (RangeBoundaryDef) bDef; + int amt = rbDef.getAmt(); + + if ( amt == BoundarySpec.UNBOUNDED_AMOUNT ) + { + return rbDef.getDirection() == Direction.PRECEDING ? 0 : p.size(); + } + + amt = rbDef.getDirection() == Direction.PRECEDING ? -amt : amt; + int idx = currRow + amt; + idx = idx < 0 ? 0 : (idx > p.size() ? p.size() : idx); + return idx + (end && idx < p.size() ? 1 : 0); + } + else + { + ValueBoundaryScanner vbs = ValueBoundaryScanner.getScanner((ValueBoundaryDef)bDef); + return vbs.computeBoundaryRange(currRow, p); + } + } + + static class Range + { + int start; + int end; + PTFPartition p; + + public Range(int start, int end, PTFPartition p) + { + super(); + this.start = start; + this.end = end; + this.p = p; + } + + public PTFPartitionIterator iterator() + { + return p.range(start, end); + } + } + + /* + * - starting from the given rowIdx scan in the given direction until a row's expr + * evaluates to an amt that crosses the 'amt' threshold specified in the ValueBoundaryDef. + */ + static abstract class ValueBoundaryScanner + { + ValueBoundaryDef bndDef; + + public ValueBoundaryScanner(ValueBoundaryDef bndDef) + { + this.bndDef = bndDef; + } + + /* + * return the other end of the Boundary + * - when scanning backwards: go back until you reach a row where the + * startingValue - rowValue >= amt + * - when scanning forward: go forward go back until you reach a row where the + * rowValue - startingValue >= amt + */ + public int computeBoundaryRange(int rowIdx, PTFPartition p) throws HiveException + { + int r = rowIdx; + Object rowValue = computeValue(p.getAt(r)); + int amt = bndDef.getAmt(); + + if ( amt == BoundarySpec.UNBOUNDED_AMOUNT ) + { + return bndDef.getDirection() == Direction.PRECEDING ? 0 : p.size(); + } + + Direction d = bndDef.getDirection(); + boolean scanNext = rowValue != null; + while ( scanNext ) + { + if ( d == Direction.PRECEDING ) { + r = r - 1; + } + else { + r = r + 1; + } + + if ( r < 0 || r >= p.size() ) + { + scanNext = false; + break; + } + + Object currVal = computeValue(p.getAt(r)); + if ( currVal == null ) + { + scanNext = false; + break; + } + + switch(d) + { + case PRECEDING: + scanNext = !isGreater(rowValue, currVal, amt); + break; + case FOLLOWING: + scanNext = !isGreater(currVal, rowValue, amt); + case CURRENT: + default: + break; + } + } + /* + * if moving backwards, then r is at a row that failed the range test. So incr r, so that + * Range starts from a row where the test succeeds. + * Whereas when moving forward, leave r as is; because the Range's end value should be the + * row idx not in the Range. + */ + if ( d == Direction.PRECEDING ) { + r = r + 1; + } + r = r < 0 ? 0 : (r >= p.size() ? p.size() : r); + return r; + } + + public Object computeValue(Object row) throws HiveException + { + Object o = bndDef.getExprEvaluator().evaluate(row); + return ObjectInspectorUtils.copyToStandardObject(o, bndDef.getOI()); + } + + public abstract boolean isGreater(Object v1, Object v2, int amt); + + + @SuppressWarnings("incomplete-switch") + public static ValueBoundaryScanner getScanner(ValueBoundaryDef vbDef) + { + PrimitiveObjectInspector pOI = (PrimitiveObjectInspector) vbDef.getOI(); + switch(pOI.getPrimitiveCategory()) + { + case BYTE: + case INT: + case LONG: + case SHORT: + case TIMESTAMP: + return new LongValueBoundaryScanner(vbDef); + case DOUBLE: + case FLOAT: + return new DoubleValueBoundaryScanner(vbDef); + } + return null; + } + } + + public static class LongValueBoundaryScanner extends ValueBoundaryScanner + { + public LongValueBoundaryScanner(ValueBoundaryDef bndDef) + { + super(bndDef); + } + + @Override + public boolean isGreater(Object v1, Object v2, int amt) + { + long l1 = PrimitiveObjectInspectorUtils.getLong(v1, + (PrimitiveObjectInspector) bndDef.getOI()); + long l2 = PrimitiveObjectInspectorUtils.getLong(v2, + (PrimitiveObjectInspector) bndDef.getOI()); + return (l1 -l2) >= amt; + } + } + + public static class DoubleValueBoundaryScanner extends ValueBoundaryScanner + { + public DoubleValueBoundaryScanner(ValueBoundaryDef bndDef) + { + super(bndDef); + } + + @Override + public boolean isGreater(Object v1, Object v2, int amt) + { + double d1 = PrimitiveObjectInspectorUtils.getDouble(v1, + (PrimitiveObjectInspector) bndDef.getOI()); + double d2 = PrimitiveObjectInspectorUtils.getDouble(v2, + (PrimitiveObjectInspector) bndDef.getOI()); + return (d1 -d2) >= amt; + } + } + + public static class SameList extends AbstractList + { + int sz; + E val; + + public SameList(int sz, E val) + { + this.sz = sz; + this.val = val; + } + + @Override + public E get(int index) + { + return val; + } + + @Override + public int size() + { + return sz; + } + + } + +} diff --git ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q new file mode 100644 index 0000000..542367a --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q @@ -0,0 +1,20 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- testAggrFuncsWithNoGBYNoPartDef +select p_mfgr, +sum(p_retailprice) as s1 +from part; \ No newline at end of file diff --git ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q new file mode 100644 index 0000000..49f7858 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q @@ -0,0 +1,28 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- testAmbiguousWindowDefn +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over (w1), +sum(p_size) as s2 over (w2), +sum(p_size) as s3 over (w3) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as (rows between unbounded preceding and current row), + w3 as w3; + diff --git ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q new file mode 100644 index 0000000..d0a7142 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q @@ -0,0 +1,19 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testPartitonBySortBy +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (distribute by p_mfgr order by p_mfgr) +from part +; \ No newline at end of file diff --git ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q new file mode 100644 index 0000000..adca0da --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q @@ -0,0 +1,22 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testDuplicateWindowAlias +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over (w1), +sum(p_size) as s2 over (w2) +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as w1, + w2 as (rows between unbounded preceding and current row); diff --git ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q new file mode 100644 index 0000000..b4e0ea3 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q @@ -0,0 +1,22 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- testHavingLeadWithNoGBYNoWindowing +select p_mfgr,p_name, p_size +from part +having lead(p_size, 1) <= p_size +distribute by p_mfgr +sort by p_name; diff --git ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q new file mode 100644 index 0000000..b5fbcc5 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q @@ -0,0 +1,22 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testHavingLeadWithPTF +select p_mfgr,p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name) +having lead(p_size, 1) <= p_size +distribute by p_mfgr +sort by p_name; diff --git ql/src/test/queries/clientnegative/ptf_negative_IncompatibleDistributeClause.q ql/src/test/queries/clientnegative/ptf_negative_IncompatibleDistributeClause.q new file mode 100644 index 0000000..4820a60 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_IncompatibleDistributeClause.q @@ -0,0 +1,21 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testIncompatibleDistributeClause +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr, +sum(p_size) as s over (w1) +from part +distribute by p_mfgr +window w1 as (partition by p_name rows between 2 preceding and 2 following); diff --git ql/src/test/queries/clientnegative/ptf_negative_IncompatibleOrderInWindowDefs.q ql/src/test/queries/clientnegative/ptf_negative_IncompatibleOrderInWindowDefs.q new file mode 100644 index 0000000..bf05570 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_IncompatibleOrderInWindowDefs.q @@ -0,0 +1,23 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testIncompatibleOrderInWindowDefs +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over (w1), +sum(p_size) as s2 over (w2) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as (partition by p_mfgr order by p_name rows between unbounded preceding and current row); diff --git ql/src/test/queries/clientnegative/ptf_negative_IncompatiblePartitionInWindowDefs.q ql/src/test/queries/clientnegative/ptf_negative_IncompatiblePartitionInWindowDefs.q new file mode 100644 index 0000000..e3bb2fd --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_IncompatiblePartitionInWindowDefs.q @@ -0,0 +1,23 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testIncompatiblePartitionInWindowDefs +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over (w1), +sum(p_size) as s2 over (w2) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as (partition by p_name order by p_name rows between unbounded preceding and current row); diff --git ql/src/test/queries/clientnegative/ptf_negative_IncompatibleSortClause.q ql/src/test/queries/clientnegative/ptf_negative_IncompatibleSortClause.q new file mode 100644 index 0000000..1f52fac --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_IncompatibleSortClause.q @@ -0,0 +1,21 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testIncompatibleSortClause +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr, +sum(p_size) as s over (w1) +from part +distribute by p_mfgr +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following); diff --git ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q new file mode 100644 index 0000000..90827aa --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q @@ -0,0 +1,24 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- testInvalidValueBoundary +select p_mfgr,p_name, p_size, +sum(p_size) as s over (w1) , +dense_rank() as dr +from part +distribute by p_mfgr +sort by p_name +window w1 as (range between p_name 2 less and current row); \ No newline at end of file diff --git ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q new file mode 100644 index 0000000..80441e4 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q @@ -0,0 +1,20 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testJoinWithAmbigousAlias +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part on abc.p_partkey = p1.p_partkey; diff --git ql/src/test/queries/clientnegative/ptf_negative_NoSortNoDistByClause.q ql/src/test/queries/clientnegative/ptf_negative_NoSortNoDistByClause.q new file mode 100644 index 0000000..ef617a9 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_NoSortNoDistByClause.q @@ -0,0 +1,21 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- testNoSortNoDistByClause +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr +from part +window w1 as (rows between 2 preceding and 2 following); \ No newline at end of file diff --git ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q new file mode 100644 index 0000000..1571f41 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q @@ -0,0 +1,19 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +-- testPartitonBySortBy +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr sort by p_mfgr) +from part +; \ No newline at end of file diff --git ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q new file mode 100644 index 0000000..d773ef0 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q @@ -0,0 +1,23 @@ +DROP TABLE part; + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- testWhereWithRankCond +select p_mfgr,p_name, p_size, +rank() as r +from part +where r < 4 +distribute by p_mfgr +sort by p_mfgr; diff --git ql/src/test/queries/clientnegative/ptf_window_boundaries.q ql/src/test/queries/clientnegative/ptf_window_boundaries.q new file mode 100644 index 0000000..07febb5 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_window_boundaries.q @@ -0,0 +1,17 @@ +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows unbounded following) + from part distribute by p_mfgr sort by p_name; + diff --git ql/src/test/queries/clientnegative/ptf_window_boundaries2.q ql/src/test/queries/clientnegative/ptf_window_boundaries2.q new file mode 100644 index 0000000..05399d2 --- /dev/null +++ ql/src/test/queries/clientnegative/ptf_window_boundaries2.q @@ -0,0 +1,17 @@ +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range unbounded following) + from part distribute by p_mfgr sort by p_name; + diff --git ql/src/test/queries/clientpositive/leadlag.q ql/src/test/queries/clientpositive/leadlag.q new file mode 100644 index 0000000..1fe88a9 --- /dev/null +++ ql/src/test/queries/clientpositive/leadlag.q @@ -0,0 +1,87 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +--1. testLagWithPTFWindowing +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_retailprice, sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +); + +-- 2. testLagWithWindowingNoPTF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name ; + +-- 3. testJoinWithLag +select p1.p_mfgr, p1.p_name, +p1.p_size, p1.p_size - lag(p1.p_size,1) as deltaSz +from part p1 join part p2 on p1.p_partkey = p2.p_partkey +distribute by p1.p_mfgr +sort by p1.p_name ; + +-- 4. testLagInSum +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) as deltaSum +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) ; + +-- 5. testLagInSumOverWindow +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) as deltaSum over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) ; + +-- 6. testRankInLead +select p_mfgr, p_name, p_size, +rank() as r1, +lead(rank(), 1) as deltaRank +from part +distribute by p_mfgr +sort by p_name; + +-- 7. testLeadWithPTF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, p_size - lead(p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) +distribute by p_mfgr +sort by p_name; + +-- 8. testOverNoPartitionMultipleAggregate +select p_name, p_retailprice, +lead(p_retailprice) as l1 over(), +lag(p_retailprice) as l2 over() +from part +order by p_name; diff --git ql/src/test/queries/clientpositive/leadlag_queries.q ql/src/test/queries/clientpositive/leadlag_queries.q new file mode 100644 index 0000000..aef14da --- /dev/null +++ ql/src/test/queries/clientpositive/leadlag_queries.q @@ -0,0 +1,53 @@ +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- 1. testLeadUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice) as l1 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1) as l2 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1,10) as l3 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1, p_retailprice) as l4 over (partition by p_mfgr order by p_name), +p_retailprice - lead(p_retailprice,1) +from part; + +-- 2.testLeadUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lead(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lead(p_retailprice,1) +from part; + +-- 3.testLagUDAF +select p_mfgr, p_retailprice, +lag(p_retailprice,1) as l1 over (partition by p_mfgr order by p_name), +lag(p_retailprice) as l2 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1, p_retailprice) as l3 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1,10) as l4 over (partition by p_mfgr order by p_name), +p_retailprice - lag(p_retailprice,1) +from part; + +-- 4.testLagUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lag(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lag(p_retailprice,1) +from part; + +-- 5.testLeadLagUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice,1) as l1 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1, p_retailprice) as l2 over (partition by p_mfgr order by p_name), +p_retailprice - lead(p_retailprice,1), +lag(p_retailprice,1) as l3 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1, p_retailprice) as l4 over (partition by p_mfgr order by p_name) +from part; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/ptf.q ql/src/test/queries/clientpositive/ptf.q new file mode 100644 index 0000000..8fb57bc --- /dev/null +++ ql/src/test/queries/clientpositive/ptf.q @@ -0,0 +1,318 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +--1. test1 +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part + partition by p_mfgr + order by p_name + ); + +-- 2. testJoinWithNoop +select p_mfgr, p_name, +p_size, p_size - lag(p_size,1) as deltaSz +from noop (on (select p1.* from part p1 join part p2 on p1.p_partkey = p2.p_partkey) j +distribute by j.p_mfgr +sort by j.p_name) +distribute by p_mfgr +sort by p_name; + +-- 3. testOnlyPTF +select p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name); + +-- 4. testPTFAlias +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part + partition by p_mfgr + order by p_name + ) abc; + +-- 5. testPTFAndWhereWithWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +having p_size > 0 +distribute by p_mfgr +sort by p_name; + +-- 6. testSWQAndPTFAndGBy +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +group by p_mfgr, p_name, p_size +distribute by p_mfgr +sort by p_name; + +-- 7. testJoin +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey; + +-- 8. testJoinRight +select abc.* +from part p1 join noop(on part +partition by p_mfgr +order by p_name +) abc on abc.p_partkey = p1.p_partkey; + +-- 9. testNoopWithMap +select p_mfgr, p_name, p_size, +rank() as r over (partition by p_mfgr order by p_name, p_size desc) +from noopwithmap(on part +partition by p_mfgr +order by p_name, p_size desc); + +-- 10. testNoopWithMapWithWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noopwithmap(on part + partition by p_mfgr + order by p_name); + +-- 11. testHavingWithWindowingPTFNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part +partition by p_mfgr +order by p_name) +having rank() < 4; + +-- 12. testFunctionChain +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on noopwithmap(on noop(on part +partition by p_mfgr +order by p_mfgr, p_name +))); + +-- 13. testPTFAndWindowingInSubQ +select p_mfgr, p_name, +sub1.cd, sub1.s1 +from (select p_mfgr, p_name, +count(p_size) as cd, +p_retailprice, +sum(p_retailprice) as s1 over w1 +from noop(on part +partition by p_mfgr +order by p_name) +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +) sub1 ; + +-- 14. testPTFJoinWithWindowingWithCount +select abc.p_mfgr, abc.p_name, +rank() as r, +dense_rank() as dr, +count(abc.p_name) as cd, +abc.p_retailprice, sum(abc.p_retailprice) as s1 over (rows between unbounded preceding and current row), +abc.p_size, abc.p_size - lag(abc.p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +distribute by abc.p_mfgr +sort by abc.p_name; + +-- 15. testDistinctInSelectWithPTF +select DISTINCT p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name); + + +-- 16. testViewAsTableInputToPTF +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand; + +select p_mfgr, p_brand, s, +sum(s) as s1 over w1 +from noop(on mfgr_price_view +partition by p_mfgr +order by p_mfgr) +window w1 as ( partition by p_mfgr order by p_brand rows between 2 preceding and current row); + +-- 17. testMultipleInserts2SWQsWithPTF +CREATE TABLE part_4( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE); + +CREATE TABLE part_5( +p_mfgr STRING, +p_name STRING, +p_size INT, +s1 INT, +s2 INT, +r INT, +dr INT, +cud DOUBLE, +fv1 INT); + +from noop(on part +partition by p_mfgr +order by p_name) +INSERT OVERWRITE TABLE part_4 select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s over (rows between unbounded preceding and current row) +distribute by p_mfgr +sort by p_name +INSERT OVERWRITE TABLE part_5 select p_mfgr,p_name, p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +rank() as r, +dense_rank() as dr, +cume_dist() as cud, +first_value(p_size, true) as fv1 over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +select * from part_4; + +select * from part_5; + +-- 18. testMulti2OperatorsFunctionChainWithMap +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr,p_name rows between unbounded preceding and current row) +from noop(on + noopwithmap(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr,p_name + order by p_mfgr,p_name) ; + +-- 19. testMulti3OperatorsFunctionChain +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr + order by p_mfgr ) ; + +-- 20. testMultiOperatorChainWithNoWindowing +select p_mfgr, p_name, +rank() as r over (partition by p_mfgr order by p_name), +dense_rank() as dr, +p_size, sum(p_size) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr)); + + +-- 21. testMultiOperatorChainEndsWithNoopMap +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr,p_name rows between unbounded preceding and current row) +from noopwithmap(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr) + partition by p_mfgr,p_name + order by p_mfgr,p_name); + +-- 22. testMultiOperatorChainWithDiffPartitionForWindow1 +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (partition by p_mfgr,p_name order by p_mfgr,p_name rows between unbounded preceding and current row) +from noop(on + noopwithmap(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + partition by p_mfgr + order by p_mfgr + )); + +-- 23. testMultiOperatorChainWithDiffPartitionForWindow2 +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (partition by p_mfgr order by p_mfgr rows between unbounded preceding and current row) +from noopwithmap(on + noop(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + )); + diff --git ql/src/test/queries/clientpositive/ptf_general_queries.q ql/src/test/queries/clientpositive/ptf_general_queries.q new file mode 100644 index 0000000..dd9c9b8 --- /dev/null +++ ql/src/test/queries/clientpositive/ptf_general_queries.q @@ -0,0 +1,35 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- 1. testNoPTFNoWindowing +select p_mfgr, p_name, p_size +from part +distribute by p_mfgr +sort by p_name ; + +-- 2. testUDAFsNoWindowingNoPTFNoGBY +select p_mfgr,p_name, p_retailprice, +sum(p_retailprice) as s, +min(p_retailprice) as mi, +max(p_retailprice) as ma, +avg(p_retailprice) as av +from part +distribute by p_mfgr +sort by p_mfgr, p_name; + +-- 3. testConstExprInSelect +select 'tst1' as key, count(1) as value from part; diff --git ql/src/test/queries/clientpositive/ptf_npath.q ql/src/test/queries/clientpositive/ptf_npath.q new file mode 100644 index 0000000..c87ddb0 --- /dev/null +++ ql/src/test/queries/clientpositive/ptf_npath.q @@ -0,0 +1,24 @@ +DROP TABLE flights_tiny; + +create table flights_tiny ( +ORIGIN_CITY_NAME string, +DEST_CITY_NAME string, +YEAR int, +MONTH int, +DAY_OF_MONTH int, +ARR_DELAY float, +FL_NUM string +); + +LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; + +-- 1. basic Npath test +select origin_city_name, fl_num, year, month, day_of_month, sz, tpath +from npath(on + flights_tiny + distribute by fl_num + sort by year, month, day_of_month + arg1('LATE.LATE+'), + arg2('LATE'), arg3(arr_delay > 15), + arg4('origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath as tpath') + ); diff --git ql/src/test/queries/clientpositive/ptf_rcfile.q ql/src/test/queries/clientpositive/ptf_rcfile.q new file mode 100644 index 0000000..d3af55f --- /dev/null +++ ql/src/test/queries/clientpositive/ptf_rcfile.q @@ -0,0 +1,24 @@ +DROP TABLE part_rc; + +CREATE TABLE part_rc( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) STORED AS RCFILE ; + +LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc; + +-- testWindowingPTFWithPartRC +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part_rc +partition by p_mfgr +order by p_name); diff --git ql/src/test/queries/clientpositive/ptf_seqfile.q ql/src/test/queries/clientpositive/ptf_seqfile.q new file mode 100644 index 0000000..ac29043 --- /dev/null +++ ql/src/test/queries/clientpositive/ptf_seqfile.q @@ -0,0 +1,24 @@ +DROP TABLE part_seq; + +CREATE TABLE part_seq( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) STORED AS SEQUENCEFILE ; + +LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq; + +-- testWindowingPTFWithPartSeqFile +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part_seq +partition by p_mfgr +order by p_name); diff --git ql/src/test/queries/clientpositive/windowing.q ql/src/test/queries/clientpositive/windowing.q new file mode 100644 index 0000000..f3d727f --- /dev/null +++ ql/src/test/queries/clientpositive/windowing.q @@ -0,0 +1,484 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- 1. testWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +distribute by p_mfgr +sort by p_name; + +-- 2. testGroupByWithPartitioning +select p_mfgr, p_name, p_size, min(p_retailprice), +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +distribute by p_mfgr +sort by p_name ; + +-- 3. testGroupByHavingWithSWQ +select p_mfgr, p_name, p_size, min(p_retailprice), +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +having p_size > 0 +distribute by p_mfgr +sort by p_name ; + +-- 4. testCount +select p_mfgr, p_name, +count(p_size) as cd +from part +distribute by p_mfgr +sort by p_name; + +-- 5. testCountWithWindowingUDAF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +count(p_size) as cd, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name; + +-- 6. testCountInSubQ +select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz +from (select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +count(p_size) as cd, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +) sub1; + +-- 7. testJoinWithWindowingAndPTF +select abc.p_mfgr, abc.p_name, +rank() as r, +dense_rank() as dr, +abc.p_retailprice, sum(abc.p_retailprice) as s1 over (rows between unbounded preceding and current row), +abc.p_size, abc.p_size - lag(abc.p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +distribute by abc.p_mfgr +sort by abc.p_name ; + +-- 8. testMixedCaseAlias +select p_mfgr, p_name, p_size, rank() as R +from part +distribute by p_mfgr +sort by p_name, p_size desc; + +-- 9. testHavingWithWindowingNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +having p_size > 5 +distribute by p_mfgr +sort by p_name; + +-- 10. testHavingWithWindowingCondRankNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +having rank() < 4 +distribute by p_mfgr +sort by p_name; + +-- 11. testFirstLast +select p_mfgr,p_name, p_size, +sum(p_size) as s2 over (rows between current row and current row), +first_value(p_size) as f over w1 , +last_value(p_size, false) as l over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following); + +-- 12. testFirstLastWithWhere +select p_mfgr,p_name, p_size, +rank() as r, +sum(p_size) as s2 over (rows between current row and current row), +first_value(p_size) as f over w1, +last_value(p_size, false) as l over w1 +from part +where p_mfgr = 'Manufacturer#3' +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following); + +-- 13. testSumWindow +select p_mfgr,p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over (rows between current row and current row) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following); + +-- 14. testNoSortClause +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr +from part +distribute by p_mfgr +window w1 as (rows between 2 preceding and 2 following); + +-- 15. testExpressions +select p_mfgr,p_name, p_size, +rank() as r, +dense_rank() as dr, +cume_dist() as cud, +percent_rank() as pr, +ntile(3) as nt, +count(p_size) as ca, +avg(p_size) as avg, +stddev(p_size) as st, +first_value(p_size % 5) as fv, +last_value(p_size) as lv, +first_value(p_size, true) as fvW1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 16. testMultipleWindows +select p_mfgr,p_name, p_size, + rank() as r, dense_rank() as dr, +cume_dist() as cud, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +first_value(p_size, true) as fv1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 17. testCountStar +select p_mfgr,p_name, p_size, +count(*) as c, +count(p_size) as ca, +first_value(p_size, true) as fvW1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 18. testUDAFs +select p_mfgr,p_name, p_size, +sum(p_retailprice) as s over w1, +min(p_retailprice) as mi over w1, +max(p_retailprice) as ma over w1, +avg(p_retailprice) as ag over w1 +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 19. testUDAFsWithGBY +select p_mfgr,p_name, p_size, p_retailprice, +sum(p_retailprice) as s over w1, +min(p_retailprice) as mi , +max(p_retailprice) as ma , +avg(p_retailprice) as ag over w1 +from part +group by p_mfgr,p_name, p_size, p_retailprice +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 20. testSTATs +select p_mfgr,p_name, p_size, +stddev(p_retailprice) as sdev over w1, +stddev_pop(p_retailprice) as sdev_pop over w1, +collect_set(p_size) as uniq_size over w1, +variance(p_retailprice) as var over w1, +corr(p_size, p_retailprice) as cor over w1, +covar_pop(p_size, p_retailprice) as covarp over w1 +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 21. testDISTs +select p_mfgr,p_name, p_size, +histogram_numeric(p_retailprice, 5) as hist over w1, +percentile(p_partkey, 0.5) as per over w1, +row_number() as rn +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 22. testViewAsTableInputWithWindowing +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand; + +select p_mfgr, p_brand, s, +sum(s) as s1 over w1 +from mfgr_price_view +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and current row); + +-- 23. testCreateViewWithWindowingQuery +create view IF NOT EXISTS mfgr_brand_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and current row); + +select * from mfgr_brand_price_view; + +-- 24. testLateralViews +select p_mfgr, p_name, +lv_col, p_size, sum(p_size) as s over w1 +from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p +lateral view explode(arr) part_lv as lv_col +distribute by p_mfgr +sort by p_name +window w1 as (rows between 2 preceding and current row); + +-- 25. testMultipleInserts3SWQs +CREATE TABLE part_1( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE); + +CREATE TABLE part_2( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +cud INT, +s1 DOUBLE, +s2 DOUBLE, +fv1 INT); + +CREATE TABLE part_3( +p_mfgr STRING, +p_name STRING, +p_size INT, +c INT, +ca INT, +fv INT); + +from part +INSERT OVERWRITE TABLE part_1 +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s over (rows between unbounded preceding and current row) +distribute by p_mfgr +sort by p_name +INSERT OVERWRITE TABLE part_2 +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr, +cume_dist() as cud, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +first_value(p_size, true) as fv1 over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +INSERT OVERWRITE TABLE part_3 +select p_mfgr,p_name, p_size, +count(*) as c, +count(p_size) as ca, +first_value(p_size, true) as fv over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following); + +select * from part_1; + +select * from part_2; + +select * from part_3; + +-- 26. testGroupByHavingWithSWQAndAlias +select p_mfgr, p_name, p_size, min(p_retailprice) as mi, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +having p_size > 0 +distribute by p_mfgr +sort by p_name; + +-- 27. testMultipleRangeWindows +select p_mfgr,p_name, p_size, +sum(p_size) as s2 over (range between p_size 10 less and current row), +sum(p_size) as s1 over (range between current row and p_size 10 more ) +from part +distribute by p_mfgr +sort by p_mfgr, p_size +window w1 as (rows between 2 preceding and 2 following); + +-- 28. testPartOrderInUDAFInvoke +select p_mfgr, p_name, p_size, +sum(p_size) as s over (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +from part; + +-- 29. testPartOrderInWdwDef +select p_mfgr, p_name, p_size, +sum(p_size) as s over w1 +from part +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following); + +-- 30. testDefaultPartitioningSpecRules +select p_mfgr, p_name, p_size, +sum(p_size) as s over w1, + sum(p_size) as s2 over w2 +from part +sort by p_name +window w1 as (partition by p_mfgr rows between 2 preceding and 2 following), + w2 as (partition by p_mfgr order by p_name); + +-- 31. testWindowCrossReference +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as w1; + + +-- 32. testWindowInheritance +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as (w1 rows between unbounded preceding and current row); + + +-- 33. testWindowForwardReference +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2, +sum(p_size) as s3 over w3 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as w3, + w3 as (rows between unbounded preceding and current row); + + +-- 34. testWindowDefinitionPropagation +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2, +sum(p_size) as s3 over (w3 rows between 2 preceding and 2 following) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as w3, + w3 as (rows between unbounded preceding and current row); + +-- 35. testDistinctWithWindowing +select DISTINCT p_mfgr, p_name, p_size, +sum(p_size) as s over w1 +from part +distribute by p_mfgr +sort by p_name +window w1 as (rows between 2 preceding and 2 following); + +-- 36. testRankWithPartitioning +select p_mfgr, p_name, p_size, +rank() as r over (partition by p_mfgr order by p_name ) +from part; + +-- 37. testPartitioningVariousForms +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_mfgr), +min(p_retailprice) as s2 over (partition by p_mfgr), +max(p_retailprice) as s3 over (distribute by p_mfgr sort by p_mfgr), +avg(p_retailprice) as s4 over (distribute by p_mfgr), +count(p_retailprice) as s5 over (cluster by p_mfgr ) +from part; + +-- 38. testPartitioningVariousForms2 +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr, p_name order by p_mfgr, p_name rows between unbounded preceding and current row), +min(p_retailprice) as s2 over (distribute by p_mfgr, p_name sort by p_mfgr, p_name rows between unbounded preceding and current row), +max(p_retailprice) as s3 over (cluster by p_mfgr, p_name ) +from part; + +-- 39. testUDFOnOrderCols +select p_mfgr, p_type, substr(p_type, 2) as short_ptype, +rank() as r over (partition by p_mfgr order by substr(p_type, 2)) +from part; + +-- 40. testNoBetweenForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows unbounded preceding) + from part distribute by p_mfgr sort by p_name; + +-- 41. testNoBetweenForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range unbounded preceding) + from part distribute by p_mfgr sort by p_name; + +-- 42. testUnboundedFollowingForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows between current row and unbounded following) + from part distribute by p_mfgr sort by p_name; + +-- 43. testUnboundedFollowingForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range between current row and unbounded following) + from part distribute by p_mfgr sort by p_name; + +-- 44. testOverNoPartitionSingleAggregate +select p_name, p_retailprice, +avg(p_retailprice) over() +from part +order by p_name; + \ No newline at end of file diff --git ql/src/test/queries/clientpositive/windowing_columnPruning.q ql/src/test/queries/clientpositive/windowing_columnPruning.q new file mode 100644 index 0000000..052b877 --- /dev/null +++ ql/src/test/queries/clientpositive/windowing_columnPruning.q @@ -0,0 +1,33 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- 1. testQueryLevelPartitionColsNotInSelect +select p_size, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +distribute by p_mfgr sort by p_name ; + +-- 2. testWindowPartitionColsNotInSelect +select p_size, +sum(p_retailprice) as s1 over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) +from part; + +-- 3. testHavingColNotInSelect +select p_mfgr, +sum(p_retailprice) as s1 over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) +from part +having p_size > 5; diff --git ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out new file mode 100644 index 0000000..0568f2d --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q.out @@ -0,0 +1,36 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +FAILED: SemanticException 6:5 No partition specification associated with Windowing. Error encountered near token 's1' diff --git ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out new file mode 100644 index 0000000..cc02d3b --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_AmbiguousWindowDefn.q.out @@ -0,0 +1,36 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +FAILED: SemanticException Cycle in Window references [w3, w3] diff --git ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out new file mode 100644 index 0000000..926edbf --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_DistributeByOrderBy.q.out @@ -0,0 +1,31 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: ParseException line 5:52 mismatched input 'order' expecting ) near 'p_mfgr' in window_specification + diff --git ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out new file mode 100644 index 0000000..6a3a036 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_DuplicateWindowAlias.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException 10:7 Duplicate definition of window w2 is not allowed. Error encountered near token 'w2' diff --git ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out new file mode 100644 index 0000000..1d5594d --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q.out @@ -0,0 +1,36 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +FAILED: SemanticException 6:23 Query has no windowing or group by clause: Unsupported place for having. Error encountered near token 'p_size' diff --git ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out new file mode 100644 index 0000000..93d247f --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_HavingLeadWithPTF.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException 8:23 Query has no windowing or group by clause: Unsupported place for having. Error encountered near token 'p_size' diff --git ql/src/test/results/clientnegative/ptf_negative_IncompatibleDistributeClause.q.out ql/src/test/results/clientnegative/ptf_negative_IncompatibleDistributeClause.q.out new file mode 100644 index 0000000..9db9f75 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_IncompatibleDistributeClause.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException Window Function 'sum' has an incompatible partition clause diff --git ql/src/test/results/clientnegative/ptf_negative_IncompatibleOrderInWindowDefs.q.out ql/src/test/results/clientnegative/ptf_negative_IncompatibleOrderInWindowDefs.q.out new file mode 100644 index 0000000..05dc233 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_IncompatibleOrderInWindowDefs.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException Window Function 'sum' has an incompatible order clause diff --git ql/src/test/results/clientnegative/ptf_negative_IncompatiblePartitionInWindowDefs.q.out ql/src/test/results/clientnegative/ptf_negative_IncompatiblePartitionInWindowDefs.q.out new file mode 100644 index 0000000..9db9f75 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_IncompatiblePartitionInWindowDefs.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException Window Function 'sum' has an incompatible partition clause diff --git ql/src/test/results/clientnegative/ptf_negative_IncompatibleSortClause.q.out ql/src/test/results/clientnegative/ptf_negative_IncompatibleSortClause.q.out new file mode 100644 index 0000000..05dc233 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_IncompatibleSortClause.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException Window Function 'sum' has an incompatible order clause diff --git ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out new file mode 100644 index 0000000..961a979 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_InvalidValueBoundary.q.out @@ -0,0 +1,36 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +FAILED: SemanticException Primitve type STRING not supported in Value Boundary expression diff --git ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out new file mode 100644 index 0000000..2cf9d5d --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_JoinWithAmbigousAlias.q.out @@ -0,0 +1,30 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: SemanticException [Error 10008]: Line 8:11 Ambiguous table alias 'part' diff --git ql/src/test/results/clientnegative/ptf_negative_NoSortNoDistByClause.q.out ql/src/test/results/clientnegative/ptf_negative_NoSortNoDistByClause.q.out new file mode 100644 index 0000000..424dcd8 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_NoSortNoDistByClause.q.out @@ -0,0 +1,36 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +FAILED: SemanticException 6:5 No partition specification associated with Windowing. Error encountered near token '2' diff --git ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out new file mode 100644 index 0000000..0a24882 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_PartitionBySortBy.q.out @@ -0,0 +1,31 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: ParseException line 5:51 mismatched input 'sort' expecting ) near 'p_mfgr' in window_specification + diff --git ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out new file mode 100644 index 0000000..277b07a --- /dev/null +++ ql/src/test/results/clientnegative/ptf_negative_WhereWithRankCond.q.out @@ -0,0 +1,36 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +FAILED: SemanticException [Error 10004]: Line 7:6 Invalid table alias or column reference 'r': (possible column names are: p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment) diff --git ql/src/test/results/clientnegative/ptf_window_boundaries.q.out ql/src/test/results/clientnegative/ptf_window_boundaries.q.out new file mode 100644 index 0000000..fa10f3e --- /dev/null +++ ql/src/test/results/clientnegative/ptf_window_boundaries.q.out @@ -0,0 +1,29 @@ +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: ParseException line 4:50 mismatched input 'following' expecting KW_PRECEDING near 'unbounded' in window_range_expression + diff --git ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out new file mode 100644 index 0000000..840a384 --- /dev/null +++ ql/src/test/results/clientnegative/ptf_window_boundaries2.q.out @@ -0,0 +1,29 @@ +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +FAILED: ParseException line 4:51 mismatched input 'following' expecting KW_PRECEDING near 'unbounded' in window_value_expression + diff --git ql/src/test/results/clientpositive/leadlag.q.out ql/src/test/results/clientpositive/leadlag.q.out new file mode 100644 index 0000000..ff96d06 --- /dev/null +++ ql/src/test/results/clientpositive/leadlag.q.out @@ -0,0 +1,421 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +PREHOOK: query: --1. testLagWithPTFWindowing +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_retailprice, sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: --1. testLagWithPTFWindowing +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_retailprice, sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 1464.48 7672.66 23 -23 +PREHOOK: query: -- 2. testLagWithWindowingNoPTF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 2. testLagWithWindowingNoPTF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 1464.48 7672.66 23 -23 +PREHOOK: query: -- 3. testJoinWithLag +select p1.p_mfgr, p1.p_name, +p1.p_size, p1.p_size - lag(p1.p_size,1) as deltaSz +from part p1 join part p2 on p1.p_partkey = p2.p_partkey +distribute by p1.p_mfgr +sort by p1.p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 3. testJoinWithLag +select p1.p_mfgr, p1.p_name, +p1.p_size, p1.p_size - lag(p1.p_size,1) as deltaSz +from part p1 join part p2 on p1.p_partkey = p2.p_partkey +distribute by p1.p_mfgr +sort by p1.p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 5 +Manufacturer#3 almond antique misty red olive 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 0 +Manufacturer#4 almond antique violet mint lemon 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 0 +Manufacturer#5 almond antique medium spring khaki 6 -25 +Manufacturer#5 almond antique sky peru orange 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 -23 +PREHOOK: query: -- 4. testLagInSum +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) as deltaSum +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 4. testLagInSum +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) as deltaSum +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 40 +Manufacturer#1 almond antique chartreuse lavender yellow 34 40 +Manufacturer#1 almond antique burnished rose metallic 2 40 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 40 +Manufacturer#1 almond aquamarine burnished black steel 28 40 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 40 +Manufacturer#2 almond antique violet chocolate turquoise 14 4 +Manufacturer#2 almond antique violet turquoise frosted 40 4 +Manufacturer#2 almond aquamarine midnight light salmon 2 4 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 4 +Manufacturer#3 almond antique metallic orange dim 19 26 +Manufacturer#3 almond antique chartreuse khaki white 17 26 +Manufacturer#3 almond antique forest lavender goldenrod 14 26 +Manufacturer#3 almond antique misty red olive 1 26 +Manufacturer#3 almond antique olive coral navajo 45 26 +Manufacturer#4 almond antique gainsboro frosted violet 10 2 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2 +Manufacturer#4 almond aquamarine yellow dodger mint 7 2 +Manufacturer#4 almond azure aquamarine papaya violet 12 2 +Manufacturer#5 almond antique blue firebrick mint 31 -8 +Manufacturer#5 almond antique medium spring khaki 6 -8 +Manufacturer#5 almond antique sky peru orange 2 -8 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 -8 +Manufacturer#5 almond azure blanched chiffon midnight 23 -8 +PREHOOK: query: -- 5. testLagInSumOverWindow +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) as deltaSum over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 5. testLagInSumOverWindow +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) as deltaSum over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4 +Manufacturer#1 almond antique burnished rose metallic 2 26 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 8 +Manufacturer#1 almond aquamarine burnished black steel 28 40 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 36 +Manufacturer#2 almond antique violet chocolate turquoise 14 -12 +Manufacturer#2 almond antique violet turquoise frosted 40 11 +Manufacturer#2 almond aquamarine midnight light salmon 2 4 +Manufacturer#2 almond aquamarine rose maroon antique 25 -22 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 16 +Manufacturer#3 almond antique metallic orange dim 19 -5 +Manufacturer#3 almond antique chartreuse khaki white 17 -18 +Manufacturer#3 almond antique forest lavender goldenrod 14 26 +Manufacturer#3 almond antique misty red olive 1 28 +Manufacturer#3 almond antique olive coral navajo 45 31 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 +Manufacturer#4 almond antique violet mint lemon 39 -3 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2 +Manufacturer#4 almond aquamarine yellow dodger mint 7 -27 +Manufacturer#4 almond azure aquamarine papaya violet 12 -15 +Manufacturer#5 almond antique blue firebrick mint 31 -29 +Manufacturer#5 almond antique medium spring khaki 6 15 +Manufacturer#5 almond antique sky peru orange 2 -8 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 17 +Manufacturer#5 almond azure blanched chiffon midnight 23 21 +PREHOOK: query: -- 6. testRankInLead +select p_mfgr, p_name, p_size, +rank() as r1, +lead(rank(), 1) as deltaRank +from part +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 6. testRankInLead +select p_mfgr, p_name, p_size, +rank() as r1, +lead(rank(), 1) as deltaRank +from part +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 4 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 5 +Manufacturer#1 almond aquamarine burnished black steel 28 5 6 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 2 +Manufacturer#2 almond antique violet turquoise frosted 40 2 3 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 4 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 5 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 2 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 3 +Manufacturer#3 almond antique metallic orange dim 19 3 4 +Manufacturer#3 almond antique misty red olive 1 4 5 +Manufacturer#3 almond antique olive coral navajo 45 5 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 2 +Manufacturer#4 almond antique violet mint lemon 39 2 3 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 4 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 5 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 2 +Manufacturer#5 almond antique medium spring khaki 6 2 3 +Manufacturer#5 almond antique sky peru orange 2 3 4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 5 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 +PREHOOK: query: -- 7. testLeadWithPTF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, p_size - lead(p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 7. testLeadWithPTF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, p_size - lead(p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1 1 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 -32 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 34 28 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 6 -22 +Manufacturer#1 almond aquamarine burnished black steel 5 4 28 -14 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 42 0 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 -26 +Manufacturer#2 almond antique violet turquoise frosted 2 2 40 38 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2 -23 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 25 7 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 18 0 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 3 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 14 -5 +Manufacturer#3 almond antique metallic orange dim 3 3 19 18 +Manufacturer#3 almond antique misty red olive 4 4 1 -44 +Manufacturer#3 almond antique olive coral navajo 5 5 45 0 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 -29 +Manufacturer#4 almond antique violet mint lemon 2 2 39 12 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 27 20 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 7 -5 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 12 0 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 25 +Manufacturer#5 almond antique medium spring khaki 2 2 6 4 +Manufacturer#5 almond antique sky peru orange 3 3 2 -44 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 46 23 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 23 0 +PREHOOK: query: -- 8. testOverNoPartitionMultipleAggregate +select p_name, p_retailprice, +lead(p_retailprice) as l1 over(), +lag(p_retailprice) as l2 over() +from part +order by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 8. testOverNoPartitionMultipleAggregate +select p_name, p_retailprice, +lead(p_retailprice) as l1 over(), +lag(p_retailprice) as l2 over() +from part +order by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +almond antique blue firebrick mint 1789.69 1611.66 1290.35 +almond antique burnished rose metallic 1173.15 1753.76 1173.15 +almond antique burnished rose metallic 1173.15 1173.15 NULL +almond antique chartreuse khaki white 1671.68 1190.27 1701.6 +almond antique chartreuse lavender yellow 1753.76 1602.59 1173.15 +almond antique forest lavender goldenrod 1190.27 1410.39 1671.68 +almond antique gainsboro frosted violet 1620.67 1375.42 1337.29 +almond antique medium spring khaki 1611.66 1788.73 1789.69 +almond antique metallic orange dim 1410.39 1922.98 1190.27 +almond antique misty red olive 1922.98 1337.29 1410.39 +almond antique olive coral navajo 1337.29 1620.67 1922.98 +almond antique salmon chartreuse burlywood 1602.59 1414.42 1753.76 +almond antique sky peru orange 1788.73 1018.1 1611.66 +almond antique violet chocolate turquoise 1690.68 1800.7 1632.66 +almond antique violet mint lemon 1375.42 1206.26 1620.67 +almond antique violet turquoise frosted 1800.7 2031.98 1690.68 +almond aquamarine burnished black steel 1414.42 1632.66 1602.59 +almond aquamarine dodger light gainsboro 1018.1 1464.48 1788.73 +almond aquamarine floral ivory bisque 1206.26 1844.92 1375.42 +almond aquamarine midnight light salmon 2031.98 1698.66 1800.7 +almond aquamarine pink moccasin thistle 1632.66 1690.68 1414.42 +almond aquamarine rose maroon antique 1698.66 1701.6 2031.98 +almond aquamarine sandy cyan gainsboro 1701.6 1671.68 1698.66 +almond aquamarine yellow dodger mint 1844.92 1290.35 1206.26 +almond azure aquamarine papaya violet 1290.35 1789.69 1844.92 +almond azure blanched chiffon midnight 1464.48 NULL 1018.1 diff --git ql/src/test/results/clientpositive/leadlag_queries.q.out ql/src/test/results/clientpositive/leadlag_queries.q.out new file mode 100644 index 0000000..380049b --- /dev/null +++ ql/src/test/results/clientpositive/leadlag_queries.q.out @@ -0,0 +1,261 @@ +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +PREHOOK: query: -- 1. testLeadUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice) as l1 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1) as l2 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1,10) as l3 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1, p_retailprice) as l4 over (partition by p_mfgr order by p_name), +p_retailprice - lead(p_retailprice,1) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 1. testLeadUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice) as l1 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1) as l2 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1,10) as l3 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1, p_retailprice) as l4 over (partition by p_mfgr order by p_name), +p_retailprice - lead(p_retailprice,1) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 1173.15 1173.15 1173.15 1173.15 1173.15 0.0 +Manufacturer#1 1173.15 1753.76 1753.76 1753.76 1753.76 -580.6099999999999 +Manufacturer#1 1753.76 1602.59 1602.59 1602.59 1602.59 151.17000000000007 +Manufacturer#1 1602.59 1414.42 1414.42 1414.42 1414.42 188.16999999999985 +Manufacturer#1 1414.42 1632.66 1632.66 1632.66 1632.66 -218.24 +Manufacturer#1 1632.66 NULL NULL 10.0 1632.66 0.0 +Manufacturer#2 1690.68 1800.7 1800.7 1800.7 1800.7 -110.01999999999998 +Manufacturer#2 1800.7 2031.98 2031.98 2031.98 2031.98 -231.27999999999997 +Manufacturer#2 2031.98 1698.66 1698.66 1698.66 1698.66 333.31999999999994 +Manufacturer#2 1698.66 1701.6 1701.6 1701.6 1701.6 -2.939999999999827 +Manufacturer#2 1701.6 NULL NULL 10.0 1701.6 0.0 +Manufacturer#3 1671.68 1190.27 1190.27 1190.27 1190.27 481.4100000000001 +Manufacturer#3 1190.27 1410.39 1410.39 1410.39 1410.39 -220.12000000000012 +Manufacturer#3 1410.39 1922.98 1922.98 1922.98 1922.98 -512.5899999999999 +Manufacturer#3 1922.98 1337.29 1337.29 1337.29 1337.29 585.69 +Manufacturer#3 1337.29 NULL NULL 10.0 1337.29 0.0 +Manufacturer#4 1620.67 1375.42 1375.42 1375.42 1375.42 245.25 +Manufacturer#4 1375.42 1206.26 1206.26 1206.26 1206.26 169.16000000000008 +Manufacturer#4 1206.26 1844.92 1844.92 1844.92 1844.92 -638.6600000000001 +Manufacturer#4 1844.92 1290.35 1290.35 1290.35 1290.35 554.5700000000002 +Manufacturer#4 1290.35 NULL NULL 10.0 1290.35 0.0 +Manufacturer#5 1789.69 1611.66 1611.66 1611.66 1611.66 178.02999999999997 +Manufacturer#5 1611.66 1788.73 1788.73 1788.73 1788.73 -177.06999999999994 +Manufacturer#5 1788.73 1018.1 1018.1 1018.1 1018.1 770.63 +Manufacturer#5 1018.1 1464.48 1464.48 1464.48 1464.48 -446.38 +Manufacturer#5 1464.48 NULL NULL 10.0 1464.48 0.0 +PREHOOK: query: -- 2.testLeadUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lead(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lead(p_retailprice,1) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 2.testLeadUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lead(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lead(p_retailprice,1) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1173.15 1173.15 0.0 +Manufacturer#1 almond antique burnished rose metallic 1173.15 NULL 0.0 +Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 NULL 0.0 +Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 NULL 0.0 +Manufacturer#1 almond aquamarine burnished black steel 1414.42 NULL 0.0 +Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 NULL 0.0 +Manufacturer#2 almond antique violet chocolate turquoise 1690.68 NULL 0.0 +Manufacturer#2 almond antique violet turquoise frosted 1800.7 NULL 0.0 +Manufacturer#2 almond aquamarine midnight light salmon 2031.98 NULL 0.0 +Manufacturer#2 almond aquamarine rose maroon antique 1698.66 NULL 0.0 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1701.6 NULL 0.0 +Manufacturer#3 almond antique chartreuse khaki white 1671.68 NULL 0.0 +Manufacturer#3 almond antique forest lavender goldenrod 1190.27 NULL 0.0 +Manufacturer#3 almond antique metallic orange dim 1410.39 NULL 0.0 +Manufacturer#3 almond antique misty red olive 1922.98 NULL 0.0 +Manufacturer#3 almond antique olive coral navajo 1337.29 NULL 0.0 +Manufacturer#4 almond antique gainsboro frosted violet 1620.67 NULL 0.0 +Manufacturer#4 almond antique violet mint lemon 1375.42 NULL 0.0 +Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 NULL 0.0 +Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 NULL 0.0 +Manufacturer#4 almond azure aquamarine papaya violet 1290.35 NULL 0.0 +Manufacturer#5 almond antique blue firebrick mint 1789.69 NULL 0.0 +Manufacturer#5 almond antique medium spring khaki 1611.66 NULL 0.0 +Manufacturer#5 almond antique sky peru orange 1788.73 NULL 0.0 +Manufacturer#5 almond aquamarine dodger light gainsboro 1018.1 NULL 0.0 +Manufacturer#5 almond azure blanched chiffon midnight 1464.48 NULL 0.0 +PREHOOK: query: -- 3.testLagUDAF +select p_mfgr, p_retailprice, +lag(p_retailprice,1) as l1 over (partition by p_mfgr order by p_name), +lag(p_retailprice) as l2 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1, p_retailprice) as l3 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1,10) as l4 over (partition by p_mfgr order by p_name), +p_retailprice - lag(p_retailprice,1) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 3.testLagUDAF +select p_mfgr, p_retailprice, +lag(p_retailprice,1) as l1 over (partition by p_mfgr order by p_name), +lag(p_retailprice) as l2 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1, p_retailprice) as l3 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1,10) as l4 over (partition by p_mfgr order by p_name), +p_retailprice - lag(p_retailprice,1) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 1173.15 NULL NULL 1173.15 10.0 0.0 +Manufacturer#1 1173.15 1173.15 1173.15 1173.15 1173.15 0.0 +Manufacturer#1 1753.76 1173.15 1173.15 1173.15 1173.15 580.6099999999999 +Manufacturer#1 1602.59 1753.76 1753.76 1753.76 1753.76 -151.17000000000007 +Manufacturer#1 1414.42 1602.59 1602.59 1602.59 1602.59 -188.16999999999985 +Manufacturer#1 1632.66 1414.42 1414.42 1414.42 1414.42 218.24 +Manufacturer#2 1690.68 NULL NULL 1690.68 10.0 0.0 +Manufacturer#2 1800.7 1690.68 1690.68 1690.68 1690.68 110.01999999999998 +Manufacturer#2 2031.98 1800.7 1800.7 1800.7 1800.7 231.27999999999997 +Manufacturer#2 1698.66 2031.98 2031.98 2031.98 2031.98 -333.31999999999994 +Manufacturer#2 1701.6 1698.66 1698.66 1698.66 1698.66 2.939999999999827 +Manufacturer#3 1671.68 NULL NULL 1671.68 10.0 0.0 +Manufacturer#3 1190.27 1671.68 1671.68 1671.68 1671.68 -481.4100000000001 +Manufacturer#3 1410.39 1190.27 1190.27 1190.27 1190.27 220.12000000000012 +Manufacturer#3 1922.98 1410.39 1410.39 1410.39 1410.39 512.5899999999999 +Manufacturer#3 1337.29 1922.98 1922.98 1922.98 1922.98 -585.69 +Manufacturer#4 1620.67 NULL NULL 1620.67 10.0 0.0 +Manufacturer#4 1375.42 1620.67 1620.67 1620.67 1620.67 -245.25 +Manufacturer#4 1206.26 1375.42 1375.42 1375.42 1375.42 -169.16000000000008 +Manufacturer#4 1844.92 1206.26 1206.26 1206.26 1206.26 638.6600000000001 +Manufacturer#4 1290.35 1844.92 1844.92 1844.92 1844.92 -554.5700000000002 +Manufacturer#5 1789.69 NULL NULL 1789.69 10.0 0.0 +Manufacturer#5 1611.66 1789.69 1789.69 1789.69 1789.69 -178.02999999999997 +Manufacturer#5 1788.73 1611.66 1611.66 1611.66 1611.66 177.06999999999994 +Manufacturer#5 1018.1 1788.73 1788.73 1788.73 1788.73 -770.63 +Manufacturer#5 1464.48 1018.1 1018.1 1018.1 1018.1 446.38 +PREHOOK: query: -- 4.testLagUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lag(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lag(p_retailprice,1) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 4.testLagUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lag(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lag(p_retailprice,1) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1173.15 NULL 0.0 +Manufacturer#1 almond antique burnished rose metallic 1173.15 1173.15 0.0 +Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 NULL 0.0 +Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 NULL 0.0 +Manufacturer#1 almond aquamarine burnished black steel 1414.42 NULL 0.0 +Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 NULL 0.0 +Manufacturer#2 almond antique violet chocolate turquoise 1690.68 NULL 0.0 +Manufacturer#2 almond antique violet turquoise frosted 1800.7 NULL 0.0 +Manufacturer#2 almond aquamarine midnight light salmon 2031.98 NULL 0.0 +Manufacturer#2 almond aquamarine rose maroon antique 1698.66 NULL 0.0 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1701.6 NULL 0.0 +Manufacturer#3 almond antique chartreuse khaki white 1671.68 NULL 0.0 +Manufacturer#3 almond antique forest lavender goldenrod 1190.27 NULL 0.0 +Manufacturer#3 almond antique metallic orange dim 1410.39 NULL 0.0 +Manufacturer#3 almond antique misty red olive 1922.98 NULL 0.0 +Manufacturer#3 almond antique olive coral navajo 1337.29 NULL 0.0 +Manufacturer#4 almond antique gainsboro frosted violet 1620.67 NULL 0.0 +Manufacturer#4 almond antique violet mint lemon 1375.42 NULL 0.0 +Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 NULL 0.0 +Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 NULL 0.0 +Manufacturer#4 almond azure aquamarine papaya violet 1290.35 NULL 0.0 +Manufacturer#5 almond antique blue firebrick mint 1789.69 NULL 0.0 +Manufacturer#5 almond antique medium spring khaki 1611.66 NULL 0.0 +Manufacturer#5 almond antique sky peru orange 1788.73 NULL 0.0 +Manufacturer#5 almond aquamarine dodger light gainsboro 1018.1 NULL 0.0 +Manufacturer#5 almond azure blanched chiffon midnight 1464.48 NULL 0.0 +PREHOOK: query: -- 5.testLeadLagUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice,1) as l1 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1, p_retailprice) as l2 over (partition by p_mfgr order by p_name), +p_retailprice - lead(p_retailprice,1), +lag(p_retailprice,1) as l3 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1, p_retailprice) as l4 over (partition by p_mfgr order by p_name) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 5.testLeadLagUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice,1) as l1 over (partition by p_mfgr order by p_name), +lead(p_retailprice,1, p_retailprice) as l2 over (partition by p_mfgr order by p_name), +p_retailprice - lead(p_retailprice,1), +lag(p_retailprice,1) as l3 over (partition by p_mfgr order by p_name), +lag(p_retailprice,1, p_retailprice) as l4 over (partition by p_mfgr order by p_name) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 1173.15 1173.15 1173.15 0.0 NULL 1173.15 +Manufacturer#1 1173.15 1753.76 1753.76 -580.6099999999999 1173.15 1173.15 +Manufacturer#1 1753.76 1602.59 1602.59 151.17000000000007 1173.15 1173.15 +Manufacturer#1 1602.59 1414.42 1414.42 188.16999999999985 1753.76 1753.76 +Manufacturer#1 1414.42 1632.66 1632.66 -218.24 1602.59 1602.59 +Manufacturer#1 1632.66 NULL 1632.66 0.0 1414.42 1414.42 +Manufacturer#2 1690.68 1800.7 1800.7 -110.01999999999998 NULL 1690.68 +Manufacturer#2 1800.7 2031.98 2031.98 -231.27999999999997 1690.68 1690.68 +Manufacturer#2 2031.98 1698.66 1698.66 333.31999999999994 1800.7 1800.7 +Manufacturer#2 1698.66 1701.6 1701.6 -2.939999999999827 2031.98 2031.98 +Manufacturer#2 1701.6 NULL 1701.6 0.0 1698.66 1698.66 +Manufacturer#3 1671.68 1190.27 1190.27 481.4100000000001 NULL 1671.68 +Manufacturer#3 1190.27 1410.39 1410.39 -220.12000000000012 1671.68 1671.68 +Manufacturer#3 1410.39 1922.98 1922.98 -512.5899999999999 1190.27 1190.27 +Manufacturer#3 1922.98 1337.29 1337.29 585.69 1410.39 1410.39 +Manufacturer#3 1337.29 NULL 1337.29 0.0 1922.98 1922.98 +Manufacturer#4 1620.67 1375.42 1375.42 245.25 NULL 1620.67 +Manufacturer#4 1375.42 1206.26 1206.26 169.16000000000008 1620.67 1620.67 +Manufacturer#4 1206.26 1844.92 1844.92 -638.6600000000001 1375.42 1375.42 +Manufacturer#4 1844.92 1290.35 1290.35 554.5700000000002 1206.26 1206.26 +Manufacturer#4 1290.35 NULL 1290.35 0.0 1844.92 1844.92 +Manufacturer#5 1789.69 1611.66 1611.66 178.02999999999997 NULL 1789.69 +Manufacturer#5 1611.66 1788.73 1788.73 -177.06999999999994 1789.69 1789.69 +Manufacturer#5 1788.73 1018.1 1018.1 770.63 1611.66 1611.66 +Manufacturer#5 1018.1 1464.48 1464.48 -446.38 1788.73 1788.73 +Manufacturer#5 1464.48 NULL 1464.48 0.0 1018.1 1018.1 diff --git ql/src/test/results/clientpositive/ptf.q.out ql/src/test/results/clientpositive/ptf.q.out new file mode 100644 index 0000000..4896139 --- /dev/null +++ ql/src/test/results/clientpositive/ptf.q.out @@ -0,0 +1,1483 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +PREHOOK: query: --1. test1 +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part + partition by p_mfgr + order by p_name + ) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: --1. test1 +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part + partition by p_mfgr + order by p_name + ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: -- 2. testJoinWithNoop +select p_mfgr, p_name, +p_size, p_size - lag(p_size,1) as deltaSz +from noop (on (select p1.* from part p1 join part p2 on p1.p_partkey = p2.p_partkey) j +distribute by j.p_mfgr +sort by j.p_name) +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 2. testJoinWithNoop +select p_mfgr, p_name, +p_size, p_size - lag(p_size,1) as deltaSz +from noop (on (select p1.* from part p1 join part p2 on p1.p_partkey = p2.p_partkey) j +distribute by j.p_mfgr +sort by j.p_name) +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 5 +Manufacturer#3 almond antique misty red olive 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 0 +Manufacturer#4 almond antique violet mint lemon 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 0 +Manufacturer#5 almond antique medium spring khaki 6 -25 +Manufacturer#5 almond antique sky peru orange 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 -23 +PREHOOK: query: -- 3. testOnlyPTF +select p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 3. testOnlyPTF +select p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 +Manufacturer#3 almond antique metallic orange dim 19 +Manufacturer#3 almond antique misty red olive 1 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond antique violet mint lemon 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#5 almond antique blue firebrick mint 31 +Manufacturer#5 almond antique medium spring khaki 6 +Manufacturer#5 almond antique sky peru orange 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 +PREHOOK: query: -- 4. testPTFAlias +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part + partition by p_mfgr + order by p_name + ) abc +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 4. testPTFAlias +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part + partition by p_mfgr + order by p_name + ) abc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: -- 5. testPTFAndWhereWithWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +having p_size > 0 +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 5. testPTFAndWhereWithWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +having p_size > 0 +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2 0 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 23 -23 +PREHOOK: query: -- 6. testSWQAndPTFAndGBy +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +group by p_mfgr, p_name, p_size +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 6. testSWQAndPTFAndGBy +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +group by p_mfgr, p_name, p_size +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 23 -23 +PREHOOK: query: -- 7. testJoin +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 7. testJoin +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +PREHOOK: query: -- 8. testJoinRight +select abc.* +from part p1 join noop(on part +partition by p_mfgr +order by p_name +) abc on abc.p_partkey = p1.p_partkey +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 8. testJoinRight +select abc.* +from part p1 join noop(on part +partition by p_mfgr +order by p_name +) abc on abc.p_partkey = p1.p_partkey +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu +17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the +17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve +33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful +40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s +42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl +45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful +48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i +49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick +65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr +78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith +85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull +86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully +90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl +105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ +110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously +112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h +132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even +144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about +146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref +155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra +191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle +192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir +195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de +PREHOOK: query: -- 9. testNoopWithMap +select p_mfgr, p_name, p_size, +rank() as r over (partition by p_mfgr order by p_name, p_size desc) +from noopwithmap(on part +partition by p_mfgr +order by p_name, p_size desc) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 9. testNoopWithMap +select p_mfgr, p_name, p_size, +rank() as r over (partition by p_mfgr order by p_name, p_size desc) +from noopwithmap(on part +partition by p_mfgr +order by p_name, p_size desc) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 +PREHOOK: query: -- 10. testNoopWithMapWithWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noopwithmap(on part + partition by p_mfgr + order by p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 10. testNoopWithMapWithWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noopwithmap(on part + partition by p_mfgr + order by p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: -- 11. testHavingWithWindowingPTFNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part +partition by p_mfgr +order by p_name) +having rank() < 4 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 11. testHavingWithWindowingPTFNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part +partition by p_mfgr +order by p_name) +having rank() < 4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +PREHOOK: query: -- 12. testFunctionChain +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on noopwithmap(on noop(on part +partition by p_mfgr +order by p_mfgr, p_name +))) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 12. testFunctionChain +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on noopwithmap(on noop(on part +partition by p_mfgr +order by p_mfgr, p_name +))) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: -- 13. testPTFAndWindowingInSubQ +select p_mfgr, p_name, +sub1.cd, sub1.s1 +from (select p_mfgr, p_name, +count(p_size) as cd, +p_retailprice, +sum(p_retailprice) as s1 over w1 +from noop(on part +partition by p_mfgr +order by p_name) +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +) sub1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 13. testPTFAndWindowingInSubQ +select p_mfgr, p_name, +sub1.cd, sub1.s1 +from (select p_mfgr, p_name, +count(p_size) as cd, +p_retailprice, +sum(p_retailprice) as s1 over w1 +from noop(on part +partition by p_mfgr +order by p_name) +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +) sub1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 6 4100.06 +Manufacturer#1 almond antique burnished rose metallic 6 5702.650000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 6 7117.070000000001 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 7576.58 +Manufacturer#1 almond aquamarine burnished black steel 6 6403.43 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 4649.67 +Manufacturer#2 almond antique violet chocolate turquoise 5 5523.360000000001 +Manufacturer#2 almond antique violet turquoise frosted 5 7222.02 +Manufacturer#2 almond aquamarine midnight light salmon 5 8923.62 +Manufacturer#2 almond aquamarine rose maroon antique 5 7232.9400000000005 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5432.24 +Manufacturer#3 almond antique chartreuse khaki white 5 4272.34 +Manufacturer#3 almond antique forest lavender goldenrod 5 6195.32 +Manufacturer#3 almond antique metallic orange dim 5 7532.61 +Manufacturer#3 almond antique misty red olive 5 5860.929999999999 +Manufacturer#3 almond antique olive coral navajo 5 4670.66 +Manufacturer#4 almond antique gainsboro frosted violet 5 4202.35 +Manufacturer#4 almond antique violet mint lemon 5 6047.27 +Manufacturer#4 almond aquamarine floral ivory bisque 5 7337.620000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 5 5716.950000000001 +Manufacturer#4 almond azure aquamarine papaya violet 5 4341.530000000001 +Manufacturer#5 almond antique blue firebrick mint 5 5190.08 +Manufacturer#5 almond antique medium spring khaki 5 6208.18 +Manufacturer#5 almond antique sky peru orange 5 7672.66 +Manufacturer#5 almond aquamarine dodger light gainsboro 5 5882.970000000001 +Manufacturer#5 almond azure blanched chiffon midnight 5 4271.3099999999995 +PREHOOK: query: -- 14. testPTFJoinWithWindowingWithCount +select abc.p_mfgr, abc.p_name, +rank() as r, +dense_rank() as dr, +count(abc.p_name) as cd, +abc.p_retailprice, sum(abc.p_retailprice) as s1 over (rows between unbounded preceding and current row), +abc.p_size, abc.p_size - lag(abc.p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +distribute by abc.p_mfgr +sort by abc.p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 14. testPTFJoinWithWindowingWithCount +select abc.p_mfgr, abc.p_name, +rank() as r, +dense_rank() as dr, +count(abc.p_name) as cd, +abc.p_retailprice, sum(abc.p_retailprice) as s1 over (rows between unbounded preceding and current row), +abc.p_size, abc.p_size - lag(abc.p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +distribute by abc.p_mfgr +sort by abc.p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1 1 8 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 8 1173.15 2346.3 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 8 1173.15 3519.4500000000003 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 8 1173.15 4692.6 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 5 2 8 1753.76 6446.360000000001 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 8 1602.59 8048.950000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 7 4 8 1414.42 9463.37 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 8 5 8 1632.66 11096.03 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 5 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 5 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 5 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 5 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 5 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 5 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 5 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 5 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 5 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 5 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 5 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 5 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 5 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 5 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 5 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 5 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 5 1464.48 7672.66 23 -23 +PREHOOK: query: -- 15. testDistinctInSelectWithPTF +select DISTINCT p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 15. testDistinctInSelectWithPTF +select DISTINCT p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#1 almond aquamarine burnished black steel 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 +Manufacturer#2 almond antique violet turquoise frosted 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 +Manufacturer#3 almond antique chartreuse khaki white 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 +Manufacturer#3 almond antique metallic orange dim 19 +Manufacturer#3 almond antique misty red olive 1 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 +Manufacturer#4 almond antique violet mint lemon 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#5 almond antique blue firebrick mint 31 +Manufacturer#5 almond antique medium spring khaki 6 +Manufacturer#5 almond antique sky peru orange 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 +PREHOOK: query: -- 16. testViewAsTableInputToPTF +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand +PREHOOK: type: CREATEVIEW +#### A masked pattern was here #### +POSTHOOK: query: -- 16. testViewAsTableInputToPTF +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand +POSTHOOK: type: CREATEVIEW +POSTHOOK: Output: default@mfgr_price_view +#### A masked pattern was here #### +PREHOOK: query: select p_mfgr, p_brand, s, +sum(s) as s1 over w1 +from noop(on mfgr_price_view +partition by p_mfgr +order by p_mfgr) +window w1 as ( partition by p_mfgr order by p_brand rows between 2 preceding and current row) +PREHOOK: type: QUERY +PREHOOK: Input: default@mfgr_price_view +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_mfgr, p_brand, s, +sum(s) as s1 over w1 +from noop(on mfgr_price_view +partition by p_mfgr +order by p_mfgr) +window w1 as ( partition by p_mfgr order by p_brand rows between 2 preceding and current row) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@mfgr_price_view +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 Brand#12 4800.84 4800.84 +Manufacturer#1 Brand#14 2346.3 7147.14 +Manufacturer#1 Brand#15 1602.59 8749.73 +Manufacturer#2 Brand#22 3491.38 3491.38 +Manufacturer#2 Brand#23 2031.98 5523.360000000001 +Manufacturer#2 Brand#24 1698.66 7222.02 +Manufacturer#2 Brand#25 1701.6 5432.24 +Manufacturer#3 Brand#31 1671.68 1671.68 +Manufacturer#3 Brand#32 3333.37 5005.05 +Manufacturer#3 Brand#34 1337.29 6342.34 +Manufacturer#3 Brand#35 1190.27 5860.93 +Manufacturer#4 Brand#41 4755.9400000000005 4755.9400000000005 +Manufacturer#4 Brand#42 2581.6800000000003 7337.620000000001 +Manufacturer#5 Brand#51 1611.66 1611.66 +Manufacturer#5 Brand#52 3254.17 4865.83 +Manufacturer#5 Brand#53 2806.83 7672.66 +PREHOOK: query: -- 17. testMultipleInserts2SWQsWithPTF +CREATE TABLE part_4( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- 17. testMultipleInserts2SWQsWithPTF +CREATE TABLE part_4( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_4 +PREHOOK: query: CREATE TABLE part_5( +p_mfgr STRING, +p_name STRING, +p_size INT, +s1 INT, +s2 INT, +r INT, +dr INT, +cud DOUBLE, +fv1 INT) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part_5( +p_mfgr STRING, +p_name STRING, +p_size INT, +s1 INT, +s2 INT, +r INT, +dr INT, +cud DOUBLE, +fv1 INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_5 +PREHOOK: query: from noop(on part +partition by p_mfgr +order by p_name) +INSERT OVERWRITE TABLE part_4 select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s over (rows between unbounded preceding and current row) +distribute by p_mfgr +sort by p_name +INSERT OVERWRITE TABLE part_5 select p_mfgr,p_name, p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +rank() as r, +dense_rank() as dr, +cume_dist() as cud, +first_value(p_size, true) as fv1 over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Output: default@part_4 +PREHOOK: Output: default@part_5 +POSTHOOK: query: from noop(on part +partition by p_mfgr +order by p_name) +INSERT OVERWRITE TABLE part_4 select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s over (rows between unbounded preceding and current row) +distribute by p_mfgr +sort by p_name +INSERT OVERWRITE TABLE part_5 select p_mfgr,p_name, p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +rank() as r, +dense_rank() as dr, +cume_dist() as cud, +first_value(p_size, true) as fv1 over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Output: default@part_4 +POSTHOOK: Output: default@part_5 +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +PREHOOK: query: select * from part_4 +PREHOOK: type: QUERY +PREHOOK: Input: default@part_4 +#### A masked pattern was here #### +POSTHOOK: query: select * from part_4 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_4 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: select * from part_5 +PREHOOK: type: QUERY +PREHOOK: Input: default@part_5 +#### A masked pattern was here #### +POSTHOOK: query: select * from part_5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_5 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique chartreuse lavender yellow 34 38 34 3 2 0.5 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 44 44 4 3 0.6666666666666666 2 +Manufacturer#1 almond aquamarine burnished black steel 28 72 28 5 4 0.8333333333333334 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 114 42 6 5 1.0 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 1 1 0.2 14 +Manufacturer#2 almond antique violet turquoise frosted 40 54 40 2 2 0.4 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 81 25 4 4 0.8 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 99 43 5 5 1.0 2 +Manufacturer#3 almond antique chartreuse khaki white 17 17 17 1 1 0.2 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 31 31 2 2 0.4 17 +Manufacturer#3 almond antique metallic orange dim 19 50 19 3 3 0.6 17 +Manufacturer#3 almond antique olive coral navajo 45 96 45 5 5 1.0 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 10 10 1 1 0.2 10 +Manufacturer#4 almond antique violet mint lemon 39 49 39 2 2 0.4 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 76 66 3 3 0.6 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 83 83 4 4 0.8 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 95 12 5 5 1.0 27 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 1 1 0.2 31 +Manufacturer#5 almond antique medium spring khaki 6 37 37 2 2 0.4 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 85 46 4 4 0.8 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 108 69 5 5 1.0 2 +PREHOOK: query: -- 18. testMulti2OperatorsFunctionChainWithMap +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr,p_name rows between unbounded preceding and current row) +from noop(on + noopwithmap(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr,p_name + order by p_mfgr,p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 18. testMulti2OperatorsFunctionChainWithMap +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr,p_name rows between unbounded preceding and current row) +from noop(on + noopwithmap(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr,p_name + order by p_mfgr,p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 1 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 4 +Manufacturer#1 almond antique chartreuse lavender yellow 1 1 34 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 1 6 6 +Manufacturer#1 almond aquamarine burnished black steel 1 1 28 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 1 42 42 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 14 +Manufacturer#2 almond antique violet turquoise frosted 1 1 40 40 +Manufacturer#2 almond aquamarine midnight light salmon 1 1 2 2 +Manufacturer#2 almond aquamarine rose maroon antique 1 1 25 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 1 18 18 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 1 1 14 14 +Manufacturer#3 almond antique metallic orange dim 1 1 19 19 +Manufacturer#3 almond antique misty red olive 1 1 1 1 +Manufacturer#3 almond antique olive coral navajo 1 1 45 45 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 10 +Manufacturer#4 almond antique violet mint lemon 1 1 39 39 +Manufacturer#4 almond aquamarine floral ivory bisque 1 1 27 27 +Manufacturer#4 almond aquamarine yellow dodger mint 1 1 7 7 +Manufacturer#4 almond azure aquamarine papaya violet 1 1 12 12 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 31 +Manufacturer#5 almond antique medium spring khaki 1 1 6 6 +Manufacturer#5 almond antique sky peru orange 1 1 2 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 1 46 46 +Manufacturer#5 almond azure blanched chiffon midnight 1 1 23 23 +PREHOOK: query: -- 19. testMulti3OperatorsFunctionChain +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr + order by p_mfgr ) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 19. testMulti3OperatorsFunctionChain +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr + order by p_mfgr ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 1 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 4 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 34 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 6 44 +Manufacturer#1 almond aquamarine burnished black steel 5 4 28 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 42 114 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 14 +Manufacturer#2 almond antique violet turquoise frosted 2 2 40 54 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2 56 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 25 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 18 99 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 14 31 +Manufacturer#3 almond antique metallic orange dim 3 3 19 50 +Manufacturer#3 almond antique misty red olive 4 4 1 51 +Manufacturer#3 almond antique olive coral navajo 5 5 45 96 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 10 +Manufacturer#4 almond antique violet mint lemon 2 2 39 49 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 27 76 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 7 83 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 12 95 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 31 +Manufacturer#5 almond antique medium spring khaki 2 2 6 37 +Manufacturer#5 almond antique sky peru orange 3 3 2 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 46 85 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 23 108 +PREHOOK: query: -- 20. testMultiOperatorChainWithNoWindowing +select p_mfgr, p_name, +rank() as r over (partition by p_mfgr order by p_name), +dense_rank() as dr, +p_size, sum(p_size) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr)) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 20. testMultiOperatorChainWithNoWindowing +select p_mfgr, p_name, +rank() as r over (partition by p_mfgr order by p_name), +dense_rank() as dr, +p_size, sum(p_size) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 1 1 2 114 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 114 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 34 114 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 6 114 +Manufacturer#1 almond aquamarine burnished black steel 5 4 28 114 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 42 114 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 99 +Manufacturer#2 almond antique violet turquoise frosted 2 2 40 99 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 25 99 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 18 99 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 96 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 14 96 +Manufacturer#3 almond antique metallic orange dim 3 3 19 96 +Manufacturer#3 almond antique misty red olive 4 4 1 96 +Manufacturer#3 almond antique olive coral navajo 5 5 45 96 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 95 +Manufacturer#4 almond antique violet mint lemon 2 2 39 95 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 7 95 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 12 95 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 108 +Manufacturer#5 almond antique medium spring khaki 2 2 6 108 +Manufacturer#5 almond antique sky peru orange 3 3 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 46 108 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 23 108 +PREHOOK: query: -- 21. testMultiOperatorChainEndsWithNoopMap +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr,p_name rows between unbounded preceding and current row) +from noopwithmap(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr) + partition by p_mfgr,p_name + order by p_mfgr,p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 21. testMultiOperatorChainEndsWithNoopMap +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, sum(p_size) as s1 over (partition by p_mfgr,p_name rows between unbounded preceding and current row) +from noopwithmap(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr) + partition by p_mfgr,p_name + order by p_mfgr,p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 1 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 4 +Manufacturer#1 almond antique chartreuse lavender yellow 1 1 34 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 1 6 6 +Manufacturer#1 almond aquamarine burnished black steel 1 1 28 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 1 42 42 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 14 +Manufacturer#2 almond antique violet turquoise frosted 1 1 40 40 +Manufacturer#2 almond aquamarine midnight light salmon 1 1 2 2 +Manufacturer#2 almond aquamarine rose maroon antique 1 1 25 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 1 18 18 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 1 1 14 14 +Manufacturer#3 almond antique metallic orange dim 1 1 19 19 +Manufacturer#3 almond antique misty red olive 1 1 1 1 +Manufacturer#3 almond antique olive coral navajo 1 1 45 45 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 10 +Manufacturer#4 almond antique violet mint lemon 1 1 39 39 +Manufacturer#4 almond aquamarine floral ivory bisque 1 1 27 27 +Manufacturer#4 almond aquamarine yellow dodger mint 1 1 7 7 +Manufacturer#4 almond azure aquamarine papaya violet 1 1 12 12 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 31 +Manufacturer#5 almond antique medium spring khaki 1 1 6 6 +Manufacturer#5 almond antique sky peru orange 1 1 2 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 1 46 46 +Manufacturer#5 almond azure blanched chiffon midnight 1 1 23 23 +PREHOOK: query: -- 22. testMultiOperatorChainWithDiffPartitionForWindow1 +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (partition by p_mfgr,p_name order by p_mfgr,p_name rows between unbounded preceding and current row) +from noop(on + noopwithmap(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + partition by p_mfgr + order by p_mfgr + )) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 22. testMultiOperatorChainWithDiffPartitionForWindow1 +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (partition by p_mfgr,p_name order by p_mfgr,p_name rows between unbounded preceding and current row) +from noop(on + noopwithmap(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + partition by p_mfgr + order by p_mfgr + )) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 1 1 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 4 4 +Manufacturer#1 almond antique chartreuse lavender yellow 1 1 34 34 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 1 6 6 6 +Manufacturer#1 almond aquamarine burnished black steel 1 1 28 28 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 1 42 42 42 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 1 1 40 40 40 +Manufacturer#2 almond aquamarine midnight light salmon 1 1 2 2 2 +Manufacturer#2 almond aquamarine rose maroon antique 1 1 25 25 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 1 18 18 18 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 1 1 14 14 14 +Manufacturer#3 almond antique metallic orange dim 1 1 19 19 19 +Manufacturer#3 almond antique misty red olive 1 1 1 1 1 +Manufacturer#3 almond antique olive coral navajo 1 1 45 45 45 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 10 10 +Manufacturer#4 almond antique violet mint lemon 1 1 39 39 39 +Manufacturer#4 almond aquamarine floral ivory bisque 1 1 27 27 27 +Manufacturer#4 almond aquamarine yellow dodger mint 1 1 7 7 7 +Manufacturer#4 almond azure aquamarine papaya violet 1 1 12 12 12 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 31 31 +Manufacturer#5 almond antique medium spring khaki 1 1 6 6 6 +Manufacturer#5 almond antique sky peru orange 1 1 2 2 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 1 46 46 46 +Manufacturer#5 almond azure blanched chiffon midnight 1 1 23 23 23 +PREHOOK: query: -- 23. testMultiOperatorChainWithDiffPartitionForWindow2 +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (partition by p_mfgr order by p_mfgr rows between unbounded preceding and current row) +from noopwithmap(on + noop(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + )) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 23. testMultiOperatorChainWithDiffPartitionForWindow2 +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +p_size, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (partition by p_mfgr order by p_mfgr rows between unbounded preceding and current row) +from noopwithmap(on + noop(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + )) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_4.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_4.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +POSTHOOK: Lineage: part_5.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), ] +Manufacturer#1 almond antique burnished rose metallic 1 1 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 1 1 34 36 36 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 38 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 1 6 44 44 +Manufacturer#1 almond aquamarine burnished black steel 1 1 28 72 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 1 42 114 114 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 1 1 40 54 54 +Manufacturer#2 almond aquamarine midnight light salmon 1 1 2 56 56 +Manufacturer#2 almond aquamarine rose maroon antique 1 1 25 81 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 1 18 99 99 +Manufacturer#3 almond antique metallic orange dim 1 1 19 19 19 +Manufacturer#3 almond antique chartreuse khaki white 1 1 17 36 36 +Manufacturer#3 almond antique forest lavender goldenrod 1 1 14 50 50 +Manufacturer#3 almond antique misty red olive 1 1 1 51 51 +Manufacturer#3 almond antique olive coral navajo 1 1 45 96 96 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 10 10 10 +Manufacturer#4 almond antique violet mint lemon 1 1 39 49 49 +Manufacturer#4 almond aquamarine floral ivory bisque 1 1 27 76 76 +Manufacturer#4 almond aquamarine yellow dodger mint 1 1 7 83 83 +Manufacturer#4 almond azure aquamarine papaya violet 1 1 12 95 95 +Manufacturer#5 almond antique blue firebrick mint 1 1 31 31 31 +Manufacturer#5 almond antique medium spring khaki 1 1 6 37 37 +Manufacturer#5 almond antique sky peru orange 1 1 2 39 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 1 46 85 85 +Manufacturer#5 almond azure blanched chiffon midnight 1 1 23 108 108 diff --git ql/src/test/results/clientpositive/ptf_general_queries.q.out ql/src/test/results/clientpositive/ptf_general_queries.q.out new file mode 100644 index 0000000..c77dfc9 --- /dev/null +++ ql/src/test/results/clientpositive/ptf_general_queries.q.out @@ -0,0 +1,140 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +PREHOOK: query: -- 1. testNoPTFNoWindowing +select p_mfgr, p_name, p_size +from part +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 1. testNoPTFNoWindowing +select p_mfgr, p_name, p_size +from part +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#5 almond antique blue firebrick mint 31 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +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 misty red olive 1 +Manufacturer#3 almond antique olive coral navajo 45 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#5 almond antique sky peru orange 2 +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#2 almond aquamarine midnight light salmon 2 +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 aquamarine yellow dodger mint 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 +Manufacturer#5 almond azure blanched chiffon midnight 23 +PREHOOK: query: -- 2. testUDAFsNoWindowingNoPTFNoGBY +select p_mfgr,p_name, p_retailprice, +sum(p_retailprice) as s, +min(p_retailprice) as mi, +max(p_retailprice) as ma, +avg(p_retailprice) as av +from part +distribute by p_mfgr +sort by p_mfgr, p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 2. testUDAFsNoWindowingNoPTFNoGBY +select p_mfgr,p_name, p_retailprice, +sum(p_retailprice) as s, +min(p_retailprice) as mi, +max(p_retailprice) as ma, +avg(p_retailprice) as av +from part +distribute by p_mfgr +sort by p_mfgr, p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1173.15 8749.730000000001 1173.15 1753.76 1458.2883333333336 +Manufacturer#1 almond antique burnished rose metallic 1173.15 8749.730000000001 1173.15 1753.76 1458.2883333333336 +Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 8749.730000000001 1173.15 1753.76 1458.2883333333336 +Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 8749.730000000001 1173.15 1753.76 1458.2883333333336 +Manufacturer#1 almond aquamarine burnished black steel 1414.42 8749.730000000001 1173.15 1753.76 1458.2883333333336 +Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 8749.730000000001 1173.15 1753.76 1458.2883333333336 +Manufacturer#2 almond antique violet chocolate turquoise 1690.68 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond antique violet turquoise frosted 1800.7 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine midnight light salmon 2031.98 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 1698.66 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1701.6 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#3 almond antique chartreuse khaki white 1671.68 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique forest lavender goldenrod 1190.27 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique metallic orange dim 1410.39 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique misty red olive 1922.98 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique olive coral navajo 1337.29 7532.61 1190.27 1922.98 1506.522 +Manufacturer#4 almond antique gainsboro frosted violet 1620.67 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond antique violet mint lemon 1375.42 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond azure aquamarine papaya violet 1290.35 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#5 almond antique blue firebrick mint 1789.69 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond antique medium spring khaki 1611.66 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond antique sky peru orange 1788.73 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 1018.1 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond azure blanched chiffon midnight 1464.48 7672.66 1018.1 1789.69 1534.532 +PREHOOK: query: -- 3. testConstExprInSelect +select 'tst1' as key, count(1) as value from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 3. testConstExprInSelect +select 'tst1' as key, count(1) as value from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +tst1 26 diff --git ql/src/test/results/clientpositive/ptf_npath.q.out ql/src/test/results/clientpositive/ptf_npath.q.out new file mode 100644 index 0000000..333a621 --- /dev/null +++ ql/src/test/results/clientpositive/ptf_npath.q.out @@ -0,0 +1,73 @@ +PREHOOK: query: DROP TABLE flights_tiny +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE flights_tiny +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table flights_tiny ( +ORIGIN_CITY_NAME string, +DEST_CITY_NAME string, +YEAR int, +MONTH int, +DAY_OF_MONTH int, +ARR_DELAY float, +FL_NUM string +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table flights_tiny ( +ORIGIN_CITY_NAME string, +DEST_CITY_NAME string, +YEAR int, +MONTH int, +DAY_OF_MONTH int, +ARR_DELAY float, +FL_NUM string +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@flights_tiny +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny +PREHOOK: type: LOAD +PREHOOK: Output: default@flights_tiny +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny +POSTHOOK: type: LOAD +POSTHOOK: Output: default@flights_tiny +PREHOOK: query: -- 1. basic Npath test +select origin_city_name, fl_num, year, month, day_of_month, sz, tpath +from npath(on + flights_tiny + distribute by fl_num + sort by year, month, day_of_month + arg1('LATE.LATE+'), + arg2('LATE'), arg3(arr_delay > 15), + arg4('origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath as tpath') + ) +PREHOOK: type: QUERY +PREHOOK: Input: default@flights_tiny +#### A masked pattern was here #### +POSTHOOK: query: -- 1. basic Npath test +select origin_city_name, fl_num, year, month, day_of_month, sz, tpath +from npath(on + flights_tiny + distribute by fl_num + sort by year, month, day_of_month + arg1('LATE.LATE+'), + arg2('LATE'), arg3(arr_delay > 15), + arg4('origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath as tpath') + ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@flights_tiny +#### A masked pattern was here #### +Baltimore 1142 2010 10 20 6 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":20,"arr_delay":23.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":105.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":54.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":25,"arr_delay":92.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":123.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":49.0,"fl_num":"1142"}] +Baltimore 1142 2010 10 21 5 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":105.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":54.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":25,"arr_delay":92.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":123.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":49.0,"fl_num":"1142"}] +Baltimore 1142 2010 10 22 4 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":54.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":25,"arr_delay":92.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":123.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":49.0,"fl_num":"1142"}] +Baltimore 1142 2010 10 25 3 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":25,"arr_delay":92.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":123.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":49.0,"fl_num":"1142"}] +Baltimore 1142 2010 10 26 2 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":123.0,"fl_num":"1142"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":49.0,"fl_num":"1142"}] +Chicago 1531 2010 10 21 2 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":53.0,"fl_num":"1531"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":16.0,"fl_num":"1531"}] +Chicago 1531 2010 10 25 3 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":25,"arr_delay":43.0,"fl_num":"1531"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":29.0,"fl_num":"1531"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":70.0,"fl_num":"1531"}] +Chicago 1531 2010 10 26 2 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":29.0,"fl_num":"1531"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":70.0,"fl_num":"1531"}] +Baltimore 1599 2010 10 21 2 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":28.0,"fl_num":"1599"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":18.0,"fl_num":"1599"}] +Baltimore 1599 2010 10 25 3 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":25,"arr_delay":106.0,"fl_num":"1599"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":90.0,"fl_num":"1599"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":92.0,"fl_num":"1599"}] +Baltimore 1599 2010 10 26 2 [{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":26,"arr_delay":90.0,"fl_num":"1599"},{"origin_city_name":"Baltimore","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":92.0,"fl_num":"1599"}] +Chicago 361 2010 10 20 2 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":20,"arr_delay":42.0,"fl_num":"361"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":142.0,"fl_num":"361"}] +Washington 7291 2010 10 27 2 [{"origin_city_name":"Washington","dest_city_name":"New York","year":2010,"month":10,"day_of_month":27,"arr_delay":26.0,"fl_num":"7291"},{"origin_city_name":"Washington","dest_city_name":"New York","year":2010,"month":10,"day_of_month":28,"arr_delay":45.0,"fl_num":"7291"}] +Chicago 897 2010 10 20 4 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":20,"arr_delay":24.0,"fl_num":"897"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":77.0,"fl_num":"897"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":24.0,"fl_num":"897"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":24,"arr_delay":113.0,"fl_num":"897"}] +Chicago 897 2010 10 21 3 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":21,"arr_delay":77.0,"fl_num":"897"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":24.0,"fl_num":"897"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":24,"arr_delay":113.0,"fl_num":"897"}] +Chicago 897 2010 10 22 2 [{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":22,"arr_delay":24.0,"fl_num":"897"},{"origin_city_name":"Chicago","dest_city_name":"New York","year":2010,"month":10,"day_of_month":24,"arr_delay":113.0,"fl_num":"897"}] diff --git ql/src/test/results/clientpositive/ptf_rcfile.q.out ql/src/test/results/clientpositive/ptf_rcfile.q.out new file mode 100644 index 0000000..6f280b9 --- /dev/null +++ ql/src/test/results/clientpositive/ptf_rcfile.q.out @@ -0,0 +1,83 @@ +PREHOOK: query: DROP TABLE part_rc +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part_rc +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part_rc( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) STORED AS RCFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part_rc( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) STORED AS RCFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_rc +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc +PREHOOK: type: LOAD +PREHOOK: Output: default@part_rc +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part_rc +PREHOOK: query: -- testWindowingPTFWithPartRC +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part_rc +partition by p_mfgr +order by p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_rc +#### A masked pattern was here #### +POSTHOOK: query: -- testWindowingPTFWithPartRC +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part_rc +partition by p_mfgr +order by p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_rc +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git ql/src/test/results/clientpositive/ptf_seqfile.q.out ql/src/test/results/clientpositive/ptf_seqfile.q.out new file mode 100644 index 0000000..00b91c3 --- /dev/null +++ ql/src/test/results/clientpositive/ptf_seqfile.q.out @@ -0,0 +1,83 @@ +PREHOOK: query: DROP TABLE part_seq +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part_seq +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE part_seq( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) STORED AS SEQUENCEFILE +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part_seq( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) STORED AS SEQUENCEFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_seq +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq +PREHOOK: type: LOAD +PREHOOK: Output: default@part_seq +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part_seq +PREHOOK: query: -- testWindowingPTFWithPartSeqFile +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part_seq +partition by p_mfgr +order by p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_seq +#### A masked pattern was here #### +POSTHOOK: query: -- testWindowingPTFWithPartSeqFile +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) +from noop(on part_seq +partition by p_mfgr +order by p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_seq +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git ql/src/test/results/clientpositive/windowing.q.out ql/src/test/results/clientpositive/windowing.q.out new file mode 100644 index 0000000..4ae2eec --- /dev/null +++ ql/src/test/results/clientpositive/windowing.q.out @@ -0,0 +1,2826 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +PREHOOK: query: -- 1. testWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 1. testWindowing +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: -- 2. testGroupByWithPartitioning +select p_mfgr, p_name, p_size, min(p_retailprice), +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 2. testGroupByWithPartitioning +select p_mfgr, p_name, p_size, min(p_retailprice), +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 +PREHOOK: query: -- 3. testGroupByHavingWithSWQ +select p_mfgr, p_name, p_size, min(p_retailprice), +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +having p_size > 0 +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 3. testGroupByHavingWithSWQ +select p_mfgr, p_name, p_size, min(p_retailprice), +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +having p_size > 0 +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 +PREHOOK: query: -- 4. testCount +select p_mfgr, p_name, +count(p_size) as cd +from part +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 4. testCount +select p_mfgr, p_name, +count(p_size) as cd +from part +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 6 +Manufacturer#1 almond antique burnished rose metallic 6 +Manufacturer#1 almond antique chartreuse lavender yellow 6 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 +Manufacturer#1 almond aquamarine burnished black steel 6 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 +Manufacturer#2 almond antique violet chocolate turquoise 5 +Manufacturer#2 almond antique violet turquoise frosted 5 +Manufacturer#2 almond aquamarine midnight light salmon 5 +Manufacturer#2 almond aquamarine rose maroon antique 5 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 +Manufacturer#3 almond antique chartreuse khaki white 5 +Manufacturer#3 almond antique forest lavender goldenrod 5 +Manufacturer#3 almond antique metallic orange dim 5 +Manufacturer#3 almond antique misty red olive 5 +Manufacturer#3 almond antique olive coral navajo 5 +Manufacturer#4 almond antique gainsboro frosted violet 5 +Manufacturer#4 almond antique violet mint lemon 5 +Manufacturer#4 almond aquamarine floral ivory bisque 5 +Manufacturer#4 almond aquamarine yellow dodger mint 5 +Manufacturer#4 almond azure aquamarine papaya violet 5 +Manufacturer#5 almond antique blue firebrick mint 5 +Manufacturer#5 almond antique medium spring khaki 5 +Manufacturer#5 almond antique sky peru orange 5 +Manufacturer#5 almond aquamarine dodger light gainsboro 5 +Manufacturer#5 almond azure blanched chiffon midnight 5 +PREHOOK: query: -- 5. testCountWithWindowingUDAF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +count(p_size) as cd, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 5. testCountWithWindowingUDAF +select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +count(p_size) as cd, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1 1 6 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 6 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 6 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 6 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 6 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 6 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 5 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 5 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 5 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 5 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 5 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 5 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 5 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 5 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 5 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 5 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 5 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 5 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 5 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 5 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 5 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 5 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 5 1464.48 7672.66 23 -23 +PREHOOK: query: -- 6. testCountInSubQ +select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz +from (select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +count(p_size) as cd, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +) sub1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 6. testCountInSubQ +select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz +from (select p_mfgr, p_name, +rank() as r, +dense_rank() as dr, +count(p_size) as cd, +p_retailprice, sum(p_retailprice) as s1 over (rows between unbounded preceding and current row), +p_size, p_size - lag(p_size,1) as deltaSz +from part +distribute by p_mfgr +sort by p_name +) sub1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +1 1 6 1173.15 0 +1 1 6 2346.3 0 +3 2 6 4100.06 32 +4 3 6 5702.650000000001 -28 +5 4 6 7117.070000000001 22 +6 5 6 8749.730000000001 14 +1 1 5 1690.68 0 +2 2 5 3491.38 26 +3 3 5 5523.360000000001 -38 +4 4 5 7222.02 23 +5 5 5 8923.62 -7 +1 1 5 1671.68 0 +2 2 5 2861.95 -3 +3 3 5 4272.34 5 +4 4 5 6195.32 -18 +5 5 5 7532.61 44 +1 1 5 1620.67 0 +2 2 5 2996.09 29 +3 3 5 4202.35 -12 +4 4 5 6047.27 -20 +5 5 5 7337.620000000001 5 +1 1 5 1789.69 0 +2 2 5 3401.3500000000004 -25 +3 3 5 5190.08 -4 +4 4 5 6208.18 44 +5 5 5 7672.66 -23 +PREHOOK: query: -- 7. testJoinWithWindowingAndPTF +select abc.p_mfgr, abc.p_name, +rank() as r, +dense_rank() as dr, +abc.p_retailprice, sum(abc.p_retailprice) as s1 over (rows between unbounded preceding and current row), +abc.p_size, abc.p_size - lag(abc.p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +distribute by abc.p_mfgr +sort by abc.p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 7. testJoinWithWindowingAndPTF +select abc.p_mfgr, abc.p_name, +rank() as r, +dense_rank() as dr, +abc.p_retailprice, sum(abc.p_retailprice) as s1 over (rows between unbounded preceding and current row), +abc.p_size, abc.p_size - lag(abc.p_size,1) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +distribute by abc.p_mfgr +sort by abc.p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 2346.3 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 3519.4500000000003 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 1173.15 4692.6 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 5 2 1753.76 6446.360000000001 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3 1602.59 8048.950000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 7 4 1414.42 9463.37 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 8 5 1632.66 11096.03 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 1464.48 7672.66 23 -23 +PREHOOK: query: -- 8. testMixedCaseAlias +select p_mfgr, p_name, p_size, rank() as R +from part +distribute by p_mfgr +sort by p_name, p_size desc +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 8. testMixedCaseAlias +select p_mfgr, p_name, p_size, rank() as R +from part +distribute by p_mfgr +sort by p_name, p_size desc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 +PREHOOK: query: -- 9. testHavingWithWindowingNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +having p_size > 5 +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 9. testHavingWithWindowingNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +having p_size > 5 +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: -- 10. testHavingWithWindowingCondRankNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +having rank() < 4 +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 10. testHavingWithWindowingCondRankNoGBY +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +having rank() < 4 +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +PREHOOK: query: -- 11. testFirstLast +select p_mfgr,p_name, p_size, +sum(p_size) as s2 over (rows between current row and current row), +first_value(p_size) as f over w1 , +last_value(p_size, false) as l over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 11. testFirstLast +select p_mfgr,p_name, p_size, +sum(p_size) as s2 over (rows between current row and current row), +first_value(p_size) as f over w1 , +last_value(p_size, false) as l over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 2 6 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 28 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 34 42 +Manufacturer#1 almond aquamarine burnished black steel 28 28 2 42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 6 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 2 +Manufacturer#2 almond antique violet turquoise frosted 40 40 14 25 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 14 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 40 18 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 2 18 +Manufacturer#3 almond antique metallic orange dim 19 19 19 14 +Manufacturer#3 almond antique chartreuse khaki white 17 17 19 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 19 45 +Manufacturer#3 almond antique misty red olive 1 1 17 45 +Manufacturer#3 almond antique olive coral navajo 45 45 14 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 10 10 27 +Manufacturer#4 almond antique violet mint lemon 39 39 10 7 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 10 12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 39 12 +Manufacturer#4 almond azure aquamarine papaya violet 12 12 27 12 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 2 +Manufacturer#5 almond antique medium spring khaki 6 6 31 46 +Manufacturer#5 almond antique sky peru orange 2 2 31 23 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 6 23 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 2 23 +PREHOOK: query: -- 12. testFirstLastWithWhere +select p_mfgr,p_name, p_size, +rank() as r, +sum(p_size) as s2 over (rows between current row and current row), +first_value(p_size) as f over w1, +last_value(p_size, false) as l over w1 +from part +where p_mfgr = 'Manufacturer#3' +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 12. testFirstLastWithWhere +select p_mfgr,p_name, p_size, +rank() as r, +sum(p_size) as s2 over (rows between current row and current row), +first_value(p_size) as f over w1, +last_value(p_size, false) as l over w1 +from part +where p_mfgr = 'Manufacturer#3' +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#3 almond antique chartreuse khaki white 17 1 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 1 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 1 19 17 45 +Manufacturer#3 almond antique misty red olive 1 1 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 1 45 19 45 +PREHOOK: query: -- 13. testSumWindow +select p_mfgr,p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over (rows between current row and current row) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 13. testSumWindow +select p_mfgr,p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over (rows between current row and current row) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 38 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 44 34 +Manufacturer#1 almond antique burnished rose metallic 2 72 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 6 +Manufacturer#1 almond aquamarine burnished black steel 28 78 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 18 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique chartreuse khaki white 17 51 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 96 14 +Manufacturer#3 almond antique misty red olive 1 77 1 +Manufacturer#3 almond antique olive coral navajo 45 60 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 12 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 6 +Manufacturer#5 almond antique sky peru orange 2 108 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 23 +PREHOOK: query: -- 14. testNoSortClause +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr +from part +distribute by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 14. testNoSortClause +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr +from part +distribute by p_mfgr +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1 1 +Manufacturer#1 almond aquamarine burnished black steel 28 1 1 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1 1 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 +Manufacturer#2 almond antique violet turquoise frosted 40 1 1 +Manufacturer#2 almond aquamarine midnight light salmon 2 1 1 +Manufacturer#2 almond aquamarine rose maroon antique 25 1 1 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1 1 +Manufacturer#3 almond antique metallic orange dim 19 1 1 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 1 1 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique olive coral navajo 45 1 1 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 +Manufacturer#4 almond antique violet mint lemon 39 1 1 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1 1 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1 1 +Manufacturer#4 almond azure aquamarine papaya violet 12 1 1 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 +Manufacturer#5 almond antique medium spring khaki 6 1 1 +Manufacturer#5 almond antique sky peru orange 2 1 1 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1 1 +Manufacturer#5 almond azure blanched chiffon midnight 23 1 1 +PREHOOK: query: -- 15. testExpressions +select p_mfgr,p_name, p_size, +rank() as r, +dense_rank() as dr, +cume_dist() as cud, +percent_rank() as pr, +ntile(3) as nt, +count(p_size) as ca, +avg(p_size) as avg, +stddev(p_size) as st, +first_value(p_size % 5) as fv, +last_value(p_size) as lv, +first_value(p_size, true) as fvW1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 15. testExpressions +select p_mfgr,p_name, p_size, +rank() as r, +dense_rank() as dr, +cume_dist() as cud, +percent_rank() as pr, +ntile(3) as nt, +count(p_size) as ca, +avg(p_size) as avg, +stddev(p_size) as st, +first_value(p_size % 5) as fv, +last_value(p_size) as lv, +first_value(p_size, true) as fvW1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 0.4 2 6 19.0 16.237815945091466 2 42 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 0.6 2 6 19.0 16.237815945091466 2 42 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 0.8 3 6 19.0 16.237815945091466 2 42 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 1.0 3 6 19.0 16.237815945091466 2 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 0.0 1 5 19.8 12.560254774486067 4 18 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 0.25 1 5 19.8 12.560254774486067 4 18 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 0.75 2 5 19.8 12.560254774486067 4 18 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 1.0 3 5 19.8 12.560254774486067 4 18 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 0.0 1 5 19.2 14.344336861632886 2 45 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 0.25 1 5 19.2 14.344336861632886 2 45 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 0.5 2 5 19.2 14.344336861632886 2 45 17 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 1.0 3 5 19.2 14.344336861632886 2 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 0.0 1 5 19.0 12.149074038789951 0 12 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 0.25 1 5 19.0 12.149074038789951 0 12 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 0.5 2 5 19.0 12.149074038789951 0 12 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 0.75 2 5 19.0 12.149074038789951 0 12 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 1.0 3 5 19.0 12.149074038789951 0 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 0.0 1 5 21.6 16.206171663906314 1 23 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 0.25 1 5 21.6 16.206171663906314 1 23 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 0.75 2 5 21.6 16.206171663906314 1 23 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 1.0 3 5 21.6 16.206171663906314 1 23 2 +PREHOOK: query: -- 16. testMultipleWindows +select p_mfgr,p_name, p_size, + rank() as r, dense_rank() as dr, +cume_dist() as cud, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +first_value(p_size, true) as fv1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 16. testMultipleWindows +select p_mfgr,p_name, p_size, + rank() as r, dense_rank() as dr, +cume_dist() as cud, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +first_value(p_size, true) as fv1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 38 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 44 44 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 72 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 114 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 54 40 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 81 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 99 43 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 17 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 31 31 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 50 19 17 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 96 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 10 10 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 49 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 76 66 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 83 83 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 95 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 31 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 37 37 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 85 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 108 69 2 +PREHOOK: query: -- 17. testCountStar +select p_mfgr,p_name, p_size, +count(*) as c, +count(p_size) as ca, +first_value(p_size, true) as fvW1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 17. testCountStar +select p_mfgr,p_name, p_size, +count(*) as c, +count(p_size) as ca, +first_value(p_size, true) as fvW1 over w1 +from part +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique chartreuse lavender yellow 34 6 6 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 6 6 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 5 5 14 +Manufacturer#2 almond antique violet turquoise frosted 40 5 5 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 5 5 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 2 +Manufacturer#3 almond antique chartreuse khaki white 17 5 5 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 5 5 17 +Manufacturer#3 almond antique metallic orange dim 19 5 5 17 +Manufacturer#3 almond antique olive coral navajo 45 5 5 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 5 5 10 +Manufacturer#4 almond antique violet mint lemon 39 5 5 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 5 5 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5 5 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 27 +Manufacturer#5 almond antique blue firebrick mint 31 5 5 31 +Manufacturer#5 almond antique medium spring khaki 6 5 5 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5 5 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 2 +PREHOOK: query: -- 18. testUDAFs +select p_mfgr,p_name, p_size, +sum(p_retailprice) as s over w1, +min(p_retailprice) as mi over w1, +max(p_retailprice) as ma over w1, +avg(p_retailprice) as ag over w1 +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 18. testUDAFs +select p_mfgr,p_name, p_size, +sum(p_retailprice) as s over w1, +min(p_retailprice) as mi over w1, +max(p_retailprice) as ma over w1, +avg(p_retailprice) as ag over w1 +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 4100.06 1173.15 1753.76 1366.6866666666667 +Manufacturer#1 almond antique burnished rose metallic 2 5702.650000000001 1173.15 1753.76 1425.6625000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 1173.15 1753.76 1423.4140000000002 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 7576.58 1173.15 1753.76 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 6403.43 1414.42 1753.76 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 4649.67 1414.42 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 5523.360000000001 1690.68 2031.98 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 7222.02 1690.68 2031.98 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 7232.9400000000005 1698.66 2031.98 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5432.24 1698.66 2031.98 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 4272.34 1190.27 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 6195.32 1190.27 1922.98 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique misty red olive 1 5860.929999999999 1190.27 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 4670.66 1337.29 1922.98 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 4202.35 1206.26 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 6047.27 1206.26 1844.92 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5716.950000000001 1206.26 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 4341.530000000001 1206.26 1844.92 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 5190.08 1611.66 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 6208.18 1018.1 1789.69 1552.045 +Manufacturer#5 almond antique sky peru orange 2 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5882.970000000001 1018.1 1788.73 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4271.3099999999995 1018.1 1788.73 1423.7699999999998 +PREHOOK: query: -- 19. testUDAFsWithGBY +select p_mfgr,p_name, p_size, p_retailprice, +sum(p_retailprice) as s over w1, +min(p_retailprice) as mi , +max(p_retailprice) as ma , +avg(p_retailprice) as ag over w1 +from part +group by p_mfgr,p_name, p_size, p_retailprice +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 19. testUDAFsWithGBY +select p_mfgr,p_name, p_size, p_retailprice, +sum(p_retailprice) as s over w1, +min(p_retailprice) as mi , +max(p_retailprice) as ma , +avg(p_retailprice) as ag over w1 +from part +group by p_mfgr,p_name, p_size, p_retailprice +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 4529.5 1173.15 1173.15 1509.8333333333333 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 5943.92 1753.76 1753.76 1485.98 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 7576.58 1602.59 1602.59 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 6403.43 1414.42 1414.42 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 4649.67 1632.66 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 5523.360000000001 1690.68 1690.68 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 7222.02 1800.7 1800.7 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 8923.62 2031.98 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 7232.9400000000005 1698.66 1698.66 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5432.24 1701.6 1701.6 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 4272.34 1671.68 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 6195.32 1190.27 1190.27 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 7532.61 1410.39 1410.39 1506.522 +Manufacturer#3 almond antique misty red olive 1 1922.98 5860.929999999999 1922.98 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 4670.66 1337.29 1337.29 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 4202.35 1620.67 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 6047.27 1375.42 1375.42 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 7337.620000000001 1206.26 1206.26 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 5716.950000000001 1844.92 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 4341.530000000001 1290.35 1290.35 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 5190.08 1789.69 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 6208.18 1611.66 1611.66 1552.045 +Manufacturer#5 almond antique sky peru orange 2 1788.73 7672.66 1788.73 1788.73 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 5882.970000000001 1018.1 1018.1 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 4271.3099999999995 1464.48 1464.48 1423.7699999999998 +PREHOOK: query: -- 20. testSTATs +select p_mfgr,p_name, p_size, +stddev(p_retailprice) as sdev over w1, +stddev_pop(p_retailprice) as sdev_pop over w1, +collect_set(p_size) as uniq_size over w1, +variance(p_retailprice) as var over w1, +corr(p_size, p_retailprice) as cor over w1, +covar_pop(p_size, p_retailprice) as covarp over w1 +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 20. testSTATs +select p_mfgr,p_name, p_size, +stddev(p_retailprice) as sdev over w1, +stddev_pop(p_retailprice) as sdev_pop over w1, +collect_set(p_size) as uniq_size over w1, +variance(p_retailprice) as var over w1, +corr(p_size, p_retailprice) as cor over w1, +covar_pop(p_size, p_retailprice) as covarp over w1 +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 [34,2] 74912.8826888888 1.0 4128.782222222221 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 [34,2,6] 66619.10876874991 0.811328754177887 2801.7074999999995 +Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 [34,2,6,28] 53315.51002399992 0.695639377397664 2210.7864 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 [34,2,6,42,28] 41099.896184 0.630785977101214 2009.9536000000007 +Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 [34,6,42,28] 14788.129118750014 0.2036684720435979 331.1337500000004 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 [6,42,28] 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502 +Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 [2,40,14] 20231.169866666663 -0.49369526554523185 -1113.7466666666658 +Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 [2,25,40,14] 18978.662075 -0.5205630897335946 -1004.4812499999995 +Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 [2,18,25,40,14] 16910.329504000005 -0.46908967495720255 -766.1791999999995 +Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 [2,18,25,40] 18374.07627499999 -0.6091405874714462 -1128.1787499999987 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 [2,18,25] 24473.534488888927 -0.9571686373491608 -1441.4466666666676 +Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 [17,19,14] 38720.09628888887 0.5557168646224995 224.6944444444446 +Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 [17,1,19,14] 75702.81305 -0.6720833036576083 -1296.9000000000003 +Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 [17,1,19,14,45] 67722.117896 -0.5703526513979519 -2129.0664 +Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 [1,19,14,45] 76128.53331875012 -0.577476899644802 -2547.7868749999993 +Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 [1,19,45] 67902.76602222225 -0.8710736366736884 -4099.731111111111 +Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 [39,27,10] 28944.25735555559 -0.6656975320098423 -1347.4777777777779 +Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 [39,7,27,10] 58693.95151875002 -0.8051852719193339 -2537.328125 +Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 [39,7,27,10,12] 54802.817784000035 -0.6046935574240581 -1719.8079999999995 +Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 [39,7,27,12] 61174.24181875003 -0.5508665654707869 -1719.0368749999975 +Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 [7,27,12] 80278.40095555557 -0.7755740084632333 -1867.4888888888881 +Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 [2,6,31] 7005.487488888913 0.39004303087285047 418.9233333333353 +Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 [2,6,46,31] 100286.53662500004 -0.713612911776183 -4090.853749999999 +Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 [2,23,6,46,31] 81456.04997600002 -0.712858514567818 -3297.2011999999986 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 [2,23,6,46] 81474.56091875004 -0.984128787153391 -4871.028125000002 +Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 [2,23,46] 99807.08486666664 -0.9978877469246936 -5664.856666666666 +PREHOOK: query: -- 21. testDISTs +select p_mfgr,p_name, p_size, +histogram_numeric(p_retailprice, 5) as hist over w1, +percentile(p_partkey, 0.5) as per over w1, +row_number() as rn +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 21. testDISTs +select p_mfgr,p_name, p_size, +histogram_numeric(p_retailprice, 5) as hist over w1, +percentile(p_partkey, 0.5) as per over w1, +row_number() as rn +from part +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1753.76,"y":1.0}] 121152.0 1 +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 115872.0 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 [{"x":1173.15,"y":2.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 110592.0 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 [{"x":1173.15,"y":1.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86428.0 4 +Manufacturer#1 almond aquamarine burnished black steel 28 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86098.0 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0}] 86428.0 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 [{"x":1690.68,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 1 +Manufacturer#2 almond antique violet turquoise frosted 40 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 139825.5 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 169347.0 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 5 +Manufacturer#3 almond antique chartreuse khaki white 17 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0}] 90681.0 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 65831.5 2 +Manufacturer#3 almond antique metallic orange dim 19 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 90681.0 3 +Manufacturer#3 almond antique misty red olive 1 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 76690.0 4 +Manufacturer#3 almond antique olive coral navajo 45 [{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 112398.0 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0}] 48427.0 1 +Manufacturer#4 almond antique violet mint lemon 39 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 46844.0 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 45261.0 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1844.92,"y":1.0}] 39309.0 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1844.92,"y":1.0}] 33357.0 5 +Manufacturer#5 almond antique blue firebrick mint 31 [{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 155733.0 1 +Manufacturer#5 almond antique medium spring khaki 6 [{"x":1018.1,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 99201.0 2 +Manufacturer#5 almond antique sky peru orange 2 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 78486.0 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0}] 60577.5 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1788.73,"y":1.0}] 78486.0 5 +PREHOOK: query: -- 22. testViewAsTableInputWithWindowing +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand +PREHOOK: type: CREATEVIEW +#### A masked pattern was here #### +POSTHOOK: query: -- 22. testViewAsTableInputWithWindowing +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand +POSTHOOK: type: CREATEVIEW +POSTHOOK: Output: default@mfgr_price_view +#### A masked pattern was here #### +PREHOOK: query: select p_mfgr, p_brand, s, +sum(s) as s1 over w1 +from mfgr_price_view +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and current row) +PREHOOK: type: QUERY +PREHOOK: Input: default@mfgr_price_view +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select p_mfgr, p_brand, s, +sum(s) as s1 over w1 +from mfgr_price_view +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and current row) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@mfgr_price_view +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 Brand#14 2346.3 2346.3 +Manufacturer#1 Brand#12 4800.84 7147.14 +Manufacturer#1 Brand#15 1602.59 8749.73 +Manufacturer#2 Brand#22 3491.38 3491.38 +Manufacturer#2 Brand#23 2031.98 5523.360000000001 +Manufacturer#2 Brand#24 1698.66 7222.02 +Manufacturer#2 Brand#25 1701.6 5432.24 +Manufacturer#3 Brand#32 3333.37 3333.37 +Manufacturer#3 Brand#31 1671.68 5005.05 +Manufacturer#3 Brand#34 1337.29 6342.34 +Manufacturer#3 Brand#35 1190.27 4199.24 +Manufacturer#4 Brand#41 4755.9400000000005 4755.9400000000005 +Manufacturer#4 Brand#42 2581.6800000000003 7337.620000000001 +Manufacturer#5 Brand#51 1611.66 1611.66 +Manufacturer#5 Brand#52 3254.17 4865.83 +Manufacturer#5 Brand#53 2806.83 7672.66 +PREHOOK: query: -- 23. testCreateViewWithWindowingQuery +create view IF NOT EXISTS mfgr_brand_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and current row) +PREHOOK: type: CREATEVIEW +#### A masked pattern was here #### +POSTHOOK: query: -- 23. testCreateViewWithWindowingQuery +create view IF NOT EXISTS mfgr_brand_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s over w1 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and current row) +POSTHOOK: type: CREATEVIEW +POSTHOOK: Output: default@mfgr_brand_price_view +#### A masked pattern was here #### +PREHOOK: query: select * from mfgr_brand_price_view +PREHOOK: type: QUERY +PREHOOK: Input: default@mfgr_brand_price_view +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: select * from mfgr_brand_price_view +POSTHOOK: type: QUERY +POSTHOOK: Input: default@mfgr_brand_price_view +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 Brand#14 1173.15 +Manufacturer#1 Brand#12 2926.91 +Manufacturer#1 Brand#14 4100.0599999999995 +Manufacturer#1 Brand#15 4529.5 +Manufacturer#1 Brand#12 4190.16 +Manufacturer#1 Brand#12 4649.67 +Manufacturer#2 Brand#22 1690.68 +Manufacturer#2 Brand#22 3491.38 +Manufacturer#2 Brand#23 5523.360000000001 +Manufacturer#2 Brand#24 5531.34 +Manufacturer#2 Brand#25 5432.24 +Manufacturer#3 Brand#32 1410.39 +Manufacturer#3 Brand#31 3082.07 +Manufacturer#3 Brand#35 4272.34 +Manufacturer#3 Brand#32 4784.93 +Manufacturer#3 Brand#34 4450.54 +Manufacturer#4 Brand#41 1620.67 +Manufacturer#4 Brand#42 2996.09 +Manufacturer#4 Brand#42 4202.35 +Manufacturer#4 Brand#41 4426.6 +Manufacturer#4 Brand#41 4341.530000000001 +Manufacturer#5 Brand#52 1789.69 +Manufacturer#5 Brand#51 3401.3500000000004 +Manufacturer#5 Brand#53 5190.08 +Manufacturer#5 Brand#53 4418.490000000001 +Manufacturer#5 Brand#52 4271.3099999999995 +PREHOOK: query: -- 24. testLateralViews +select p_mfgr, p_name, +lv_col, p_size, sum(p_size) as s over w1 +from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p +lateral view explode(arr) part_lv as lv_col +distribute by p_mfgr +sort by p_name +window w1 as (rows between 2 preceding and current row) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 24. testLateralViews +select p_mfgr, p_name, +lv_col, p_size, sum(p_size) as s over w1 +from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p +lateral view explode(arr) part_lv as lv_col +distribute by p_mfgr +sort by p_name +window w1 as (rows between 2 preceding and current row) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 almond antique burnished rose metallic 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 2 4 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique burnished rose metallic 1 2 6 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique chartreuse lavender yellow 1 34 38 +Manufacturer#1 almond antique chartreuse lavender yellow 2 34 70 +Manufacturer#1 almond antique chartreuse lavender yellow 3 34 102 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 6 74 +Manufacturer#1 almond antique salmon chartreuse burlywood 2 6 46 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 6 18 +Manufacturer#1 almond aquamarine burnished black steel 1 28 40 +Manufacturer#1 almond aquamarine burnished black steel 2 28 62 +Manufacturer#1 almond aquamarine burnished black steel 3 28 84 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 42 98 +Manufacturer#1 almond aquamarine pink moccasin thistle 2 42 112 +Manufacturer#1 almond aquamarine pink moccasin thistle 3 42 126 +Manufacturer#2 almond antique violet chocolate turquoise 2 14 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 14 28 +Manufacturer#2 almond antique violet chocolate turquoise 3 14 42 +Manufacturer#2 almond antique violet turquoise frosted 1 40 68 +Manufacturer#2 almond antique violet turquoise frosted 2 40 94 +Manufacturer#2 almond antique violet turquoise frosted 3 40 120 +Manufacturer#2 almond aquamarine midnight light salmon 1 2 82 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 44 +Manufacturer#2 almond aquamarine midnight light salmon 3 2 6 +Manufacturer#2 almond aquamarine rose maroon antique 1 25 29 +Manufacturer#2 almond aquamarine rose maroon antique 2 25 52 +Manufacturer#2 almond aquamarine rose maroon antique 3 25 75 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 18 68 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 2 18 61 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 3 18 54 +Manufacturer#3 almond antique chartreuse khaki white 1 17 17 +Manufacturer#3 almond antique chartreuse khaki white 2 17 34 +Manufacturer#3 almond antique chartreuse khaki white 3 17 51 +Manufacturer#3 almond antique forest lavender goldenrod 1 14 48 +Manufacturer#3 almond antique forest lavender goldenrod 2 14 45 +Manufacturer#3 almond antique forest lavender goldenrod 3 14 42 +Manufacturer#3 almond antique metallic orange dim 1 19 47 +Manufacturer#3 almond antique metallic orange dim 2 19 52 +Manufacturer#3 almond antique metallic orange dim 3 19 57 +Manufacturer#3 almond antique misty red olive 1 1 39 +Manufacturer#3 almond antique misty red olive 2 1 21 +Manufacturer#3 almond antique misty red olive 3 1 3 +Manufacturer#3 almond antique olive coral navajo 1 45 47 +Manufacturer#3 almond antique olive coral navajo 2 45 91 +Manufacturer#3 almond antique olive coral navajo 3 45 135 +Manufacturer#4 almond antique gainsboro frosted violet 1 10 10 +Manufacturer#4 almond antique gainsboro frosted violet 2 10 20 +Manufacturer#4 almond antique gainsboro frosted violet 3 10 30 +Manufacturer#4 almond antique violet mint lemon 1 39 59 +Manufacturer#4 almond antique violet mint lemon 2 39 88 +Manufacturer#4 almond antique violet mint lemon 3 39 117 +Manufacturer#4 almond aquamarine floral ivory bisque 1 27 105 +Manufacturer#4 almond aquamarine floral ivory bisque 2 27 93 +Manufacturer#4 almond aquamarine floral ivory bisque 3 27 81 +Manufacturer#4 almond aquamarine yellow dodger mint 1 7 61 +Manufacturer#4 almond aquamarine yellow dodger mint 2 7 41 +Manufacturer#4 almond aquamarine yellow dodger mint 3 7 21 +Manufacturer#4 almond azure aquamarine papaya violet 1 12 26 +Manufacturer#4 almond azure aquamarine papaya violet 2 12 31 +Manufacturer#4 almond azure aquamarine papaya violet 3 12 36 +Manufacturer#5 almond antique blue firebrick mint 2 31 31 +Manufacturer#5 almond antique blue firebrick mint 1 31 62 +Manufacturer#5 almond antique blue firebrick mint 3 31 93 +Manufacturer#5 almond antique medium spring khaki 1 6 68 +Manufacturer#5 almond antique medium spring khaki 2 6 43 +Manufacturer#5 almond antique medium spring khaki 3 6 18 +Manufacturer#5 almond antique sky peru orange 2 2 14 +Manufacturer#5 almond antique sky peru orange 1 2 10 +Manufacturer#5 almond antique sky peru orange 3 2 6 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 46 50 +Manufacturer#5 almond aquamarine dodger light gainsboro 2 46 94 +Manufacturer#5 almond aquamarine dodger light gainsboro 3 46 138 +Manufacturer#5 almond azure blanched chiffon midnight 1 23 115 +Manufacturer#5 almond azure blanched chiffon midnight 2 23 92 +Manufacturer#5 almond azure blanched chiffon midnight 3 23 69 +PREHOOK: query: -- 25. testMultipleInserts3SWQs +CREATE TABLE part_1( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- 25. testMultipleInserts3SWQs +CREATE TABLE part_1( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_1 +PREHOOK: query: CREATE TABLE part_2( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +cud INT, +s1 DOUBLE, +s2 DOUBLE, +fv1 INT) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part_2( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +cud INT, +s1 DOUBLE, +s2 DOUBLE, +fv1 INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_2 +PREHOOK: query: CREATE TABLE part_3( +p_mfgr STRING, +p_name STRING, +p_size INT, +c INT, +ca INT, +fv INT) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE part_3( +p_mfgr STRING, +p_name STRING, +p_size INT, +c INT, +ca INT, +fv INT) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_3 +PREHOOK: query: from part +INSERT OVERWRITE TABLE part_1 +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s over (rows between unbounded preceding and current row) +distribute by p_mfgr +sort by p_name +INSERT OVERWRITE TABLE part_2 +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr, +cume_dist() as cud, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +first_value(p_size, true) as fv1 over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +INSERT OVERWRITE TABLE part_3 +select p_mfgr,p_name, p_size, +count(*) as c, +count(p_size) as ca, +first_value(p_size, true) as fv over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +PREHOOK: Output: default@part_1 +PREHOOK: Output: default@part_2 +PREHOOK: Output: default@part_3 +POSTHOOK: query: from part +INSERT OVERWRITE TABLE part_1 +select p_mfgr, p_name, p_size, +rank() as r, +dense_rank() as dr, +sum(p_retailprice) as s over (rows between unbounded preceding and current row) +distribute by p_mfgr +sort by p_name +INSERT OVERWRITE TABLE part_2 +select p_mfgr,p_name, p_size, +rank() as r, dense_rank() as dr, +cume_dist() as cud, +sum(p_size) as s1 over (rows between unbounded preceding and current row), +sum(p_size) as s2 over (range between p_size 5 less and current row), +first_value(p_size, true) as fv1 over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +INSERT OVERWRITE TABLE part_3 +select p_mfgr,p_name, p_size, +count(*) as c, +count(p_size) as ca, +first_value(p_size, true) as fv over w1 +having p_size > 5 +distribute by p_mfgr +sort by p_mfgr, p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +POSTHOOK: Output: default@part_1 +POSTHOOK: Output: default@part_2 +POSTHOOK: Output: default@part_3 +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +PREHOOK: query: select * from part_1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part_1 +#### A masked pattern was here #### +POSTHOOK: query: select * from part_1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_1 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 +PREHOOK: query: select * from part_2 +PREHOOK: type: QUERY +PREHOOK: Input: default@part_2 +#### A masked pattern was here #### +POSTHOOK: query: select * from part_2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_2 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0 38.0 34.0 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0 44.0 44.0 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0 72.0 28.0 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1 114.0 42.0 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0 14.0 14.0 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0 54.0 40.0 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0 81.0 25.0 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1 99.0 43.0 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0 17.0 17.0 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0 31.0 31.0 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0 50.0 19.0 17 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1 96.0 45.0 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0 10.0 10.0 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0 49.0 39.0 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0 76.0 66.0 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0 83.0 83.0 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1 95.0 12.0 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0 31.0 31.0 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0 37.0 37.0 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0 85.0 46.0 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1 108.0 69.0 2 +PREHOOK: query: select * from part_3 +PREHOOK: type: QUERY +PREHOOK: Input: default@part_3 +#### A masked pattern was here #### +POSTHOOK: query: select * from part_3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_3 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique chartreuse lavender yellow 34 6 6 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 6 6 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 5 5 14 +Manufacturer#2 almond antique violet turquoise frosted 40 5 5 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 5 5 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 2 +Manufacturer#3 almond antique chartreuse khaki white 17 5 5 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 5 5 17 +Manufacturer#3 almond antique metallic orange dim 19 5 5 17 +Manufacturer#3 almond antique olive coral navajo 45 5 5 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 5 5 10 +Manufacturer#4 almond antique violet mint lemon 39 5 5 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 5 5 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5 5 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 27 +Manufacturer#5 almond antique blue firebrick mint 31 5 5 31 +Manufacturer#5 almond antique medium spring khaki 6 5 5 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5 5 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 2 +PREHOOK: query: -- 26. testGroupByHavingWithSWQAndAlias +select p_mfgr, p_name, p_size, min(p_retailprice) as mi, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +having p_size > 0 +distribute by p_mfgr +sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 26. testGroupByHavingWithSWQAndAlias +select p_mfgr, p_name, p_size, min(p_retailprice) as mi, +rank() as r, +dense_rank() as dr, +p_size, p_size - lag(p_size,1) as deltaSz +from part +group by p_mfgr, p_name, p_size +having p_size > 0 +distribute by p_mfgr +sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 +PREHOOK: query: -- 27. testMultipleRangeWindows +select p_mfgr,p_name, p_size, +sum(p_size) as s2 over (range between p_size 10 less and current row), +sum(p_size) as s1 over (range between current row and p_size 10 more ) +from part +distribute by p_mfgr +sort by p_mfgr, p_size +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 27. testMultipleRangeWindows +select p_mfgr,p_name, p_size, +sum(p_size) as s2 over (range between p_size 10 less and current row), +sum(p_size) as s1 over (range between current row and p_size 10 more ) +from part +distribute by p_mfgr +sort by p_mfgr, p_size +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 2 10 +Manufacturer#1 almond antique burnished rose metallic 2 4 8 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 10 6 +Manufacturer#1 almond aquamarine burnished black steel 28 28 62 +Manufacturer#1 almond antique chartreuse lavender yellow 34 62 76 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 32 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 32 43 +Manufacturer#2 almond aquamarine rose maroon antique 25 43 25 +Manufacturer#2 almond antique violet turquoise frosted 40 40 40 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 50 +Manufacturer#3 almond antique chartreuse khaki white 17 31 36 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique olive coral navajo 45 45 45 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 29 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 22 +Manufacturer#4 almond azure aquamarine papaya violet 12 29 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 27 +Manufacturer#4 almond antique violet mint lemon 39 39 39 +Manufacturer#5 almond antique sky peru orange 2 2 8 +Manufacturer#5 almond antique medium spring khaki 6 8 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 54 +Manufacturer#5 almond antique blue firebrick mint 31 54 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 46 +PREHOOK: query: -- 28. testPartOrderInUDAFInvoke +select p_mfgr, p_name, p_size, +sum(p_size) as s over (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 28. testPartOrderInUDAFInvoke +select p_mfgr, p_name, p_size, +sum(p_size) as s over (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 +PREHOOK: query: -- 29. testPartOrderInWdwDef +select p_mfgr, p_name, p_size, +sum(p_size) as s over w1 +from part +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 29. testPartOrderInWdwDef +select p_mfgr, p_name, p_size, +sum(p_size) as s over w1 +from part +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 +PREHOOK: query: -- 30. testDefaultPartitioningSpecRules +select p_mfgr, p_name, p_size, +sum(p_size) as s over w1, + sum(p_size) as s2 over w2 +from part +sort by p_name +window w1 as (partition by p_mfgr rows between 2 preceding and 2 following), + w2 as (partition by p_mfgr order by p_name) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 30. testDefaultPartitioningSpecRules +select p_mfgr, p_name, p_size, +sum(p_size) as s over w1, + sum(p_size) as s2 over w2 +from part +sort by p_name +window w1 as (partition by p_mfgr rows between 2 preceding and 2 following), + w2 as (partition by p_mfgr order by p_name) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 114 +Manufacturer#1 almond antique burnished rose metallic 2 44 114 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 114 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 114 +Manufacturer#1 almond aquamarine burnished black steel 28 110 114 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 99 +Manufacturer#2 almond antique violet turquoise frosted 40 81 99 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 99 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 +Manufacturer#3 almond antique chartreuse khaki white 17 50 96 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 96 +Manufacturer#3 almond antique metallic orange dim 19 96 96 +Manufacturer#3 almond antique misty red olive 1 79 96 +Manufacturer#3 almond antique olive coral navajo 45 65 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 95 +Manufacturer#4 almond antique violet mint lemon 39 83 95 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 95 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 108 +Manufacturer#5 almond antique medium spring khaki 6 85 108 +Manufacturer#5 almond antique sky peru orange 2 108 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 108 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 +PREHOOK: query: -- 31. testWindowCrossReference +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as w1 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 31. testWindowCrossReference +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as w1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 38 +Manufacturer#1 almond antique chartreuse lavender yellow 34 44 44 +Manufacturer#1 almond antique burnished rose metallic 2 72 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 112 +Manufacturer#1 almond aquamarine burnished black steel 28 78 78 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 45 +Manufacturer#3 almond antique metallic orange dim 19 50 50 +Manufacturer#3 almond antique chartreuse khaki white 17 51 51 +Manufacturer#3 almond antique forest lavender goldenrod 14 96 96 +Manufacturer#3 almond antique misty red olive 1 77 77 +Manufacturer#3 almond antique olive coral navajo 45 60 60 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 76 +Manufacturer#4 almond antique violet mint lemon 39 83 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 39 +Manufacturer#5 almond antique medium spring khaki 6 85 85 +Manufacturer#5 almond antique sky peru orange 2 108 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 71 +PREHOOK: query: -- 32. testWindowInheritance +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as (w1 rows between unbounded preceding and current row) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 32. testWindowInheritance +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as (w1 rows between unbounded preceding and current row) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 44 36 +Manufacturer#1 almond antique burnished rose metallic 2 72 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 +Manufacturer#1 almond aquamarine burnished black steel 28 78 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique chartreuse khaki white 17 51 36 +Manufacturer#3 almond antique forest lavender goldenrod 14 96 50 +Manufacturer#3 almond antique misty red olive 1 77 51 +Manufacturer#3 almond antique olive coral navajo 45 60 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 49 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 37 +Manufacturer#5 almond antique sky peru orange 2 108 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 +PREHOOK: query: -- 33. testWindowForwardReference +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2, +sum(p_size) as s3 over w3 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as w3, + w3 as (rows between unbounded preceding and current row) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 33. testWindowForwardReference +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2, +sum(p_size) as s3 over w3 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as w3, + w3 as (rows between unbounded preceding and current row) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 44 36 36 +Manufacturer#1 almond antique burnished rose metallic 2 72 38 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 44 +Manufacturer#1 almond aquamarine burnished black steel 28 78 72 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 114 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 54 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 56 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 99 +Manufacturer#3 almond antique metallic orange dim 19 50 19 19 +Manufacturer#3 almond antique chartreuse khaki white 17 51 36 36 +Manufacturer#3 almond antique forest lavender goldenrod 14 96 50 50 +Manufacturer#3 almond antique misty red olive 1 77 51 51 +Manufacturer#3 almond antique olive coral navajo 45 60 96 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 10 +Manufacturer#4 almond antique violet mint lemon 39 83 49 49 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 76 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 83 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 31 +Manufacturer#5 almond antique medium spring khaki 6 85 37 37 +Manufacturer#5 almond antique sky peru orange 2 108 39 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 85 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 108 +PREHOOK: query: -- 34. testWindowDefinitionPropagation +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2, +sum(p_size) as s3 over (w3 rows between 2 preceding and 2 following) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as w3, + w3 as (rows between unbounded preceding and current row) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 34. testWindowDefinitionPropagation +select p_mfgr, p_name, p_size, +sum(p_size) as s1 over w1, +sum(p_size) as s2 over w2, +sum(p_size) as s3 over (w3 rows between 2 preceding and 2 following) +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as w3, + w3 as (rows between unbounded preceding and current row) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 38 2 38 +Manufacturer#1 almond antique chartreuse lavender yellow 34 44 36 44 +Manufacturer#1 almond antique burnished rose metallic 2 72 38 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 112 +Manufacturer#1 almond aquamarine burnished black steel 28 78 72 78 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 45 +Manufacturer#3 almond antique metallic orange dim 19 50 19 50 +Manufacturer#3 almond antique chartreuse khaki white 17 51 36 51 +Manufacturer#3 almond antique forest lavender goldenrod 14 96 50 96 +Manufacturer#3 almond antique misty red olive 1 77 51 77 +Manufacturer#3 almond antique olive coral navajo 45 60 96 60 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 49 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 37 85 +Manufacturer#5 almond antique sky peru orange 2 108 39 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 71 +PREHOOK: query: -- 35. testDistinctWithWindowing +select DISTINCT p_mfgr, p_name, p_size, +sum(p_size) as s over w1 +from part +distribute by p_mfgr +sort by p_name +window w1 as (rows between 2 preceding and 2 following) +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 35. testDistinctWithWindowing +select DISTINCT p_mfgr, p_name, p_size, +sum(p_size) as s over w1 +from part +distribute by p_mfgr +sort by p_name +window w1 as (rows between 2 preceding and 2 following) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 42 +Manufacturer#1 almond antique chartreuse lavender yellow 34 70 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 +PREHOOK: query: -- 36. testRankWithPartitioning +select p_mfgr, p_name, p_size, +rank() as r over (partition by p_mfgr order by p_name ) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 36. testRankWithPartitioning +select p_mfgr, p_name, p_size, +rank() as r over (partition by p_mfgr order by p_name ) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 +PREHOOK: query: -- 37. testPartitioningVariousForms +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_mfgr), +min(p_retailprice) as s2 over (partition by p_mfgr), +max(p_retailprice) as s3 over (distribute by p_mfgr sort by p_mfgr), +avg(p_retailprice) as s4 over (distribute by p_mfgr), +count(p_retailprice) as s5 over (cluster by p_mfgr ) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 37. testPartitioningVariousForms +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr order by p_mfgr), +min(p_retailprice) as s2 over (partition by p_mfgr), +max(p_retailprice) as s3 over (distribute by p_mfgr sort by p_mfgr), +avg(p_retailprice) as s4 over (distribute by p_mfgr), +count(p_retailprice) as s5 over (cluster by p_mfgr ) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 8749.73 1173.15 1753.76 1458.2883333333332 6 +Manufacturer#1 almond antique chartreuse lavender yellow 34 8749.73 1173.15 1753.76 1458.2883333333332 6 +Manufacturer#1 almond antique burnished rose metallic 2 8749.73 1173.15 1753.76 1458.2883333333332 6 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 8749.73 1173.15 1753.76 1458.2883333333332 6 +Manufacturer#1 almond aquamarine burnished black steel 28 8749.73 1173.15 1753.76 1458.2883333333332 6 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.73 1173.15 1753.76 1458.2883333333332 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 1690.68 2031.98 1784.7240000000002 5 +Manufacturer#2 almond antique violet turquoise frosted 40 8923.62 1690.68 2031.98 1784.7240000000002 5 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 1690.68 2031.98 1784.7240000000002 5 +Manufacturer#2 almond aquamarine rose maroon antique 25 8923.62 1690.68 2031.98 1784.7240000000002 5 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 1690.68 2031.98 1784.7240000000002 5 +Manufacturer#3 almond antique metallic orange dim 19 7532.61 1190.27 1922.98 1506.522 5 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 1190.27 1922.98 1506.522 5 +Manufacturer#3 almond antique forest lavender goldenrod 14 7532.61 1190.27 1922.98 1506.522 5 +Manufacturer#3 almond antique misty red olive 1 7532.61 1190.27 1922.98 1506.522 5 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 1190.27 1922.98 1506.522 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 1206.26 1844.92 1467.5240000000001 5 +Manufacturer#4 almond antique violet mint lemon 39 7337.620000000001 1206.26 1844.92 1467.5240000000001 5 +Manufacturer#4 almond aquamarine floral ivory bisque 27 7337.620000000001 1206.26 1844.92 1467.5240000000001 5 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7337.620000000001 1206.26 1844.92 1467.5240000000001 5 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 1206.26 1844.92 1467.5240000000001 5 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 1018.1 1789.69 1534.532 5 +Manufacturer#5 almond antique medium spring khaki 6 7672.66 1018.1 1789.69 1534.532 5 +Manufacturer#5 almond antique sky peru orange 2 7672.66 1018.1 1789.69 1534.532 5 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 7672.66 1018.1 1789.69 1534.532 5 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 1018.1 1789.69 1534.532 5 +PREHOOK: query: -- 38. testPartitioningVariousForms2 +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr, p_name order by p_mfgr, p_name rows between unbounded preceding and current row), +min(p_retailprice) as s2 over (distribute by p_mfgr, p_name sort by p_mfgr, p_name rows between unbounded preceding and current row), +max(p_retailprice) as s3 over (cluster by p_mfgr, p_name ) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 38. testPartitioningVariousForms2 +select p_mfgr, p_name, p_size, +sum(p_retailprice) as s1 over (partition by p_mfgr, p_name order by p_mfgr, p_name rows between unbounded preceding and current row), +min(p_retailprice) as s2 over (distribute by p_mfgr, p_name sort by p_mfgr, p_name rows between unbounded preceding and current row), +max(p_retailprice) as s3 over (cluster by p_mfgr, p_name ) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1173.15 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 1173.15 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 1753.76 1753.76 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 1602.59 1602.59 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 1414.42 1414.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 1632.66 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1690.68 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 1800.7 1800.7 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 2031.98 2031.98 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 1698.66 1698.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 1701.6 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1671.68 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 1190.27 1190.27 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 1410.39 1410.39 +Manufacturer#3 almond antique misty red olive 1 1922.98 1922.98 1922.98 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 1337.29 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1620.67 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 1375.42 1375.42 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 1206.26 1206.26 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 1844.92 1844.92 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 1290.35 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1789.69 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 1611.66 1611.66 +Manufacturer#5 almond antique sky peru orange 2 1788.73 1788.73 1788.73 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 1018.1 1018.1 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 1464.48 1464.48 +PREHOOK: query: -- 39. testUDFOnOrderCols +select p_mfgr, p_type, substr(p_type, 2) as short_ptype, +rank() as r over (partition by p_mfgr order by substr(p_type, 2)) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 39. testUDFOnOrderCols +select p_mfgr, p_type, substr(p_type, 2) as short_ptype, +rank() as r over (partition by p_mfgr order by substr(p_type, 2)) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 LARGE BRUSHED STEEL ARGE BRUSHED STEEL 1 +Manufacturer#1 LARGE BURNISHED STEEL ARGE BURNISHED STEEL 2 +Manufacturer#1 PROMO BURNISHED NICKEL ROMO BURNISHED NICKEL 3 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 STANDARD ANODIZED STEEL TANDARD ANODIZED STEEL 6 +Manufacturer#2 ECONOMY POLISHED STEEL CONOMY POLISHED STEEL 1 +Manufacturer#2 MEDIUM ANODIZED COPPER EDIUM ANODIZED COPPER 2 +Manufacturer#2 MEDIUM BURNISHED COPPER EDIUM BURNISHED COPPER 3 +Manufacturer#2 SMALL POLISHED NICKEL MALL POLISHED NICKEL 4 +Manufacturer#2 STANDARD PLATED TIN TANDARD PLATED TIN 5 +Manufacturer#3 ECONOMY PLATED COPPER CONOMY PLATED COPPER 1 +Manufacturer#3 MEDIUM BURNISHED BRASS EDIUM BURNISHED BRASS 2 +Manufacturer#3 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#3 PROMO ANODIZED TIN ROMO ANODIZED TIN 4 +Manufacturer#3 STANDARD POLISHED STEEL TANDARD POLISHED STEEL 5 +Manufacturer#4 ECONOMY BRUSHED COPPER CONOMY BRUSHED COPPER 1 +Manufacturer#4 SMALL BRUSHED BRASS MALL BRUSHED BRASS 2 +Manufacturer#4 SMALL PLATED STEEL MALL PLATED STEEL 3 +Manufacturer#4 PROMO POLISHED STEEL ROMO POLISHED STEEL 4 +Manufacturer#4 STANDARD ANODIZED TIN TANDARD ANODIZED TIN 5 +Manufacturer#5 LARGE BRUSHED BRASS ARGE BRUSHED BRASS 1 +Manufacturer#5 ECONOMY BURNISHED STEEL CONOMY BURNISHED STEEL 2 +Manufacturer#5 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#5 SMALL PLATED BRASS MALL PLATED BRASS 4 +Manufacturer#5 STANDARD BURNISHED TIN TANDARD BURNISHED TIN 5 +PREHOOK: query: -- 40. testNoBetweenForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows unbounded preceding) + from part distribute by p_mfgr sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 40. testNoBetweenForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows unbounded preceding) + from part distribute by p_mfgr sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 4272.34 +Manufacturer#3 almond antique misty red olive 1 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 +PREHOOK: query: -- 41. testNoBetweenForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range unbounded preceding) + from part distribute by p_mfgr sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 41. testNoBetweenForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range unbounded preceding) + from part distribute by p_mfgr sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 4272.34 +Manufacturer#3 almond antique misty red olive 1 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 +PREHOOK: query: -- 42. testUnboundedFollowingForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows between current row and unbounded following) + from part distribute by p_mfgr sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 42. testUnboundedFollowingForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (rows between current row and unbounded following) + from part distribute by p_mfgr sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique chartreuse lavender yellow 34 6403.43 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4649.67 +Manufacturer#1 almond aquamarine burnished black steel 28 3047.08 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 +Manufacturer#2 almond antique violet turquoise frosted 40 7232.9400000000005 +Manufacturer#2 almond aquamarine midnight light salmon 2 5432.24 +Manufacturer#2 almond aquamarine rose maroon antique 25 3400.26 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 +Manufacturer#3 almond antique forest lavender goldenrod 14 5860.929999999999 +Manufacturer#3 almond antique metallic orange dim 19 4670.66 +Manufacturer#3 almond antique misty red olive 1 3260.27 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 +Manufacturer#4 almond antique violet mint lemon 39 5716.950000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4341.530000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 3135.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 +Manufacturer#5 almond antique medium spring khaki 6 5882.970000000001 +Manufacturer#5 almond antique sky peru orange 2 4271.3099999999995 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2482.58 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 +PREHOOK: query: -- 43. testUnboundedFollowingForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range between current row and unbounded following) + from part distribute by p_mfgr sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 43. testUnboundedFollowingForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) as s1 over (range between current row and unbounded following) + from part distribute by p_mfgr sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique chartreuse lavender yellow 34 6403.43 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4649.67 +Manufacturer#1 almond aquamarine burnished black steel 28 3047.08 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 +Manufacturer#2 almond antique violet turquoise frosted 40 7232.9400000000005 +Manufacturer#2 almond aquamarine midnight light salmon 2 5432.24 +Manufacturer#2 almond aquamarine rose maroon antique 25 3400.26 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 +Manufacturer#3 almond antique forest lavender goldenrod 14 5860.929999999999 +Manufacturer#3 almond antique metallic orange dim 19 4670.66 +Manufacturer#3 almond antique misty red olive 1 3260.27 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 +Manufacturer#4 almond antique violet mint lemon 39 5716.950000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4341.530000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 3135.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 +Manufacturer#5 almond antique medium spring khaki 6 5882.970000000001 +Manufacturer#5 almond antique sky peru orange 2 4271.3099999999995 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2482.58 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 +PREHOOK: query: -- 44. testOverNoPartitionSingleAggregate +select p_name, p_retailprice, +avg(p_retailprice) over() +from part +order by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 44. testOverNoPartitionSingleAggregate +select p_name, p_retailprice, +avg(p_retailprice) over() +from part +order by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: Lineage: part_1.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_1.s SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.cud SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.dr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.fv1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.r SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s1 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_2.s2 SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.c SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.ca SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.fv SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_mfgr SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_name SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +POSTHOOK: Lineage: part_3.p_size SCRIPT [(part)part.FieldSchema(name:p_partkey, type:int, comment:null), (part)part.FieldSchema(name:p_name, type:string, comment:null), (part)part.FieldSchema(name:p_mfgr, type:string, comment:null), (part)part.FieldSchema(name:p_brand, type:string, comment:null), (part)part.FieldSchema(name:p_type, type:string, comment:null), (part)part.FieldSchema(name:p_size, type:int, comment:null), (part)part.FieldSchema(name:p_container, type:string, comment:null), (part)part.FieldSchema(name:p_retailprice, type:double, comment:null), (part)part.FieldSchema(name:p_comment, type:string, comment:null), (part)part.FieldSchema(name:BLOCK__OFFSET__INSIDE__FILE, type:bigint, comment:), (part)part.FieldSchema(name:INPUT__FILE__NAME, type:string, comment:), ] +almond antique blue firebrick mint 1789.69 1546.7784615384621 +almond antique burnished rose metallic 1173.15 1546.7784615384621 +almond antique burnished rose metallic 1173.15 1546.7784615384621 +almond antique chartreuse khaki white 1671.68 1546.7784615384621 +almond antique chartreuse lavender yellow 1753.76 1546.7784615384621 +almond antique forest lavender goldenrod 1190.27 1546.7784615384621 +almond antique gainsboro frosted violet 1620.67 1546.7784615384621 +almond antique medium spring khaki 1611.66 1546.7784615384621 +almond antique metallic orange dim 1410.39 1546.7784615384621 +almond antique misty red olive 1922.98 1546.7784615384621 +almond antique olive coral navajo 1337.29 1546.7784615384621 +almond antique salmon chartreuse burlywood 1602.59 1546.7784615384621 +almond antique sky peru orange 1788.73 1546.7784615384621 +almond antique violet chocolate turquoise 1690.68 1546.7784615384621 +almond antique violet mint lemon 1375.42 1546.7784615384621 +almond antique violet turquoise frosted 1800.7 1546.7784615384621 +almond aquamarine burnished black steel 1414.42 1546.7784615384621 +almond aquamarine dodger light gainsboro 1018.1 1546.7784615384621 +almond aquamarine floral ivory bisque 1206.26 1546.7784615384621 +almond aquamarine midnight light salmon 2031.98 1546.7784615384621 +almond aquamarine pink moccasin thistle 1632.66 1546.7784615384621 +almond aquamarine rose maroon antique 1698.66 1546.7784615384621 +almond aquamarine sandy cyan gainsboro 1701.6 1546.7784615384621 +almond aquamarine yellow dodger mint 1844.92 1546.7784615384621 +almond azure aquamarine papaya violet 1290.35 1546.7784615384621 +almond azure blanched chiffon midnight 1464.48 1546.7784615384621 diff --git ql/src/test/results/clientpositive/windowing_columnPruning.q.out ql/src/test/results/clientpositive/windowing_columnPruning.q.out new file mode 100644 index 0000000..51a38ef --- /dev/null +++ ql/src/test/results/clientpositive/windowing_columnPruning.q.out @@ -0,0 +1,156 @@ +PREHOOK: query: DROP TABLE part +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE part +POSTHOOK: type: DROPTABLE +PREHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +PREHOOK: type: LOAD +PREHOOK: Output: default@part +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part +POSTHOOK: type: LOAD +POSTHOOK: Output: default@part +PREHOOK: query: -- 1. testQueryLevelPartitionColsNotInSelect +select p_size, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +distribute by p_mfgr sort by p_name +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 1. testQueryLevelPartitionColsNotInSelect +select p_size, +sum(p_retailprice) as s1 over (rows between unbounded preceding and current row) +from part +distribute by p_mfgr sort by p_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 +PREHOOK: query: -- 2. testWindowPartitionColsNotInSelect +select p_size, +sum(p_retailprice) as s1 over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) +from part +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 2. testWindowPartitionColsNotInSelect +select p_size, +sum(p_retailprice) as s1 over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) +from part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 +PREHOOK: query: -- 3. testHavingColNotInSelect +select p_mfgr, +sum(p_retailprice) as s1 over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) +from part +having p_size > 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@part +#### A masked pattern was here #### +POSTHOOK: query: -- 3. testHavingColNotInSelect +select p_mfgr, +sum(p_retailprice) as s1 over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) +from part +having p_size > 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part +#### A masked pattern was here #### +Manufacturer#1 4100.06 +Manufacturer#1 5702.650000000001 +Manufacturer#1 7117.070000000001 +Manufacturer#1 8749.730000000001 +Manufacturer#2 1690.68 +Manufacturer#2 3491.38 +Manufacturer#2 7222.02 +Manufacturer#2 8923.62 +Manufacturer#3 1671.68 +Manufacturer#3 2861.95 +Manufacturer#3 4272.34 +Manufacturer#3 7532.61 +Manufacturer#4 1620.67 +Manufacturer#4 2996.09 +Manufacturer#4 4202.35 +Manufacturer#4 6047.27 +Manufacturer#4 7337.620000000001 +Manufacturer#5 1789.69 +Manufacturer#5 3401.3500000000004 +Manufacturer#5 6208.18 +Manufacturer#5 7672.66