You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by na...@apache.org on 2016/06/29 17:54:50 UTC

[4/5] incubator-quickstep git commit: Added PhysicalGenerator support for Window Aggregation.

Added PhysicalGenerator support for Window Aggregation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/5c4e8dbf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/5c4e8dbf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/5c4e8dbf

Branch: refs/heads/hashtable_collector
Commit: 5c4e8dbf19dd604fdb728367031dcf33d345240c
Parents: f6f0c8b
Author: shixuan-fan <sh...@apache.org>
Authored: Tue Jun 28 16:25:52 2016 +0000
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jun 28 22:50:17 2016 -0700

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/ExecutionGenerator.cpp          |   4 +
 query_optimizer/physical/CMakeLists.txt         |  15 +-
 query_optimizer/physical/PhysicalType.hpp       |   3 +-
 query_optimizer/physical/WindowAggregate.cpp    |  66 +++++
 query_optimizer/physical/WindowAggregate.hpp    | 133 ++++++++++
 query_optimizer/strategy/CMakeLists.txt         |   5 +-
 query_optimizer/strategy/OneToOne.cpp           |  11 +-
 .../tests/physical_generator/Select.test        | 264 +++++++++++++++++++
 9 files changed, 495 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 8f08130..8912414 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -109,6 +109,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_UpdateTable
+                      quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_relationaloperators_AggregationOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_CreateIndexOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index f9fd742..45f5f78 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -83,6 +83,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
+#include "query_optimizer/physical/WindowAggregate.hpp"
 #include "relational_operators/AggregationOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
 #include "relational_operators/CreateIndexOperator.hpp"
@@ -282,6 +283,9 @@ void ExecutionGenerator::generatePlanInternal(
     case P::PhysicalType::kUpdateTable:
       return convertUpdateTable(
           std::static_pointer_cast<const P::UpdateTable>(physical_plan));
+    case P::PhysicalType::kWindowAggregate:
+      THROW_SQL_ERROR()
+          << "Window aggregate function is not supported yet :(";
     default:
       LOG(FATAL) << "Unknown physical plan node "
                  << physical_plan->getShortString();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index ea3752d..1ad30e4 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -39,6 +39,7 @@ add_library(quickstep_queryoptimizer_physical_TableGenerator ../../empty_src.cpp
 add_library(quickstep_queryoptimizer_physical_TableReference TableReference.cpp TableReference.hpp)
 add_library(quickstep_queryoptimizer_physical_TopLevelPlan TopLevelPlan.cpp TopLevelPlan.hpp)
 add_library(quickstep_queryoptimizer_physical_UpdateTable UpdateTable.cpp UpdateTable.hpp)
+add_library(quickstep_queryoptimizer_physical_WindowAggregate WindowAggregate.cpp WindowAggregate.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_queryoptimizer_physical_Aggregate
@@ -250,6 +251,17 @@ target_link_libraries(quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_WindowAggregate
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
 
 # Module all-in-one library:
 add_library(quickstep_queryoptimizer_physical ../../empty_src.cpp OptimizerPhysicalModule.hpp)
@@ -276,4 +288,5 @@ target_link_libraries(quickstep_queryoptimizer_physical
                       quickstep_queryoptimizer_physical_TableGenerator
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
-                      quickstep_queryoptimizer_physical_UpdateTable)
+                      quickstep_queryoptimizer_physical_UpdateTable
+                      quickstep_queryoptimizer_physical_WindowAggregate)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index b036557..14f8e1a 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -47,7 +47,8 @@ enum class PhysicalType {
   kTableGenerator,
   kTableReference,
   kTopLevelPlan,
-  kUpdateTable
+  kUpdateTable,
+  kWindowAggregate
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/WindowAggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/WindowAggregate.cpp b/query_optimizer/physical/WindowAggregate.cpp
new file mode 100644
index 0000000..2927107
--- /dev/null
+++ b/query_optimizer/physical/WindowAggregate.cpp
@@ -0,0 +1,66 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed 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.
+ **/
+
+#include "query_optimizer/physical/WindowAggregate.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "utility/Cast.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+std::vector<E::AttributeReferencePtr> WindowAggregate::getOutputAttributes() const {
+  std::vector<E::AttributeReferencePtr> output_attributes(
+      input_->getOutputAttributes());
+  output_attributes.push_back(E::ToRef(window_aggregate_expression_));
+  return output_attributes;
+}
+
+std::vector<E::AttributeReferencePtr> WindowAggregate::getReferencedAttributes()
+    const {
+  return window_aggregate_expression_->getReferencedAttributes();
+}
+
+void WindowAggregate::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  non_container_child_field_names->push_back("input");
+  non_container_child_fields->push_back(input_);
+
+  non_container_child_field_names->push_back("window_aggregate_expression");
+  non_container_child_fields->push_back(window_aggregate_expression_);
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/physical/WindowAggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/WindowAggregate.hpp b/query_optimizer/physical/WindowAggregate.hpp
new file mode 100644
index 0000000..4b17d07
--- /dev/null
+++ b/query_optimizer/physical/WindowAggregate.hpp
@@ -0,0 +1,133 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerLogical
+ *  @{
+ */
+
+class WindowAggregate;
+typedef std::shared_ptr<const WindowAggregate> WindowAggregatePtr;
+
+/**
+ * @brief Window Aggregate operator that computes window aggregate expressions.
+ */
+class WindowAggregate : public Physical {
+ public:
+  PhysicalType getPhysicalType() const override {
+    return PhysicalType::kWindowAggregate;
+  }
+
+  std::string getName() const override { return "WindowAggregate"; }
+
+  /**
+   * @return The input physical node.
+   */
+  const PhysicalPtr input() const { return input_; }
+
+  /**
+   * @return Window aggregate expression.
+   */
+  inline const expressions::AliasPtr window_aggregate_expression() const {
+    return window_aggregate_expression_;
+  }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override {
+    DCHECK_EQ(getNumChildren(), new_children.size());
+    return Create(new_children[0], window_aggregate_expression_);
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override;
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  bool maybeCopyWithPrunedExpressions(
+      const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+      PhysicalPtr *output) const override {
+    // The project expressions for an WindowAggregate cannot be changed.
+    return false;
+  }
+
+  /**
+   * @brief Creates an WindowAggregate physical node.
+   *
+   * @param input The input node.
+   * @param window_aggregate_expression The window aggregate expression.
+   * @return An immutable WindowAggregate node.
+   */
+  static WindowAggregatePtr Create(
+      const PhysicalPtr &input,
+      const expressions::AliasPtr &window_aggregate_expression) {
+    return WindowAggregatePtr(new WindowAggregate(input,
+                                                  window_aggregate_expression));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  WindowAggregate(
+      const PhysicalPtr &input,
+      const expressions::AliasPtr &window_aggregate_expression)
+      : input_(input),
+        window_aggregate_expression_(window_aggregate_expression) {
+    addChild(input_);
+  }
+
+  const PhysicalPtr input_;
+  const expressions::AliasPtr window_aggregate_expression_;
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregate);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_WINDOW_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/strategy/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/CMakeLists.txt b/query_optimizer/strategy/CMakeLists.txt
index 84e151e..517bea3 100644
--- a/query_optimizer/strategy/CMakeLists.txt
+++ b/query_optimizer/strategy/CMakeLists.txt
@@ -89,6 +89,7 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_logical_TableReference
                       quickstep_queryoptimizer_logical_TopLevelPlan
                       quickstep_queryoptimizer_logical_UpdateTable
+                      quickstep_queryoptimizer_logical_WindowAggregate
                       quickstep_queryoptimizer_physical_CopyFrom
                       quickstep_queryoptimizer_physical_CreateIndex
                       quickstep_queryoptimizer_physical_CreateTable
@@ -104,9 +105,9 @@ target_link_libraries(quickstep_queryoptimizer_strategy_OneToOne
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_UpdateTable
+                      quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_queryoptimizer_strategy_Strategy
-                      quickstep_utility_Macros
-                      quickstep_utility_SqlError)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_strategy_Selection
                       glog
                       quickstep_queryoptimizer_LogicalToPhysicalMapper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index f49a25c..e9a8897 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -41,6 +41,7 @@
 #include "query_optimizer/logical/TableReference.hpp"
 #include "query_optimizer/logical/TopLevelPlan.hpp"
 #include "query_optimizer/logical/UpdateTable.hpp"
+#include "query_optimizer/logical/WindowAggregate.hpp"
 #include "query_optimizer/physical/CopyFrom.hpp"
 #include "query_optimizer/physical/CreateIndex.hpp"
 #include "query_optimizer/physical/CreateTable.hpp"
@@ -55,7 +56,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/UpdateTable.hpp"
-#include "utility/SqlError.hpp"
+#include "query_optimizer/physical/WindowAggregate.hpp"
 
 namespace quickstep {
 namespace optimizer {
@@ -210,8 +211,12 @@ bool OneToOne::generatePlan(const L::LogicalPtr &logical_input,
       return true;
     }
     case L::LogicalType::kWindowAggregate: {
-      THROW_SQL_ERROR()
-          << "Window aggregate function is not supported currently :(";
+      const L::WindowAggregatePtr window_aggregate =
+          std::static_pointer_cast<const L::WindowAggregate>(logical_input);
+      *physical_output = P::WindowAggregate::Create(
+          physical_mapper_->createOrGetPhysicalFromLogical(window_aggregate->input()),
+          window_aggregate->window_aggregate_expression());
+      return true;
     }
     default:
       return false;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5c4e8dbf/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index 3365206..d99916c 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -2876,3 +2876,267 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=5,name=x,relation=,type=Int]
   +-AttributeReference[id=6,name=y,relation=,type=Int]
+==
+
+# Window Aggregate Function Test.
+SELECT avg(int_col) OVER w FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col DESC NULLS LAST
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,false],nulls_first=[false,false]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=AVG,window_name=w,is_ascending=[false],
+| |     nulls_first=[false],frame_mode=row,num_preceding=-1,num_following=0]
+| |     +-arguments=
+| |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| |     +-order_by=
+| |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,false],nulls_first=[false,false]]
+| | | +-input=TableReference[relation=Test,alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_attributes=
+| | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=AVG,window_name=w,is_ascending=[false],
+| |     nulls_first=[false],frame_mode=row,num_preceding=-1,num_following=0]
+| |     +-arguments=
+| |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| |     +-order_by=
+| |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=avg(int_col),relation=,type=Double NULL]
+==
+
+SELECT int_col, sum(float_col) OVER
+(PARTITION BY vchar_col, long_col
+ ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
+ RANGE BETWEEN 3 PRECEDING AND 3 FOLLOWING)
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,true,false,true],
+| | | nulls_first=[false,false,false,true]]
+| | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_expressions=
+| | |   +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   | type=VarChar(20) NULL]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=SUM,window_name=,
+| |     is_ascending=[false,true],nulls_first=[false,true],frame_mode=range,
+| |     num_preceding=3,num_following=3]
+| |     +-arguments=
+| |     | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| |     | | type=VarChar(20) NULL]
+| |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |     +-order_by=
+| |       +-AttributeReference[id=3,name=double_col,relation=test,
+| |       | type=Double NULL]
+| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| +-project_list=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-Alias[id=6,name=,alias=sum(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=,alias=sum(float_col),relation=,
+    type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=WindowAggregate
+| | +-input=Sort[is_ascending=[true,true,false,true],
+| | | nulls_first=[false,false,false,true]]
+| | | +-input=TableReference[relation=Test,alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-sort_attributes=
+| | |   +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   | type=VarChar(20) NULL]
+| | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| |   relation=$window_aggregate,type=Double NULL]
+| |   +-WindowAggregateFunction[function=SUM,window_name=,
+| |     is_ascending=[false,true],nulls_first=[false,true],frame_mode=range,
+| |     num_preceding=3,num_following=3]
+| |     +-arguments=
+| |     | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |     +-partition_by=
+| |     | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| |     | | type=VarChar(20) NULL]
+| |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |     +-order_by=
+| |       +-AttributeReference[id=3,name=double_col,relation=test,
+| |       | type=Double NULL]
+| |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| +-project_expressions=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   +-Alias[id=6,name=,alias=sum(float_col),relation=,type=Double NULL]
+|     +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+|       relation=$window_aggregate,type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+  +-AttributeReference[id=6,name=,alias=sum(float_col),relation=,
+    type=Double NULL]
+==
+
+SELECT sum(avg(int_col) OVER w) FROM test
+WINDOW w AS
+(PARTITION BY char_col
+ ORDER BY long_col
+ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=WindowAggregate
+| | | +-input=Sort[is_ascending=[true,true],nulls_first=[false,false]]
+| | | | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | | +-AttributeReference[id=3,name=double_col,relation=test,
+| | | | | | type=Double NULL]
+| | | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | | |   type=VarChar(20) NULL]
+| | | | +-sort_expressions=
+| | | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| | |   relation=$window_aggregate,type=Double NULL]
+| | |   +-WindowAggregateFunction[function=AVG,window_name=w,
+| | |     is_ascending=[true],nulls_first=[false],frame_mode=row,
+| | |     num_preceding=-1,num_following=0]
+| | |     +-arguments=
+| | |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-partition_by=
+| | |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |     +-order_by=
+| | |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+| |         relation=$window_aggregate,type=Double NULL]
+| +-project_list=
+|   +-Alias[id=7,name=,alias=sum(avg(int_col)),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=sum(avg(int_col)),relation=,
+    type=Double NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Aggregate
+| | +-input=WindowAggregate
+| | | +-input=Sort[is_ascending=[true,true],nulls_first=[false,false]]
+| | | | +-input=TableReference[relation=Test,alias=test]
+| | | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | | +-AttributeReference[id=3,name=double_col,relation=test,
+| | | | | | type=Double NULL]
+| | | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | | |   type=VarChar(20) NULL]
+| | | | +-sort_attributes=
+| | | |   +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-window_aggregate_expression=Alias[id=6,name=,alias=$window_aggregate0,
+| | |   relation=$window_aggregate,type=Double NULL]
+| | |   +-WindowAggregateFunction[function=AVG,window_name=w,
+| | |     is_ascending=[true],nulls_first=[false],frame_mode=row,
+| | |     num_preceding=-1,num_following=0]
+| | |     +-arguments=
+| | |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-partition_by=
+| | |     | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | |     +-order_by=
+| | |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=7,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-AttributeReference[id=6,name=,alias=$window_aggregate0,
+| |         relation=$window_aggregate,type=Double NULL]
+| +-project_expressions=
+|   +-Alias[id=7,name=,alias=sum(avg(int_col)),relation=,type=Double NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate0,relation=$aggregate,
+|       type=Double NULL]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=sum(avg(int_col)),relation=,
+    type=Double NULL]
+==