You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2016/07/30 06:35:32 UTC

[01/10] incubator-quickstep git commit: - Supported ROWS mode for AVG window aggregation. - Created WindowAggregateFunctions in expressions/window_aggregation. - Created WindowAggregationHandle for AVG to actually do the calculation. - Other functions wi [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/LIP-for-tpch 2242550d3 -> 97d8dca85 (forced update)


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp b/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
new file mode 100644
index 0000000..c044a98
--- /dev/null
+++ b/expressions/window_aggregation/tests/WindowAggregationHandleAvg_unittest.cpp
@@ -0,0 +1,387 @@
+/**
+ *   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.
+ *   limitations under the License.
+ **/
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionFactory.hpp"
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "expressions/window_aggregation/WindowAggregationHandleAvg.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "types/CharType.hpp"
+#include "types/DateOperatorOverloads.hpp"
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/IntervalLit.hpp"
+#include "types/LongType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+
+namespace {
+
+  constexpr int kNumTuples = 100;
+  constexpr int kNumTuplesPerPartition = 8;
+  constexpr int kNullInterval = 25;
+  constexpr int kNumPreceding = 2;
+  constexpr int kNumFollowing = 2;
+
+}  // namespace
+
+// Attribute value could be null if set true.
+class WindowAggregationHandleAvgTest : public::testing::Test {
+ protected:
+  // Handle initialization.
+  void initializeHandle(const Type &argument_type) {
+    const WindowAggregateFunction &function =
+        WindowAggregateFunctionFactory::Get(WindowAggregationID::kAvg);
+    std::vector<const Type*> partition_key_types(1, &TypeFactory::GetType(kInt, false));
+    handle_avg_.reset(function.createHandle(std::vector<const Type*>(1, &argument_type),
+                                            std::move(partition_key_types)));
+  }
+
+  // Test canApplyToTypes().
+  static bool CanApplyToTypesTest(TypeID typeID) {
+    const Type &type = (typeID == kChar || typeID == kVarChar) ?
+        TypeFactory::GetType(typeID, static_cast<std::size_t>(10)) :
+        TypeFactory::GetType(typeID);
+
+    return WindowAggregateFunctionFactory::Get(WindowAggregationID::kAvg).canApplyToTypes(
+        std::vector<const Type*>(1, &type));
+  }
+
+  // Test resultTypeForArgumentTypes().
+  static bool ResultTypeForArgumentTypesTest(TypeID input_type_id,
+                                             TypeID output_type_id) {
+    const Type *result_type
+        = WindowAggregateFunctionFactory::Get(WindowAggregationID::kAvg).resultTypeForArgumentTypes(
+            std::vector<const Type*>(1, &TypeFactory::GetType(input_type_id)));
+    return (result_type->getTypeID() == output_type_id);
+  }
+
+  template <typename CppType>
+  static void CheckAvgValues(
+      const std::vector<CppType*> &expected,
+      const ColumnVector *actual) {
+    EXPECT_TRUE(actual->isNative());
+    const NativeColumnVector *native = static_cast<const NativeColumnVector*>(actual);
+
+    EXPECT_EQ(expected.size(), native->size());
+    for (std::size_t i = 0; i < expected.size(); ++i) {
+      if (expected[i] == nullptr) {
+        EXPECT_TRUE(native->getTypedValue(i).isNull());
+      } else {
+        EXPECT_EQ(*expected[i], native->getTypedValue(i).getLiteral<CppType>());
+      }
+    }
+  }
+
+  // Static templated methods for set a meaningful value to data types.
+  template <typename CppType>
+  static void SetDataType(int value, CppType *data) {
+    *data = value;
+  }
+
+  template <typename GenericType, typename OutputType = DoubleType>
+  void checkWindowAggregationAvgGeneric() {
+    const GenericType &type = GenericType::Instance(true);
+    initializeHandle(type);
+
+    // Create argument, partition key and cpptype vectors.
+    std::vector<typename GenericType::cpptype*> argument_cpp_vector;
+    argument_cpp_vector.reserve(kNumTuples);
+    ColumnVector *argument_type_vector =
+        createArgumentGeneric<GenericType>(&argument_cpp_vector);
+    NativeColumnVector *partition_key_vector =
+        new NativeColumnVector(IntType::InstanceNonNullable(), kNumTuples + 2);
+
+    for (int i = 0; i < kNumTuples; ++i) {
+      partition_key_vector->appendTypedValue(TypedValue(i / kNumTuplesPerPartition));
+    }
+
+    // Create tuple ValueAccessor.
+    ColumnVectorsValueAccessor *tuple_accessor = new ColumnVectorsValueAccessor();
+    tuple_accessor->addColumn(partition_key_vector);
+    tuple_accessor->addColumn(argument_type_vector);
+
+    // Test UNBOUNDED PRECEDING AND CURRENT ROW.
+    checkAccumulate<GenericType, OutputType>(tuple_accessor,
+                                             argument_type_vector,
+                                             argument_cpp_vector);
+    // Test kNumPreceding PRECEDING AND kNumFollowing FOLLOWING.
+    checkSlidingWindow<GenericType, OutputType>(tuple_accessor,
+                                                argument_type_vector,
+                                                argument_cpp_vector);
+  }
+
+  template <typename GenericType>
+  ColumnVector *createArgumentGeneric(
+      std::vector<typename GenericType::cpptype*> *argument_cpp_vector) {
+    const GenericType &type = GenericType::Instance(true);
+    NativeColumnVector *column = new NativeColumnVector(type, kNumTuples);
+
+    for (int i = 0; i < kNumTuples; ++i) {
+      // Insert a NULL every kNullInterval tuples.
+      if (i % kNullInterval == 0) {
+        argument_cpp_vector->push_back(nullptr);
+        column->appendTypedValue(type.makeNullValue());
+        continue;
+      }
+
+      typename GenericType::cpptype *val = new typename GenericType::cpptype;
+
+      if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
+        SetDataType(i - 10, val);
+      } else {
+        SetDataType(static_cast<float>(i - 10) / 10, val);
+      }
+
+      column->appendTypedValue(type.makeValue(val));
+      argument_cpp_vector->push_back(val);
+    }
+
+    return column;
+  }
+
+  template <typename GenericType, typename OutputType>
+  void checkAccumulate(ColumnVectorsValueAccessor *tuple_accessor,
+                       ColumnVector *argument_type_vector,
+                       const std::vector<typename GenericType::cpptype*> &argument_cpp_vector) {
+    std::vector<ColumnVector*> arguments;
+    arguments.push_back(argument_type_vector);
+    // The partition key index is 0.
+    std::vector<attribute_id> partition_key(1, 0);
+
+    ColumnVector *result =
+        handle_avg_->calculate(tuple_accessor,
+                               std::move(arguments),
+                               partition_key,
+                               true  /* is_row */,
+                               -1  /* num_preceding: UNBOUNDED PRECEDING */,
+                               0  /* num_following: CURRENT ROW */);
+
+    // Get the cpptype result.
+    std::vector<typename OutputType::cpptype*> result_cpp_vector;
+    typename GenericType::cpptype sum;
+    int count;
+    for (std::size_t i = 0; i < argument_cpp_vector.size(); ++i) {
+      // Start of new partition
+      if (i % kNumTuplesPerPartition == 0) {
+        SetDataType(0, &sum);
+        count = 0;
+      }
+
+      typename GenericType::cpptype *value = argument_cpp_vector[i];
+      if (value != nullptr) {
+        sum += *value;
+        count++;
+      }
+
+      if (count == 0) {
+        result_cpp_vector.push_back(nullptr);
+      } else {
+        typename OutputType::cpptype *result_cpp_value =
+            new typename OutputType::cpptype;
+        *result_cpp_value = static_cast<typename OutputType::cpptype>(sum) / count;
+        result_cpp_vector.push_back(result_cpp_value);
+      }
+    }
+
+    CheckAvgValues(result_cpp_vector, result);
+  }
+
+  template <typename GenericType, typename OutputType>
+  void checkSlidingWindow(ColumnVectorsValueAccessor *tuple_accessor,
+                          ColumnVector *argument_type_vector,
+                          const std::vector<typename GenericType::cpptype*> &argument_cpp_vector) {
+    std::vector<ColumnVector*> arguments;
+    arguments.push_back(argument_type_vector);
+    // The partition key index is 0.
+    std::vector<attribute_id> partition_key(1, 0);
+
+    ColumnVector *result =
+        handle_avg_->calculate(tuple_accessor,
+                               std::move(arguments),
+                               partition_key,
+                               true  /* is_row */,
+                               kNumPreceding,
+                               kNumFollowing);
+
+    // Get the cpptype result.
+    // For each value, calculate all surrounding values in the window.
+    std::vector<typename OutputType::cpptype*> result_cpp_vector;
+
+    for (std::size_t i = 0; i < argument_cpp_vector.size(); ++i) {
+      typename GenericType::cpptype sum;
+      SetDataType(0, &sum);
+      int count = 0;
+
+      if (argument_cpp_vector[i] != nullptr) {
+        sum += *argument_cpp_vector[i];
+        count++;
+      }
+
+      for (std::size_t precede = 1; precede <= kNumPreceding; ++precede) {
+        // Not the same partition.
+        if (i / kNumTuplesPerPartition != (i - precede) / kNumTuplesPerPartition ||
+            i < precede) {
+          break;
+        }
+
+        if (argument_cpp_vector[i - precede] != nullptr) {
+          sum += *argument_cpp_vector[i - precede];
+          count++;
+        }
+      }
+
+      for (int follow = 1; follow <= kNumPreceding; ++follow) {
+        // Not the same partition.
+        if (i / kNumTuplesPerPartition != (i + follow) / kNumTuplesPerPartition ||
+            i + follow >= kNumTuples) {
+          break;
+        }
+
+        if (argument_cpp_vector[i + follow] != nullptr) {
+          sum += *argument_cpp_vector[i + follow];
+          count++;
+        }
+      }
+
+      if (count == 0) {
+        result_cpp_vector.push_back(nullptr);
+      } else {
+        typename OutputType::cpptype *result_cpp_value =
+            new typename OutputType::cpptype;
+        *result_cpp_value = static_cast<typename OutputType::cpptype>(sum) / count;
+        result_cpp_vector.push_back(result_cpp_value);
+      }
+    }
+
+    CheckAvgValues(result_cpp_vector, result);
+  }
+
+  std::unique_ptr<WindowAggregationHandle> handle_avg_;
+};
+
+template <>
+void WindowAggregationHandleAvgTest::CheckAvgValues<double>(
+    const std::vector<double*> &expected,
+    const ColumnVector *actual) {
+  EXPECT_TRUE(actual->isNative());
+  const NativeColumnVector *native = static_cast<const NativeColumnVector*>(actual);
+
+  EXPECT_EQ(expected.size(), native->size());
+  for (std::size_t i = 0; i < expected.size(); ++i) {
+    if (expected[i] == nullptr) {
+      EXPECT_TRUE(native->getTypedValue(i).isNull());
+    } else {
+      EXPECT_EQ(*expected[i], native->getTypedValue(i).getLiteral<double>());
+    }
+  }
+}
+
+template <>
+void WindowAggregationHandleAvgTest::SetDataType<DatetimeIntervalLit>(
+    int value, DatetimeIntervalLit *data) {
+  data->interval_ticks = value;
+}
+
+template <>
+void WindowAggregationHandleAvgTest::SetDataType<YearMonthIntervalLit>(
+    int value, YearMonthIntervalLit *data) {
+  data->months = value;
+}
+
+typedef WindowAggregationHandleAvgTest WindowAggregationHandleAvgDeathTest;
+
+TEST_F(WindowAggregationHandleAvgTest, IntTypeTest) {
+  checkWindowAggregationAvgGeneric<IntType>();
+}
+
+TEST_F(WindowAggregationHandleAvgTest, LongTypeTest) {
+  checkWindowAggregationAvgGeneric<LongType>();
+}
+
+TEST_F(WindowAggregationHandleAvgTest, FloatTypeTest) {
+  checkWindowAggregationAvgGeneric<FloatType>();
+}
+
+TEST_F(WindowAggregationHandleAvgTest, DoubleTypeTest) {
+  checkWindowAggregationAvgGeneric<DoubleType>();
+}
+
+TEST_F(WindowAggregationHandleAvgTest, DatetimeIntervalTypeTest) {
+  checkWindowAggregationAvgGeneric<DatetimeIntervalType, DatetimeIntervalType>();
+}
+
+TEST_F(WindowAggregationHandleAvgTest, YearMonthIntervalTypeTest) {
+  checkWindowAggregationAvgGeneric<YearMonthIntervalType, YearMonthIntervalType>();
+}
+
+#ifdef QUICKSTEP_DEBUG
+TEST_F(WindowAggregationHandleAvgDeathTest, CharTypeTest) {
+  const Type &type = CharType::Instance(true, 10);
+  EXPECT_DEATH(initializeHandle(type), "");
+}
+
+TEST_F(WindowAggregationHandleAvgDeathTest, VarTypeTest) {
+  const Type &type = VarCharType::Instance(true, 10);
+  EXPECT_DEATH(initializeHandle(type), "");
+}
+#endif
+
+TEST_F(WindowAggregationHandleAvgDeathTest, canApplyToTypeTest) {
+  EXPECT_TRUE(CanApplyToTypesTest(kInt));
+  EXPECT_TRUE(CanApplyToTypesTest(kLong));
+  EXPECT_TRUE(CanApplyToTypesTest(kFloat));
+  EXPECT_TRUE(CanApplyToTypesTest(kDouble));
+  EXPECT_FALSE(CanApplyToTypesTest(kChar));
+  EXPECT_FALSE(CanApplyToTypesTest(kVarChar));
+  EXPECT_FALSE(CanApplyToTypesTest(kDatetime));
+  EXPECT_TRUE(CanApplyToTypesTest(kDatetimeInterval));
+  EXPECT_TRUE(CanApplyToTypesTest(kYearMonthInterval));
+}
+
+TEST_F(WindowAggregationHandleAvgDeathTest, ResultTypeForArgumentTypeTest) {
+  EXPECT_TRUE(ResultTypeForArgumentTypesTest(kInt, kDouble));
+  EXPECT_TRUE(ResultTypeForArgumentTypesTest(kLong, kDouble));
+  EXPECT_TRUE(ResultTypeForArgumentTypesTest(kFloat, kDouble));
+  EXPECT_TRUE(ResultTypeForArgumentTypesTest(kDouble, kDouble));
+  EXPECT_TRUE(ResultTypeForArgumentTypesTest(kDatetimeInterval, kDatetimeInterval));
+  EXPECT_TRUE(ResultTypeForArgumentTypesTest(kYearMonthInterval, kYearMonthInterval));
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 7e53b9d..a56b714 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -69,6 +69,8 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_expressions_scalar_ScalarAttribute
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryoptimizer_ExecutionHeuristics

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 43d63f9..ce21ade 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -48,6 +48,8 @@
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "expressions/scalar/ScalarAttribute.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.pb.h"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/ExecutionHeuristics.hpp"
@@ -1652,7 +1654,7 @@ void ExecutionGenerator::convertWindowAggregate(
   // Get input.
   const CatalogRelationInfo *input_relation_info =
       findRelationInfoOutputByPhysical(physical_plan->input());
-  window_aggr_state_proto->set_relation_id(input_relation_info->relation->getID());
+  window_aggr_state_proto->set_input_relation_id(input_relation_info->relation->getID());
 
   // Get window aggregate function expression.
   const E::AliasPtr &named_window_aggregate_expression =
@@ -1713,6 +1715,7 @@ void ExecutionGenerator::convertWindowAggregate(
   const QueryPlan::DAGNodeIndex window_aggregation_operator_index =
       execution_plan_->addRelationalOperator(
           new WindowAggregationOperator(query_handle_->query_id(),
+                                        *input_relation_info->relation,
                                         *output_relation,
                                         window_aggr_state_index,
                                         insert_destination_index));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index 08d7df5..d12644a 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -304,7 +304,7 @@ target_link_libraries(quickstep_queryoptimizer_expressions_UnaryExpression
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_WindowAggregateFunction
                       glog
-                      quickstep_expressions_aggregation_AggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
                       quickstep_queryoptimizer_OptimizerTree
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_Expression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/expressions/WindowAggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/WindowAggregateFunction.cpp b/query_optimizer/expressions/WindowAggregateFunction.cpp
index 7b1f304..be5db59 100644
--- a/query_optimizer/expressions/WindowAggregateFunction.cpp
+++ b/query_optimizer/expressions/WindowAggregateFunction.cpp
@@ -22,7 +22,7 @@
 #include <utility>
 #include <vector>
 
-#include "expressions/aggregation/AggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/PatternMatcher.hpp"
@@ -59,7 +59,7 @@ const Type& WindowAggregateFunction::getValueType() const {
 }
 
 WindowAggregateFunctionPtr WindowAggregateFunction::Create(
-    const ::quickstep::AggregateFunction &window_aggregate,
+    const ::quickstep::WindowAggregateFunction &window_aggregate,
     const std::vector<ScalarPtr> &arguments,
     const WindowInfo &window_info,
     const std::string &window_name,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/expressions/WindowAggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/WindowAggregateFunction.hpp b/query_optimizer/expressions/WindowAggregateFunction.hpp
index 0bee28f..0cc80df 100644
--- a/query_optimizer/expressions/WindowAggregateFunction.hpp
+++ b/query_optimizer/expressions/WindowAggregateFunction.hpp
@@ -33,8 +33,8 @@
 
 namespace quickstep {
 
-class AggregateFunction;
 class Type;
+class WindowAggregateFunction;
 
 namespace optimizer {
 namespace expressions {
@@ -140,7 +140,7 @@ class WindowAggregateFunction : public Expression {
    * @return The WindowAggregateFunction singleton (from the expression system)
    *         for this node.
    **/
-  inline const ::quickstep::AggregateFunction& window_aggregate() const {
+  inline const ::quickstep::WindowAggregateFunction& window_aggregate() const {
     return window_aggregate_;
   }
 
@@ -185,7 +185,7 @@ class WindowAggregateFunction : public Expression {
    * @param is_distinct Whether this is a DISTINCT aggregation.
    * @return A new AggregateFunctionPtr.
    **/
-  static WindowAggregateFunctionPtr Create(const ::quickstep::AggregateFunction &window_aggregate,
+  static WindowAggregateFunctionPtr Create(const ::quickstep::WindowAggregateFunction &window_aggregate,
                                            const std::vector<ScalarPtr> &arguments,
                                            const WindowInfo &window_info,
                                            const std::string &window_name,
@@ -209,7 +209,7 @@ class WindowAggregateFunction : public Expression {
    * @param window_info The window info of the window aggregate function.
    * @param is_distinct Indicates whether this is a DISTINCT aggregation.
    */
-  WindowAggregateFunction(const ::quickstep::AggregateFunction &window_aggregate,
+  WindowAggregateFunction(const ::quickstep::WindowAggregateFunction &window_aggregate,
                           const std::vector<ScalarPtr> &arguments,
                           const WindowInfo &window_info,
                           const std::string &window_name,
@@ -228,7 +228,7 @@ class WindowAggregateFunction : public Expression {
   // window_aggregate_. If it really needs to be seperated from the
   // AggregationFunction, a new class for WindowAggregationFunction should be
   // created as quickstep::WindowAggregateFunction.
-  const ::quickstep::AggregateFunction &window_aggregate_;
+  const ::quickstep::WindowAggregateFunction &window_aggregate_;
   std::vector<ScalarPtr> arguments_;
   const WindowInfo window_info_;
   const std::string window_name_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index fb75767..9313e51 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -39,6 +39,8 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_expressions_tablegenerator_GeneratorFunction
                       quickstep_expressions_tablegenerator_GeneratorFunctionFactory
                       quickstep_expressions_tablegenerator_GeneratorFunctionHandle
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionFactory
                       quickstep_parser_ParseAssignment
                       quickstep_parser_ParseBasicExpressions
                       quickstep_parser_ParseBlockProperties

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index f10378b..c224388 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -35,6 +35,8 @@
 #include "expressions/table_generator/GeneratorFunction.hpp"
 #include "expressions/table_generator/GeneratorFunctionFactory.hpp"
 #include "expressions/table_generator/GeneratorFunctionHandle.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionFactory.hpp"
 #include "parser/ParseAssignment.hpp"
 #include "parser/ParseBasicExpressions.hpp"
 #include "parser/ParseBlockProperties.hpp"
@@ -2624,11 +2626,19 @@ E::ScalarPtr Resolver::resolveFunctionCall(
         << "COUNT aggregate has both star (*) and non-star arguments.";
   }
 
-  // Try to look up the AggregateFunction by name using
-  // AggregateFunctionFactory.
-  const ::quickstep::AggregateFunction *aggregate
-      = AggregateFunctionFactory::GetByName(function_name);
-  if (aggregate == nullptr) {
+  // Try to look up the AggregateFunction/WindowAggregationFunction by name.
+  // TODO(Shixuan): We might want to create a new abstract class Function to
+  // include both AggregateFunction and WindowAggregateFunction, which will make
+  // this part of code cleaner.
+  const ::quickstep::AggregateFunction *aggregate = nullptr;
+  const ::quickstep::WindowAggregateFunction *window_aggregate = nullptr;
+  if (parse_function_call.isWindow()) {
+    window_aggregate = WindowAggregateFunctionFactory::GetByName(function_name);
+  } else {
+    aggregate = AggregateFunctionFactory::GetByName(function_name);
+  }
+
+  if (aggregate == nullptr && window_aggregate == nullptr) {
     THROW_SQL_ERROR_AT(&parse_function_call)
         << "Unrecognized function name \""
         << parse_function_call.name()->value()
@@ -2656,11 +2666,14 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   }
 
   // Make sure a naked COUNT() with no arguments wasn't specified.
-  if ((aggregate->getAggregationID() == AggregationID::kCount)
-      && (resolved_arguments.empty())
-      && (!count_star)) {
-    THROW_SQL_ERROR_AT(&parse_function_call)
-        << "COUNT aggregate requires an argument (either scalar or star (*))";
+  if ((aggregate != nullptr &&
+       aggregate->getAggregationID() == AggregationID::kCount) ||
+      (window_aggregate != nullptr &&
+       window_aggregate->getWindowAggregationID() == WindowAggregationID::kCount)) {
+    if ((resolved_arguments.empty()) && !count_star) {
+      THROW_SQL_ERROR_AT(&parse_function_call)
+          << "COUNT aggregate requires an argument (either scalar or star (*))";
+    }
   }
 
   // Resolve each of the Scalar arguments to the aggregate.
@@ -2670,7 +2683,8 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   }
 
   // Make sure that the aggregate can apply to the specified argument(s).
-  if (!aggregate->canApplyToTypes(argument_types)) {
+  if ((aggregate != nullptr && !aggregate->canApplyToTypes(argument_types))
+      || (window_aggregate != nullptr && !window_aggregate->canApplyToTypes(argument_types))) {
     THROW_SQL_ERROR_AT(&parse_function_call)
         << "Aggregate function " << aggregate->getName()
         << " can not apply to the given argument(s).";
@@ -2679,7 +2693,7 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   if (parse_function_call.isWindow()) {
     return resolveWindowAggregateFunction(parse_function_call,
                                           expression_resolution_info,
-                                          aggregate,
+                                          window_aggregate,
                                           resolved_arguments);
   }
 
@@ -2705,7 +2719,7 @@ E::ScalarPtr Resolver::resolveFunctionCall(
 E::ScalarPtr Resolver::resolveWindowAggregateFunction(
     const ParseFunctionCall &parse_function_call,
     ExpressionResolutionInfo *expression_resolution_info,
-    const ::quickstep::AggregateFunction *window_aggregate,
+    const ::quickstep::WindowAggregateFunction *window_aggregate,
     const std::vector<E::ScalarPtr> &resolved_arguments) {
   // A window aggregate function might be defined OVER a window name or a window.
   E::WindowAggregateFunctionPtr window_aggregate_function;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index f4024e9..373430c 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -23,7 +23,6 @@
 #include <unordered_set>
 #include <vector>
 
-#include "query_optimizer/expressions/AggregateFunction.hpp"
 #include "query_optimizer/expressions/Alias.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/NamedExpression.hpp"
@@ -460,14 +459,14 @@ class Resolver {
    * @param expression_resolution_info Resolution info that contains the name
    *                                   resolver and info to be updated after
    *                                   resolution.
-   * @param aggregate The aggregate function.
+   * @param aggregate The window aggregate function.
    * @param resolved_arguments The resolved arguments.
    * @return An expression in the query optimizer.
    */
   expressions::ScalarPtr resolveWindowAggregateFunction(
       const ParseFunctionCall &parse_function_call,
       ExpressionResolutionInfo *expression_resolution_info,
-      const ::quickstep::AggregateFunction *aggregate,
+      const ::quickstep::WindowAggregateFunction *aggregate,
       const std::vector<expressions::ScalarPtr> &resolved_arguments);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/query_optimizer/tests/execution_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Select.test b/query_optimizer/tests/execution_generator/Select.test
index 16127cc..30a3c39 100644
--- a/query_optimizer/tests/execution_generator/Select.test
+++ b/query_optimizer/tests/execution_generator/Select.test
@@ -953,19 +953,79 @@ WHERE double_col < 0
 ==
 
 # Window Aggregation Test.
-# Currently this is not supported, an empty table will be returned.
-SELECT avg(int_col) OVER w FROM test
+SELECT char_col, long_col, avg(long_col) OVER w FROM test
 WINDOW w AS
-(PARTITION BY char_col
- ORDER BY long_col DESC NULLS LAST
+(ORDER BY char_col DESC NULLS LAST
  ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW);
 --
-+------------------------+
-|avg(int_col)            |
-+------------------------+
-+------------------------+
++--------------------+--------------------+------------------------+
+|char_col            |long_col            |avg(long_col)           |
++--------------------+--------------------+------------------------+
+|          8 2.828427|                  64|                      64|
+|          6 2.449490|                  36|                      50|
+|          4 2.000000|                  16|      38.666666666666664|
+|         24 4.898979|                 576|                     173|
+|         22 4.690416|                 484|      235.19999999999999|
+|         20 4.472136|                 400|      262.66666666666669|
+|          2 1.414214|                   4|      225.71428571428572|
+|         18 4.242641|                 324|                     238|
+|         16 4.000000|                 256|                     240|
+|         14 3.741657|                 196|      235.59999999999999|
+|         12 3.464102|                 144|      227.27272727272728|
+|         10 3.162278|                 100|      216.66666666666666|
+|          0 0.000000|                   0|                     200|
+|         -9 3.000000|                  81|                   191.5|
+|         -7 2.645751|                  49|                     182|
+|         -5 2.236068|                  25|                172.1875|
+|         -3 1.732051|                   9|      162.58823529411765|
+|        -23 4.795832|                 529|      182.94444444444446|
+|        -21 4.582576|                 441|      196.52631578947367|
+|        -19 4.358899|                 361|                  204.75|
+|        -17 4.123106|                 289|      208.76190476190476|
+|        -15 3.872983|                 225|                   209.5|
+|        -13 3.605551|                 169|       207.7391304347826|
+|        -11 3.316625|                 121|                 204.125|
+|         -1 1.000000|                   1|                     196|
++--------------------+--------------------+------------------------+
 ==
 
+SELECT long_col, int_col, avg(int_col) OVER w FROM test
+WINDOW w AS
+(ORDER BY long_col DESC NULLS LAST
+ ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING);
+--
++--------------------+-----------+------------------------+
+|long_col            |int_col    |avg(int_col)            |
++--------------------+-----------+------------------------+
+|                 576|         24|       7.666666666666667|
+|                 529|        -23|                     0.5|
+|                 484|         22|                     0.5|
+|                 441|        -21|                  -10.25|
+|                 400|       NULL|                       0|
+|                 361|        -19|                   -9.75|
+|                 324|         18|                    -0.5|
+|                 289|        -17|     -3.3999999999999999|
+|                 256|         16|      3.2000000000000002|
+|                 225|        -15|                      -3|
+|                 196|         14|      2.7999999999999998|
+|                 169|        -13|     -2.6000000000000001|
+|                 144|         12|                     0.5|
+|                 121|        -11|                   -5.25|
+|                 100|       NULL|                       0|
+|                  81|         -9|                   -4.75|
+|                  64|          8|                    -0.5|
+|                  49|         -7|     -1.3999999999999999|
+|                  36|          6|                     1.2|
+|                  25|         -5|                      -1|
+|                  16|          4|     0.80000000000000004|
+|                   9|         -3|    -0.59999999999999998|
+|                   4|          2|                     0.5|
+|                   1|         -1|    -0.66666666666666663|
+|                   0|       NULL|                     0.5|
++--------------------+-----------+------------------------+
+==
+
+# Currently this is not supported, an empty table will be returned.
 SELECT int_col, sum(float_col) OVER
 (PARTITION BY char_col, long_col
  ORDER BY double_col DESC NULLS LAST, int_col ASC NULLS FIRST
@@ -987,5 +1047,5 @@ WINDOW w AS
 +------------------------+
 |sum(avg(int_col))       |
 +------------------------+
-|                    NULL|
+|                     -18|
 +------------------------+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 249441d..a51370b 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -434,6 +434,7 @@ target_link_libraries(quickstep_relationaloperators_WindowAggregationOperator
                       quickstep_relationaloperators_WorkOrder
                       quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
+                      quickstep_storage_WindowAggregationOperationState
                       quickstep_utility_Macros
                       tmb)                      
 target_link_libraries(quickstep_relationaloperators_WorkOrder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/relational_operators/WindowAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WindowAggregationOperator.cpp b/relational_operators/WindowAggregationOperator.cpp
index 93cb9d4..3149864 100644
--- a/relational_operators/WindowAggregationOperator.cpp
+++ b/relational_operators/WindowAggregationOperator.cpp
@@ -21,11 +21,13 @@
 
 #include <vector>
 
+#include "catalog/CatalogRelation.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
+#include "storage/WindowAggregationOperationState.hpp"
 
 #include "tmb/id_typedefs.h"
 
@@ -40,10 +42,14 @@ bool WindowAggregationOperator::getAllWorkOrders(
   DCHECK(query_context != nullptr);
 
   if (blocking_dependencies_met_ && !generated_) {
+    std::vector<block_id> relation_blocks =
+        input_relation_.getBlocksSnapshot();
+
     container->addNormalWorkOrder(
         new WindowAggregationWorkOrder(
             query_id_,
             query_context->releaseWindowAggregationState(window_aggregation_state_index_),
+            std::move(relation_blocks),
             query_context->getInsertDestination(output_destination_index_)),
         op_index_);
     generated_ = true;
@@ -67,6 +73,11 @@ serialization::WorkOrder* WindowAggregationOperator::createWorkOrderProto() {
   proto->set_query_id(query_id_);
   proto->SetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index,
                       window_aggregation_state_index_);
+
+  const std::vector<block_id> relation_blocks = input_relation_.getBlocksSnapshot();
+  for (const block_id bid : relation_blocks) {
+    proto->AddExtension(serialization::WindowAggregationWorkOrder::block_ids, bid);
+  }
   proto->SetExtension(serialization::WindowAggregationWorkOrder::insert_destination_index,
                       output_destination_index_);
 
@@ -75,8 +86,8 @@ serialization::WorkOrder* WindowAggregationOperator::createWorkOrderProto() {
 
 
 void WindowAggregationWorkOrder::execute() {
-  std::cout << "Window aggregation is not supported yet.\n"
-      << "An empty table is returned\n";
+  state_->windowAggregateBlocks(output_destination_,
+                                block_ids_);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/relational_operators/WindowAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WindowAggregationOperator.hpp b/relational_operators/WindowAggregationOperator.hpp
index f3dfd14..bd83248 100644
--- a/relational_operators/WindowAggregationOperator.hpp
+++ b/relational_operators/WindowAggregationOperator.hpp
@@ -58,16 +58,19 @@ class WindowAggregationOperator : public RelationalOperator {
    *
    * @param query_id The ID of this query.
    * @param input_relation The relation to perform aggregation over.
+   * @param output_relation The relation for output.
    * @param window_aggregation_state_index The index of WindowAggregationState
    *                                       in QueryContext.
    * @param output_destination_index The index of InsertDestination in
    *                                 QueryContext for the output.
    **/
   WindowAggregationOperator(const std::size_t query_id,
+                            const CatalogRelation &input_relation,
                             const CatalogRelation &output_relation,
                             const QueryContext::window_aggregation_state_id window_aggregation_state_index,
                             const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         window_aggregation_state_index_(window_aggregation_state_index),
         output_destination_index_(output_destination_index),
@@ -99,6 +102,7 @@ class WindowAggregationOperator : public RelationalOperator {
    **/
   serialization::WorkOrder* createWorkOrderProto();
 
+  const CatalogRelation &input_relation_;
   const CatalogRelation &output_relation_;
   const QueryContext::window_aggregation_state_id window_aggregation_state_index_;
   const QueryContext::insert_destination_id output_destination_index_;
@@ -117,43 +121,25 @@ class WindowAggregationWorkOrder : public WorkOrder {
    *
    * @param query_id The ID of this query.
    * @param state The WindowAggregationOperatorState to use.
+   * @param block_ids The blocks' id of the input relation.
    * @param output_destination The InsertDestination for output.
    **/
   WindowAggregationWorkOrder(const std::size_t query_id,
                              WindowAggregationOperationState *state,
+                             std::vector<block_id> &&block_ids,
                              InsertDestination *output_destination)
       : WorkOrder(query_id),
         state_(state),
+        block_ids_(std::move(block_ids)),
         output_destination_(output_destination)  {}
 
   ~WindowAggregationWorkOrder() override {}
 
-  /**
-   * @brief Get the pointer to WindowAggregationOperationState.
-   * @note This is a quickfix for "unused variable". After the window aggregate
-   *       functions are built, these methods might be dropped.
-   *
-   * @return A pointer to the window aggregation operation state.
-   **/
-  WindowAggregationOperationState* state() {
-    return state_;
-  }
-
-  /**
-   * @brief Get the pointer to output destination.
-   * @note This is a quickfix for "unused variable". After the window aggregate
-   *       functions are built, these methods might be dropped.
-   *
-   * @return A pointer to the output destination.
-   **/
-  InsertDestination* output_destination() {
-    return output_destination_;
-  }
-
   void execute() override;
 
  private:
   WindowAggregationOperationState *state_;
+  const std::vector<block_id> block_ids_;
   InsertDestination *output_destination_;
 
   DISALLOW_COPY_AND_ASSIGN(WindowAggregationWorkOrder);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 69dee1b..076735f 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -249,6 +249,7 @@ message WindowAggregationWorkOrder {
   extend WorkOrder {
     // All required
     optional uint32 window_aggr_state_index = 336;
-    optional int32 insert_destination_index = 337;
+    repeated fixed64 block_ids = 337;
+    optional int32 insert_destination_index = 338;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 9df66e1..582effd 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -1053,19 +1053,26 @@ target_link_libraries(quickstep_storage_WindowAggregationOperationState
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_ExpressionFactories
                       quickstep_expressions_Expressions_proto
-                      quickstep_expressions_aggregation_AggregateFunction
-                      quickstep_expressions_aggregation_AggregateFunctionFactory
-                      quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_scalar_Scalar
                       quickstep_expressions_scalar_ScalarAttribute
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionFactory
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
+                      quickstep_storage_SubBlocksReference
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
                       quickstep_storage_WindowAggregationOperationState_proto
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorUtil
+                      quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_WindowAggregationOperationState_proto
-                      quickstep_expressions_aggregation_AggregateFunction_proto
                       quickstep_expressions_Expressions_proto
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
                       ${PROTOBUF_LIBRARY})
 
 # Module all-in-one library:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/storage/WindowAggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.cpp b/storage/WindowAggregationOperationState.cpp
index a0bcc37..0cdfc1a 100644
--- a/storage/WindowAggregationOperationState.cpp
+++ b/storage/WindowAggregationOperationState.cpp
@@ -31,14 +31,21 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/ExpressionFactories.hpp"
 #include "expressions/Expressions.pb.h"
-#include "expressions/aggregation/AggregateFunction.hpp"
-#include "expressions/aggregation/AggregateFunctionFactory.hpp"
-#include "expressions/aggregation/AggregationHandle.hpp"
-#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "expressions/scalar/ScalarAttribute.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionFactory.hpp"
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "storage/InsertDestination.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/SubBlocksReference.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
 #include "storage/WindowAggregationOperationState.pb.h"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "types/containers/ColumnVectorUtil.hpp"
 
 #include "glog/logging.h"
 
@@ -46,23 +53,21 @@ namespace quickstep {
 
 WindowAggregationOperationState::WindowAggregationOperationState(
     const CatalogRelationSchema &input_relation,
-    const AggregateFunction *window_aggregate_function,
+    const WindowAggregateFunction *window_aggregate_function,
     std::vector<std::unique_ptr<const Scalar>> &&arguments,
-    std::vector<std::unique_ptr<const Scalar>> &&partition_by_attributes,
+    const std::vector<std::unique_ptr<const Scalar>> &partition_by_attributes,
     const bool is_row,
     const std::int64_t num_preceding,
     const std::int64_t num_following,
     StorageManager *storage_manager)
     : input_relation_(input_relation),
       arguments_(std::move(arguments)),
-      partition_by_attributes_(std::move(partition_by_attributes)),
       is_row_(is_row),
       num_preceding_(num_preceding),
       num_following_(num_following),
       storage_manager_(storage_manager) {
   // Get the Types of this window aggregate's arguments so that we can create an
   // AggregationHandle.
-  // TODO(Shixuan): Next step: New handles for window aggregation function.
   std::vector<const Type*> argument_types;
   for (const std::unique_ptr<const Scalar> &argument : arguments_) {
     argument_types.emplace_back(&argument->getType());
@@ -71,28 +76,18 @@ WindowAggregationOperationState::WindowAggregationOperationState(
   // Check if window aggregate function could apply to the arguments.
   DCHECK(window_aggregate_function->canApplyToTypes(argument_types));
 
+  // IDs and types of partition keys.
+  std::vector<const Type*> partition_by_types;
+  for (const std::unique_ptr<const Scalar> &partition_by_attribute : partition_by_attributes) {
+    partition_by_ids_.push_back(
+        partition_by_attribute->getAttributeIdForValueAccessor());
+    partition_by_types.push_back(&partition_by_attribute->getType());
+  }
+
   // Create the handle and initial state.
   window_aggregation_handle_.reset(
-      window_aggregate_function->createHandle(argument_types));
-  window_aggregation_state_.reset(
-      window_aggregation_handle_->createInitialState());
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  // See if all of this window aggregate's arguments are attributes in the input
-  // relation. If so, remember the attribute IDs so that we can do copy elision
-  // when actually performing the window aggregation.
-  arguments_as_attributes_.reserve(arguments_.size());
-  for (const std::unique_ptr<const Scalar> &argument : arguments_) {
-    const attribute_id argument_id = argument->getAttributeIdForValueAccessor();
-    if (argument_id == -1) {
-      arguments_as_attributes_.clear();
-      break;
-    } else {
-      DCHECK_EQ(input_relation_.getID(), argument->getRelationIdForValueAccessor());
-      arguments_as_attributes_.push_back(argument_id);
-    }
-  }
-#endif
+      window_aggregate_function->createHandle(std::move(argument_types),
+                                              std::move(partition_by_types)));
 }
 
 WindowAggregationOperationState* WindowAggregationOperationState::ReconstructFromProto(
@@ -101,10 +96,6 @@ WindowAggregationOperationState* WindowAggregationOperationState::ReconstructFro
     StorageManager *storage_manager) {
   DCHECK(ProtoIsValid(proto, database));
 
-  // Rebuild contructor arguments from their representation in 'proto'.
-  const AggregateFunction *aggregate_function
-      = &AggregateFunctionFactory::ReconstructFromProto(proto.function());
-
   std::vector<std::unique_ptr<const Scalar>> arguments;
   arguments.reserve(proto.arguments_size());
   for (int argument_idx = 0; argument_idx < proto.arguments_size(); ++argument_idx) {
@@ -126,10 +117,10 @@ WindowAggregationOperationState* WindowAggregationOperationState::ReconstructFro
   const std::int64_t num_preceding = proto.num_preceding();
   const std::int64_t num_following = proto.num_following();
 
-  return new WindowAggregationOperationState(database.getRelationSchemaById(proto.relation_id()),
-                                             aggregate_function,
+  return new WindowAggregationOperationState(database.getRelationSchemaById(proto.input_relation_id()),
+                                             &WindowAggregateFunctionFactory::ReconstructFromProto(proto.function()),
                                              std::move(arguments),
-                                             std::move(partition_by_attributes),
+                                             partition_by_attributes,
                                              is_row,
                                              num_preceding,
                                              num_following,
@@ -139,11 +130,11 @@ WindowAggregationOperationState* WindowAggregationOperationState::ReconstructFro
 bool WindowAggregationOperationState::ProtoIsValid(const serialization::WindowAggregationOperationState &proto,
                                                    const CatalogDatabaseLite &database) {
   if (!proto.IsInitialized() ||
-      !database.hasRelationWithId(proto.relation_id())) {
+      !database.hasRelationWithId(proto.input_relation_id())) {
     return false;
   }
 
-  if (!AggregateFunctionFactory::ProtoIsValid(proto.function())) {
+  if (!WindowAggregateFunctionFactory::ProtoIsValid(proto.function())) {
     return false;
   }
 
@@ -176,4 +167,122 @@ bool WindowAggregationOperationState::ProtoIsValid(const serialization::WindowAg
   return true;
 }
 
+void WindowAggregationOperationState::windowAggregateBlocks(
+    InsertDestination *output_destination,
+    const std::vector<block_id> &block_ids) {
+  // TODO(Shixuan): This is a quick fix for currently unsupported functions in
+  // order to pass the query_optimizer test.
+  if (window_aggregation_handle_.get() == nullptr) {
+    std::cout << "The function will be supported in the near future :)\n";
+    return;
+  }
+
+  // TODO(Shixuan): RANGE frame mode should be supported to make SQL grammar
+  // work. This will need Order Key to be passed so that we know where the
+  // window should start and end.
+  if (!is_row_) {
+    std::cout << "Currently we don't support RANGE frame mode :(\n";
+    return;
+  }
+
+  // Get the total number of tuples.
+  int num_tuples = 0;
+  for (const block_id block_idx : block_ids) {
+    num_tuples +=
+        storage_manager_->getBlock(block_idx, input_relation_)->getNumTuples();
+  }
+
+  // Construct column vectors for attributes.
+  std::vector<ColumnVector*> attribute_vecs;
+  for (std::size_t attr_id = 0; attr_id < input_relation_.size(); ++attr_id) {
+    const CatalogAttribute *attr = input_relation_.getAttributeById(attr_id);
+    const Type &type = attr->getType();
+
+    if (NativeColumnVector::UsableForType(type)) {
+      attribute_vecs.push_back(new NativeColumnVector(type, num_tuples));
+    } else {
+      attribute_vecs.push_back(new IndirectColumnVector(type, num_tuples));
+    }
+  }
+
+  // Construct column vectors for arguments.
+  std::vector<ColumnVector*> argument_vecs;
+  for (const std::unique_ptr<const Scalar> &argument : arguments_) {
+    const Type &type = argument->getType();
+
+    if (NativeColumnVector::UsableForType(type)) {
+      argument_vecs.push_back(new NativeColumnVector(type, num_tuples));
+    } else {
+      argument_vecs.push_back(new IndirectColumnVector(type, num_tuples));
+    }
+  }
+
+  // TODO(Shixuan): Add Support for Vector Copy Elision Selection.
+  // Add tuples and arguments into ColumnVectors.
+  for (const block_id block_idx : block_ids) {
+    BlockReference block = storage_manager_->getBlock(block_idx, input_relation_);
+    const TupleStorageSubBlock &tuple_block = block->getTupleStorageSubBlock();
+    SubBlocksReference sub_block_ref(tuple_block,
+                                     block->getIndices(),
+                                     block->getIndicesConsistent());
+    ValueAccessor *tuple_accessor = tuple_block.createValueAccessor();
+    ColumnVectorsValueAccessor *argument_accessor = new ColumnVectorsValueAccessor();
+    for (const std::unique_ptr<const Scalar> &argument : arguments_) {
+      argument_accessor->addColumn(argument->getAllValues(tuple_accessor,
+                                                          &sub_block_ref));
+    }
+
+    InvokeOnAnyValueAccessor(tuple_accessor,
+                             [&] (auto *tuple_accessor) -> void {  // NOLINT(build/c++11)
+      tuple_accessor->beginIteration();
+      argument_accessor->beginIteration();
+
+      while (tuple_accessor->next() && argument_accessor->next()) {
+        for (std::size_t attr_id = 0; attr_id < attribute_vecs.size(); ++attr_id) {
+          ColumnVector *attr_vec = attribute_vecs[attr_id];
+          if (attr_vec->isNative()) {
+            static_cast<NativeColumnVector*>(attr_vec)->appendTypedValue(
+                tuple_accessor->getTypedValue(attr_id));
+          } else {
+            static_cast<IndirectColumnVector*>(attr_vec)->appendTypedValue(
+                tuple_accessor->getTypedValue(attr_id));
+          }
+        }
+
+        for (std::size_t argument_idx = 0;
+             argument_idx < argument_vecs.size();
+             ++argument_idx) {
+          ColumnVector *argument = argument_vecs[argument_idx];
+          if (argument->isNative()) {
+            static_cast<NativeColumnVector*>(argument)->appendTypedValue(
+                argument_accessor->getTypedValue(argument_idx));
+          } else {
+            static_cast<IndirectColumnVector*>(argument)->appendTypedValue(
+                argument_accessor->getTypedValue(argument_idx));
+          }
+        }
+      }
+    });
+  }
+
+  // Construct the value accessor for tuples in all blocks
+  ColumnVectorsValueAccessor *all_blocks_accessor
+      = new ColumnVectorsValueAccessor();
+  for (ColumnVector *attr_vec : attribute_vecs) {
+    all_blocks_accessor->addColumn(attr_vec);
+  }
+
+  // Do actual calculation in handle.
+  ColumnVector *window_aggregates =
+      window_aggregation_handle_->calculate(all_blocks_accessor,
+                                            std::move(argument_vecs),
+                                            partition_by_ids_,
+                                            is_row_,
+                                            num_preceding_,
+                                            num_following_);
+
+  all_blocks_accessor->addColumn(window_aggregates);
+  output_destination->bulkInsertTuples(all_blocks_accessor);
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/storage/WindowAggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.hpp b/storage/WindowAggregationOperationState.hpp
index d7b3e6a..9792a99 100644
--- a/storage/WindowAggregationOperationState.hpp
+++ b/storage/WindowAggregationOperationState.hpp
@@ -25,20 +25,20 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "expressions/scalar/ScalarAttribute.hpp"
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/WindowAggregationOperationState.pb.h"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
-class AggregateFunction;
 class CatalogDatabaseLite;
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WindowAggregateFunction;
 
 /** \addtogroup Storage
  *  @{
@@ -63,13 +63,12 @@ class WindowAggregationOperationState {
    *                      current row. -1 means UNBOUNDED PRECEDING.
    * @param num_following The number of rows/range for the tuples following the
    *                      current row. -1 means UNBOUNDED FOLLOWING.
-   * @param storage_manager The StorageManager to use for allocating hash
-   *        tables.
+   * @param storage_manager The StorageManager to get block references.
    */
   WindowAggregationOperationState(const CatalogRelationSchema &input_relation,
-                                  const AggregateFunction *window_aggregate_function,
+                                  const WindowAggregateFunction *window_aggregate_function,
                                   std::vector<std::unique_ptr<const Scalar>> &&arguments,
-                                  std::vector<std::unique_ptr<const Scalar>> &&partition_by_attributes,
+                                  const std::vector<std::unique_ptr<const Scalar>> &partition_by_attributes,
                                   const bool is_row,
                                   const std::int64_t num_preceding,
                                   const std::int64_t num_following,
@@ -107,66 +106,29 @@ class WindowAggregationOperationState {
                            const CatalogDatabaseLite &database);
 
   /**
-   * @brief Get the is_row info.
-   * @note This is a quickfix for "unused variable". After the window aggregate
-   *       functions are built, these methods might be dropped.
-   * 
-   * @return True if the frame mode is ROW, false if it is RANGE.
-   **/
-  const bool is_row() const { return is_row_; }
-
-  /**
-   * @brief Get the num_preceding info.
-   * @note This is a quickfix for "unused variable". After the window aggregate
-   *       functions are built, these methods might be dropped.
-   *
-   * @return The number of rows/range that precedes the current row.
-   **/
-  const std::int64_t num_preceding() const { return num_preceding_; }
-
-  /**
-   * @brief Get the num_following info.
-   * @note This is a quickfix for "unused variable". After the window aggregate
-   *       functions are built, these methods might be dropped.
+   * @brief Compute window aggregates on the tuples of the given relation.
    *
-   * @return The number of rows/range that follows the current row.
+   * @param output_destination The output destination for the computed window
+   *                           aggregate.
+   * @param block_ids The id of the blocks to be computed.
    **/
-  const std::int64_t num_following() const { return num_following_; }
-
-  /**
-   * @brief Get the pointer to StorageManager.
-   * @note This is a quickfix for "unused variable". After the window aggregate
-   *       functions are built, these methods might be dropped.
-   *
-   * @return A pointer to the storage manager.
-   **/
-  StorageManager *storage_manager() { return storage_manager_; }
+  void windowAggregateBlocks(InsertDestination *output_destination,
+                             const std::vector<block_id> &block_ids);
 
  private:
   const CatalogRelationSchema &input_relation_;
-
-  // TODO(Shixuan): Handle and State for window aggregation will be needed for
-  //                actual calculation.
-  std::unique_ptr<AggregationHandle> window_aggregation_handle_;
-  std::unique_ptr<AggregationState> window_aggregation_state_;
+  const std::vector<block_id> block_ids_;
+  std::unique_ptr<WindowAggregationHandle> window_aggregation_handle_;
   std::vector<std::unique_ptr<const Scalar>> arguments_;
+  std::vector<attribute_id> partition_by_ids_;
 
-  // We don't add order_by_attributes here since it is not needed after sorting.
-  std::vector<std::unique_ptr<const Scalar>> partition_by_attributes_;
-
-  // Window framing information.
+  // Frame info.
   const bool is_row_;
   const std::int64_t num_preceding_;
   const std::int64_t num_following_;
 
   StorageManager *storage_manager_;
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  // If all an aggregate's argument expressions are simply attributes in
-  // 'input_relation_', then this caches the attribute IDs of those arguments.
-  std::vector<attribute_id> arguments_as_attributes_;
-#endif
-
   DISALLOW_COPY_AND_ASSIGN(WindowAggregationOperationState);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/storage/WindowAggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.proto b/storage/WindowAggregationOperationState.proto
index c7bd0ef..d888461 100644
--- a/storage/WindowAggregationOperationState.proto
+++ b/storage/WindowAggregationOperationState.proto
@@ -19,12 +19,12 @@ syntax = "proto2";
 
 package quickstep.serialization;
 
-import "expressions/aggregation/AggregateFunction.proto";
+import "expressions/window_aggregation/WindowAggregateFunction.proto";
 import "expressions/Expressions.proto";
 
 message WindowAggregationOperationState {
-  required int32 relation_id = 1;
-  required AggregateFunction function = 2;
+  required int32 input_relation_id = 1;
+  required WindowAggregateFunction function = 2;
   repeated Scalar arguments = 3;
   repeated Scalar partition_by_attributes = 4;
   required bool is_row = 5;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/storage/tests/WindowAggregationOperationState_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/WindowAggregationOperationState_unittest.cpp b/storage/tests/WindowAggregationOperationState_unittest.cpp
index c572034..d58f0f5 100644
--- a/storage/tests/WindowAggregationOperationState_unittest.cpp
+++ b/storage/tests/WindowAggregationOperationState_unittest.cpp
@@ -23,7 +23,7 @@
 #include "catalog/CatalogDatabase.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregateFunction.pb.h"
+#include "expressions/window_aggregation/WindowAggregateFunction.pb.h"
 #include "storage/WindowAggregationOperationState.hpp"
 #include "storage/WindowAggregationOperationState.pb.h"
 
@@ -57,8 +57,8 @@ TEST_F(WindowAggregationOperationStateProtoTest, UninitializationTest) {
 
 TEST_F(WindowAggregationOperationStateProtoTest, InvalidRelationIdTest) {
   serialization::WindowAggregationOperationState proto;
-  proto.set_relation_id(kInvalidTableId);
-  proto.mutable_function()->set_aggregation_id(serialization::AggregateFunction::AVG);
+  proto.set_input_relation_id(kInvalidTableId);
+  proto.mutable_function()->set_window_aggregation_id(serialization::WindowAggregateFunction::AVG);
   proto.set_is_row(true);
   proto.set_num_preceding(kValidNum);
   proto.set_num_following(kValidNum);
@@ -67,8 +67,8 @@ TEST_F(WindowAggregationOperationStateProtoTest, InvalidRelationIdTest) {
 
 TEST_F(WindowAggregationOperationStateProtoTest, InvalidNumTest) {
   serialization::WindowAggregationOperationState proto;
-  proto.set_relation_id(rel_id_);
-  proto.mutable_function()->set_aggregation_id(serialization::AggregateFunction::AVG);
+  proto.set_input_relation_id(rel_id_);
+  proto.mutable_function()->set_window_aggregation_id(serialization::WindowAggregateFunction::AVG);
   proto.set_is_row(true);
   proto.set_num_preceding(kInvalidNum);
   proto.set_num_following(kValidNum);
@@ -81,8 +81,8 @@ TEST_F(WindowAggregationOperationStateProtoTest, InvalidNumTest) {
 
 TEST_F(WindowAggregationOperationStateProtoTest, ValidTest) {
   serialization::WindowAggregationOperationState proto;
-  proto.set_relation_id(rel_id_);
-  proto.mutable_function()->set_aggregation_id(serialization::AggregateFunction::AVG);
+  proto.set_input_relation_id(rel_id_);
+  proto.mutable_function()->set_window_aggregation_id(serialization::WindowAggregateFunction::AVG);
   proto.set_is_row(true);
   proto.set_num_preceding(kValidNum);
   proto.set_num_following(kValidNum);


[07/10] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/EventProfiler.hpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.hpp b/utility/EventProfiler.hpp
new file mode 100644
index 0000000..70024e6
--- /dev/null
+++ b/utility/EventProfiler.hpp
@@ -0,0 +1,188 @@
+/**
+ *   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_UTILITY_EVENT_PROFILER_HPP_
+#define QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_
+
+#include <chrono>
+#include <cstddef>
+#include <cstring>
+#include <ctime>
+#include <iomanip>
+#include <map>
+#include <ostream>
+#include <thread>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "threading/Mutex.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+using clock = std::chrono::steady_clock;
+
+template <typename TagT, typename ...PayloadT>
+class EventProfiler {
+
+ public:
+  EventProfiler()
+      : zero_time_(clock::now()) {
+  }
+
+  struct EventInfo {
+    clock::time_point start_time;
+    clock::time_point end_time;
+    bool is_finished;
+    std::tuple<PayloadT...> payload;
+
+    explicit EventInfo(const clock::time_point &start_time_in)
+        : start_time(start_time_in),
+          is_finished(false) {
+    }
+
+    EventInfo()
+        : start_time(clock::now()),
+          is_finished(false) {
+    }
+
+    inline void setPayload(PayloadT &&...in_payload) {
+      payload = std::make_tuple(in_payload...);
+    }
+
+    inline void endEvent() {
+      end_time = clock::now();
+      is_finished = true;
+    }
+  };
+
+  struct EventContainer {
+    EventContainer()
+        : context(0) {}
+
+    inline void startEvent(const TagT &tag) {
+      events[tag].emplace_back(clock::now());
+    }
+
+    inline void endEvent(const TagT &tag) {
+      auto &event_info = events.at(tag).back();
+      event_info.is_finished = true;
+      event_info.end_time = clock::now();
+    }
+
+    inline std::vector<EventInfo> *getEventLine(const TagT &tag) {
+      return &events[tag];
+    }
+
+    inline void setContext(int context_in) {
+      context = context_in;
+    }
+
+    inline int getContext() const {
+      return context;
+    }
+
+    std::map<TagT, std::vector<EventInfo>> events;
+    int context;
+  };
+
+  EventContainer *getContainer() {
+    MutexLock lock(mutex_);
+    return &thread_map_[std::this_thread::get_id()];
+  }
+
+  void writeToStream(std::ostream &os) const {
+    time_t rawtime;
+    time(&rawtime);
+    char event_id[32];
+    strftime(event_id, sizeof event_id, "%Y-%m-%d %H:%M:%S", localtime(&rawtime));
+
+    int thread_id = 0;
+    for (const auto &thread_ctx : thread_map_) {
+      for (const auto &event_group : thread_ctx.second.events) {
+        for (const auto &event_info : event_group.second) {
+          CHECK(event_info.is_finished) << "Unfinished profiling event";
+
+          os << std::setprecision(12)
+             << event_id << ","
+             << thread_id << "," << event_group.first << ",";
+
+          PrintTuple(os, event_info.payload, ",");
+
+          os << std::chrono::duration<double>(event_info.start_time - zero_time_).count()
+             << ","
+             << std::chrono::duration<double>(event_info.end_time - zero_time_).count()
+             << "\n";
+        }
+      }
+      ++thread_id;
+    }
+  }
+
+  void clear() {
+    zero_time_ = clock::now();
+    thread_map_.clear();
+  }
+
+  const std::map<std::thread::id, EventContainer> &containers() {
+    return thread_map_;
+  }
+
+  const clock::time_point &zero_time() {
+    return zero_time_;
+  }
+
+ private:
+  template<class Tuple, std::size_t N>
+  struct TuplePrinter {
+    static void Print(std::ostream &os, const Tuple &t, const std::string &sep) {
+      TuplePrinter<Tuple, N-1>::Print(os, t, sep);
+      os << std::get<N-1>(t) << sep;
+    }
+  };
+
+  template<class Tuple>
+  struct TuplePrinter<Tuple, 1> {
+    static void Print(std::ostream &os, const Tuple &t, const std::string &sep) {
+      os << std::get<0>(t) << sep;
+    }
+  };
+
+  template<class... Args>
+  static void PrintTuple(std::ostream &os, const std::tuple<Args...>& t, const std::string &sep) {
+    TuplePrinter<decltype(t), sizeof...(Args)>::Print(os, t, sep);
+  }
+
+  clock::time_point zero_time_;
+  std::map<std::thread::id, EventContainer> thread_map_;
+  Mutex mutex_;
+};
+
+extern EventProfiler<int, std::size_t> simple_profiler;
+extern EventProfiler<std::size_t> relop_profiler;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 962d577..4cc1b0f 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -19,6 +19,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <set>
 #include <sstream>
 #include <string>
 #include <unordered_map>
@@ -28,6 +29,7 @@
 
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -101,6 +103,10 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
   int node_id = ++id_counter_;
   node_id_map_.emplace(input, node_id);
 
+  std::set<E::ExprId> referenced_ids;
+  for (const auto &attr : input->getReferencedAttributes()) {
+    referenced_ids.emplace(attr->id());
+  }
   for (const auto &child : input->children()) {
     visit(child);
 
@@ -111,10 +117,8 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     edge_info.src_node_id = child_id;
     edge_info.dst_node_id = node_id;
 
-    // Print output attributes except for TableReference -- there are just too many
-    // attributes out of TableReference.
-    if (child->getPhysicalType() != P::PhysicalType::kTableReference) {
-      for (const auto &attr : child->getOutputAttributes()) {
+    for (const auto &attr : child->getOutputAttributes()) {
+      if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
         edge_info.labels.emplace_back(attr->attribute_alias());
       }
     }
@@ -145,6 +149,36 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
         node_info.labels.emplace_back(
             left_attributes[i]->attribute_alias() + " = " + right_attributes[i]->attribute_alias());
       }
+      if (hash_join->left()->impliesUniqueAttributes(left_attributes)) {
+        node_info.labels.emplace_back("LEFT join attrs unique");
+      }
+      if (hash_join->right()->impliesUniqueAttributes(right_attributes)) {
+        node_info.labels.emplace_back("RIGHT join attrs unique");
+      }
+
+      const auto &bf_config = hash_join->bloom_filter_config();
+      for (const auto &bf : bf_config.build_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF build] ") + bf.attribute->attribute_alias());
+      }
+      for (const auto &bf : bf_config.probe_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF probe] ") + bf.attribute->attribute_alias());
+      }
+
+      break;
+    }
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr aggregate =
+        std::static_pointer_cast<const P::Aggregate>(input);
+      node_info.labels.emplace_back(input->getName());
+
+      const auto &bf_config = aggregate->bloom_filter_config();
+      for (const auto &bf : bf_config.probe_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF probe] ") + bf.attribute->attribute_alias());
+      }
+
       break;
     }
     default: {


[04/10] incubator-quickstep git commit: Introduced Shiftboss for the distributed version.

Posted by ji...@apache.org.
Introduced Shiftboss for the distributed version.


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

Branch: refs/heads/LIP-for-tpch
Commit: 7415ee87fa088054e09b2a9efb88389c43351b12
Parents: 9f9e3b7
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Fri Jul 22 13:29:03 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Wed Jul 27 15:06:13 2016 -0700

----------------------------------------------------------------------
 catalog/CatalogDatabaseCache.hpp             |   5 +
 query_execution/CMakeLists.txt               |  24 ++
 query_execution/QueryExecutionMessages.proto |  30 ++
 query_execution/QueryExecutionTypedefs.hpp   |   8 +
 query_execution/Shiftboss.cpp                | 360 ++++++++++++++++++++++
 query_execution/Shiftboss.hpp                | 241 +++++++++++++++
 storage/StorageManager.hpp                   |   1 +
 7 files changed, 669 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/catalog/CatalogDatabaseCache.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabaseCache.hpp b/catalog/CatalogDatabaseCache.hpp
index 77afe2a..b3e73a6 100644
--- a/catalog/CatalogDatabaseCache.hpp
+++ b/catalog/CatalogDatabaseCache.hpp
@@ -54,6 +54,11 @@ namespace serialization { class CatalogDatabase; }
 class CatalogDatabaseCache : public CatalogDatabaseLite {
  public:
   /**
+   * @brief Constructor.
+   **/
+  CatalogDatabaseCache() {}
+
+  /**
    * @brief Constructor. Reconstruct a database cache from its serialized
    *        Protocol Buffer form.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index f582ba5..8bf1ab1 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -52,6 +52,7 @@ if (ENABLE_DISTRIBUTED)
 endif()
 add_library(quickstep_queryexecution_QueryManagerSingleNode QueryManagerSingleNode.cpp QueryManagerSingleNode.hpp)
 if (ENABLE_DISTRIBUTED)
+  add_library(quickstep_queryexecution_Shiftboss Shiftboss.cpp Shiftboss.hpp)
   add_library(quickstep_queryexecution_ShiftbossDirectory ../empty_src.cpp ShiftbossDirectory.hpp)
 endif()
 add_library(quickstep_queryexecution_WorkOrderProtosContainer ../empty_src.cpp WorkOrderProtosContainer.hpp)
@@ -157,6 +158,8 @@ target_link_libraries(quickstep_queryexecution_QueryContext_proto
                       quickstep_utility_SortConfiguration_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_queryexecution_QueryExecutionMessages_proto
+                      quickstep_catalog_Catalog_proto
+                      quickstep_queryexecution_QueryContext_proto
                       quickstep_relationaloperators_WorkOrder_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_queryexecution_QueryExecutionState
@@ -214,6 +217,26 @@ target_link_libraries(quickstep_queryexecution_QueryManagerSingleNode
                       quickstep_utility_Macros
                       tmb)
 if (ENABLE_DISTRIBUTED)
+  target_link_libraries(quickstep_queryexecution_Shiftboss
+                        glog
+                        quickstep_catalog_CatalogDatabaseCache
+                        quickstep_catalog_CatalogTypedefs
+                        quickstep_queryexecution_QueryContext
+                        quickstep_queryexecution_QueryExecutionMessages_proto
+                        quickstep_queryexecution_QueryExecutionTypedefs
+                        quickstep_queryexecution_QueryExecutionUtil
+                        quickstep_queryexecution_WorkerDirectory
+                        quickstep_queryexecution_WorkerMessage
+                        quickstep_relationaloperators_RebuildWorkOrder
+                        quickstep_relationaloperators_WorkOrderFactory
+                        quickstep_storage_InsertDestination
+                        quickstep_storage_StorageBlock
+                        quickstep_storage_StorageBlockInfo
+                        quickstep_storage_StorageManager
+                        quickstep_threading_Thread
+                        quickstep_threading_ThreadUtil
+                        quickstep_utility_Macros
+                        tmb)
   target_link_libraries(quickstep_queryexecution_ShiftbossDirectory
                         quickstep_utility_Macros
                         tmb)
@@ -272,6 +295,7 @@ if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution
                         quickstep_queryexecution_BlockLocator
                         quickstep_queryexecution_QueryManagerDistributed
+                        quickstep_queryexecution_Shiftboss
                         quickstep_queryexecution_ShiftbossDirectory)
 endif()
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index fa20993..591ca6c 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -16,6 +16,8 @@ syntax = "proto2";
 
 package quickstep.serialization;
 
+import "catalog/Catalog.proto";
+import "query_execution/QueryContext.proto";
 import "relational_operators/WorkOrder.proto";
 
 // Used for any messages that do not carry payloads.
@@ -73,6 +75,25 @@ message WorkOrdersAvailableMessage {
 }
 
 // Distributed version related messages.
+message ShiftbossRegistrationMessage {
+  // The total Work Order processing capacity in Shiftboss, which equals to the
+  // sum of the capacity of each worker managed by Shiftboss.
+  required uint64 work_order_capacity = 1;
+}
+
+message ShiftbossRegistrationResponseMessage {
+}
+
+message QueryInitiateMessage {
+  required uint64 query_id = 1;
+  required CatalogDatabase catalog_database_cache = 2;
+  required QueryContext query_context = 3;
+}
+
+message QueryInitiateResponseMessage {
+  required uint64 query_id = 1;
+}
+
 message WorkOrderMessage {
   required uint64 query_id = 1;
   required uint64 operator_index = 2;
@@ -92,6 +113,15 @@ message InitiateRebuildResponseMessage {
   required uint64 num_rebuild_work_orders = 3;
 }
 
+message QueryResultRelationMessage {
+  required int32 relation_id = 1;
+  repeated fixed64 blocks = 2 [packed=true];
+}
+
+message QueryResultRelationResponseMessage {
+  required int32 relation_id = 1;
+}
+
 // BlockLocator related messages.
 message BlockDomainRegistrationMessage {
   // Format IP:Port, i.e., "0.0.0.0:0".

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 61e76d7..d73d4ee 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -73,9 +73,17 @@ enum QueryExecutionMessageType : message_type_id {
   kPoisonMessage,  // From the main thread to Foreman and Workers.
 
 #ifdef QUICKSTEP_DISTRIBUTED
+  kShiftbossRegistrationMessage,  // From Shiftboss to Foreman.
+  kShiftbossRegistrationResponseMessage,  // From Foreman to Shiftboss.
+  kQueryInitiateMessage,  // From Foreman to Shiftboss.
+  kQueryInitiateResponseMessage,  // From Shiftboss to Foreman.
+
   kInitiateRebuildMessage,  // From Foreman to Shiftboss.
   kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
 
+  kQueryResultRelationMessage,  // From Foreman to Shiftboss.
+  kQueryResultRelationResponseMessage,  // From Shiftboss to Foreman.
+
   // BlockLocator related messages, sorted in a life cycle of StorageManager
   // with a unique block domain.
   kBlockDomainRegistrationMessage,  // From Worker to BlockLocator.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
new file mode 100644
index 0000000..af56306
--- /dev/null
+++ b/query_execution/Shiftboss.cpp
@@ -0,0 +1,360 @@
+/**
+ *   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_execution/Shiftboss.hpp"
+
+#include <cstddef>
+#include <cstdlib>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/WorkerMessage.hpp"
+#include "relational_operators/RebuildWorkOrder.hpp"
+#include "relational_operators/WorkOrderFactory.hpp"
+#include "storage/InsertDestination.hpp"
+#include "storage/StorageBlock.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "threading/ThreadUtil.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/address.h"
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+#include "tmb/message_style.h"
+#include "tmb/tagged_message.h"
+
+using std::free;
+using std::malloc;
+using std::move;
+using std::size_t;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+using tmb::TaggedMessage;
+
+namespace quickstep {
+
+class WorkOrder;
+
+void Shiftboss::run() {
+  if (cpu_id_ >= 0) {
+    // We can pin the shiftboss thread to a CPU if specified.
+    ThreadUtil::BindToCPU(cpu_id_);
+  }
+
+  for (;;) {
+    // Receive() is a blocking call, causing this thread to sleep until next
+    // message is received.
+    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
+    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+              << "') received the typed '" << annotated_message.tagged_message.message_type()
+              << "' message from client " << annotated_message.sender;
+    switch (annotated_message.tagged_message.message_type()) {
+      case kShiftbossRegistrationResponseMessage: {
+        foreman_client_id_ = annotated_message.sender;
+        break;
+      }
+      case kQueryInitiateMessage: {
+        const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+        serialization::QueryInitiateMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        processQueryInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
+        break;
+      }
+      case kWorkOrderMessage: {
+        const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+        serialization::WorkOrderMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        const std::size_t query_id = proto.query_id();
+        DCHECK_EQ(1u, query_contexts_.count(query_id));
+
+        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
+                                                                       &database_cache_,
+                                                                       query_contexts_[query_id].get(),
+                                                                       storage_manager_,
+                                                                       shiftboss_client_id_,
+                                                                       bus_);
+
+        unique_ptr<WorkerMessage> worker_message(
+            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
+
+        TaggedMessage worker_tagged_message(worker_message.get(),
+                                            sizeof(*worker_message),
+                                            kWorkOrderMessage);
+
+        const size_t worker_index = getSchedulableWorker();
+        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
+                  << "') from Foreman to worker " << worker_index;
+
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           shiftboss_client_id_,
+                                           workers_->getClientID(worker_index),
+                                           move(worker_tagged_message));
+        break;
+      }
+      case kInitiateRebuildMessage: {
+        // Construct rebuild work orders, and send back their number to
+        // 'ForemanDistributed'.
+        const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+        serialization::InitiateRebuildMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        processInitiateRebuildMessage(proto.query_id(),
+                                      proto.operator_index(),
+                                      proto.insert_destination_index(),
+                                      proto.relation_id());
+        break;
+      }
+      case kWorkOrderCompleteMessage:  // Fall through.
+      case kRebuildWorkOrderCompleteMessage:
+      case kDataPipelineMessage:
+      case kWorkOrdersAvailableMessage:
+      case kWorkOrderFeedbackMessage: {
+        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
+                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
+
+        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           shiftboss_client_id_,
+                                           foreman_client_id_,
+                                           move(annotated_message.tagged_message));
+        break;
+      }
+      case kQueryResultRelationMessage: {
+        // TODO(zuyu): Rename to kSaveQueryResultMessage.
+        const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+        serialization::QueryResultRelationMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        for (int i = 0; i < proto.blocks_size(); ++i) {
+          const block_id block = proto.blocks(i);
+          storage_manager_->saveBlockOrBlob(block);
+          if (storage_manager_->blockOrBlobIsLoaded(block)) {
+            // NOTE(zuyu): eviction is required to avoid accesses to the query
+            // result relation schema in CatalogDatabaseCache, for all query
+            // optimizer execution generator unit tests and the single-process
+            // Quickstep CLI.
+            storage_manager_->evictBlockOrBlob(block);
+          }
+        }
+
+        serialization::QueryResultRelationResponseMessage ack_proto;
+        ack_proto.set_relation_id(proto.relation_id());
+
+        const size_t ack_proto_length = ack_proto.ByteSize();
+        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
+        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
+
+        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
+                                  ack_proto_length,
+                                  kQueryResultRelationResponseMessage);
+        free(ack_proto_bytes);
+
+        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
+                  << ") to Foreman";
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           shiftboss_client_id_,
+                                           foreman_client_id_,
+                                           move(ack_message));
+        break;
+      }
+      case kPoisonMessage: {
+        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
+                  << "') from Foreman to all workers";
+
+        tmb::MessageStyle broadcast_style;
+        broadcast_style.Broadcast(true);
+
+        tmb::MessageBus::SendStatus send_status =
+            bus_->Send(shiftboss_client_id_,
+                       worker_addresses_,
+                       broadcast_style,
+                       move(annotated_message.tagged_message));
+        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
+        return;
+      }
+      default: {
+        LOG(FATAL) << "Unknown TMB message type";
+      }
+    }
+  }
+}
+
+size_t Shiftboss::getSchedulableWorker() {
+  const size_t num_workers = workers_->getNumWorkers();
+
+  size_t curr_worker = start_worker_index_;
+  for (;;) {
+    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
+      start_worker_index_ = (curr_worker + 1) % num_workers;
+      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
+      // But we need a WorkOrder queue first.
+      return curr_worker;
+    }
+
+    curr_worker = (curr_worker + 1) % num_workers;
+  }
+}
+
+void Shiftboss::registerWithForeman() {
+  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
+            << "') to all";
+
+  tmb::Address all_addresses;
+  all_addresses.All(true);
+
+  tmb::MessageStyle style;
+
+  serialization::ShiftbossRegistrationMessage proto;
+  proto.set_work_order_capacity(getWorkOrderCapacity());
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage message(static_cast<const void*>(proto_bytes),
+                        proto_length,
+                        kShiftbossRegistrationMessage);
+  free(proto_bytes);
+
+  tmb::MessageBus::SendStatus send_status =
+      bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
+  DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
+}
+
+void Shiftboss::processQueryInitiateMessage(
+    const std::size_t query_id,
+    const serialization::CatalogDatabase &catalog_database_cache_proto,
+    const serialization::QueryContext &query_context_proto) {
+  database_cache_.update(catalog_database_cache_proto);
+
+  unique_ptr<QueryContext> query_context(
+      new QueryContext(query_context_proto,
+                       database_cache_,
+                       storage_manager_,
+                       shiftboss_client_id_,
+                       bus_));
+  query_contexts_.emplace(query_id, move(query_context));
+
+  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+            << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
+            << "') to Foreman";
+
+  serialization::QueryInitiateResponseMessage proto;
+  proto.set_query_id(query_id);
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage ack_message(static_cast<const void*>(proto_bytes),
+                            proto_length,
+                            kQueryInitiateResponseMessage);
+  free(proto_bytes);
+
+  QueryExecutionUtil::SendTMBMessage(bus_,
+                                     shiftboss_client_id_,
+                                     foreman_client_id_,
+                                     move(ack_message));
+}
+
+void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
+                                              const std::size_t op_index,
+                                              const QueryContext::insert_destination_id dest_index,
+                                              const relation_id rel_id) {
+  DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
+
+  DCHECK_EQ(1u, query_contexts_.count(query_id));
+  InsertDestination *insert_destination = query_contexts_[query_id]->getInsertDestination(dest_index);
+  DCHECK(insert_destination != nullptr);
+
+  vector<MutableBlockReference> partially_filled_block_refs;
+  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
+
+  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+            << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
+            << "') to Foreman";
+
+  serialization::InitiateRebuildResponseMessage proto;
+  proto.set_query_id(query_id);
+  proto.set_operator_index(op_index);
+  proto.set_num_rebuild_work_orders(partially_filled_block_refs.size());
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage ack_message(static_cast<const void*>(proto_bytes),
+                            proto_length,
+                            kInitiateRebuildResponseMessage);
+  free(proto_bytes);
+
+  QueryExecutionUtil::SendTMBMessage(bus_,
+                                     shiftboss_client_id_,
+                                     foreman_client_id_,
+                                     move(ack_message));
+
+  for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
+    // NOTE(zuyu): Worker releases the memory after the execution of
+    // RebuildWorkOrder on the Worker.
+    WorkOrder *rebuild_work_order =
+        new RebuildWorkOrder(query_id,
+                             move(partially_filled_block_refs[i]),
+                             op_index,
+                             rel_id,
+                             shiftboss_client_id_,
+                             bus_);
+
+    unique_ptr<WorkerMessage> worker_message(
+        WorkerMessage::RebuildWorkOrderMessage(rebuild_work_order, op_index));
+
+    TaggedMessage worker_tagged_message(worker_message.get(),
+                                        sizeof(*worker_message),
+                                        kRebuildWorkOrderMessage);
+
+    const size_t worker_index = getSchedulableWorker();
+    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+              << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
+              << "') to worker " << worker_index;
+
+    QueryExecutionUtil::SendTMBMessage(bus_,
+                                       shiftboss_client_id_,
+                                       workers_->getClientID(worker_index),
+                                       move(worker_tagged_message));
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
new file mode 100644
index 0000000..096ab74
--- /dev/null
+++ b/query_execution/Shiftboss.hpp
@@ -0,0 +1,241 @@
+/**
+ *   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_EXECUTION_SHIFTBOSS_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <unordered_map>
+
+#include "catalog/CatalogDatabaseCache.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/WorkerDirectory.hpp"
+#include "threading/Thread.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/address.h"
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+
+namespace quickstep {
+
+class StorageManager;
+
+namespace serialization {
+class CatalogDatabase;
+class QueryContext;
+}  // namespace serialization
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
+ *        the workorders to workers.
+ **/
+class Shiftboss : public Thread {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param bus A pointer to the TMB.
+   * @param storage_manager The StorageManager to use.
+   * @param workers A pointer to the WorkerDirectory.
+   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
+   *
+   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
+   *       around on different CPUs by the OS.
+  **/
+  Shiftboss(tmb::MessageBus *bus,
+            StorageManager *storage_manager,
+            WorkerDirectory *workers,
+            const int cpu_id = -1)
+      : bus_(DCHECK_NOTNULL(bus)),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)),
+        workers_(DCHECK_NOTNULL(workers)),
+        cpu_id_(cpu_id),
+        shiftboss_client_id_(tmb::kClientIdNone),
+        foreman_client_id_(tmb::kClientIdNone),
+        max_msgs_per_worker_(1),
+        start_worker_index_(0u) {
+    // Check to have at least one Worker.
+    DCHECK_GT(workers->getNumWorkers(), 0u);
+
+    shiftboss_client_id_ = bus_->Connect();
+    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
+    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
+
+    // Messages between Foreman and Shiftboss.
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kShiftbossRegistrationResponseMessage);
+
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kQueryInitiateMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kQueryInitiateResponseMessage);
+
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kInitiateRebuildMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kInitiateRebuildResponseMessage);
+
+    // Message sent to Worker.
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderMessage);
+
+    // Message sent to Foreman.
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kCatalogRelationNewBlockMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kDataPipelineMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrdersAvailableMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderFeedbackMessage);
+
+    // Forward the following message types from Foreman to Workers.
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kWorkOrderMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderMessage);
+
+    // Forward the following message types from Workers to Foreman.
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kWorkOrderCompleteMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderCompleteMessage);
+
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
+
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kQueryResultRelationMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kQueryResultRelationResponseMessage);
+
+    // Stop itself.
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kPoisonMessage);
+    // Stop all workers.
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kPoisonMessage);
+
+    for (std::size_t i = 0; i < workers_->getNumWorkers(); ++i) {
+      worker_addresses_.AddRecipient(workers_->getClientID(i));
+    }
+
+    registerWithForeman();
+  }
+
+  ~Shiftboss() override {
+  }
+
+  /**
+   * @brief Get the TMB client ID of Shiftboss thread.
+   *
+   * @return TMB client ID of shiftboss thread.
+   **/
+  inline tmb::client_id getBusClientID() const {
+    return shiftboss_client_id_;
+  }
+
+  /**
+   * @brief Get the Work Order processing capacity of all Workers managed by
+   *        Shiftboss during a single round of WorkOrder dispatch.
+   **/
+  inline std::size_t getWorkOrderCapacity() const {
+    DCHECK_NE(max_msgs_per_worker_, 0u);
+    return max_msgs_per_worker_ * workers_->getNumWorkers();
+  }
+
+  /**
+   * @brief Get the Worker to assign WorkOrders for execution. Block to wait if
+   *        all Workers have reached their capacity for queued WorkOrders.
+   **/
+  // TODO(zuyu): To achieve non-blocking, we need a queue to cache received
+  // normal Work Order protos from Foreman and the generated rebuild Work Orders.
+  inline std::size_t getSchedulableWorker();
+
+  /**
+   * @brief Set the maximum number of messages that should be allocated to each
+   *        worker during a single round of WorkOrder dispatch.
+   *
+   * @param max_msgs_per_worker Maximum number of messages.
+   **/
+  inline void setMaxMessagesPerWorker(const std::size_t max_msgs_per_worker) {
+    max_msgs_per_worker_ = max_msgs_per_worker;
+  }
+
+ protected:
+  /**
+   * @brief The shiftboss receives workorders, and based on the response it
+   *        assigns workorders to workers.
+   *
+   * @note  The workers who get the messages from the Shiftboss execute and
+   *        subsequently delete the WorkOrder contained in the message.
+   **/
+  void run() override;
+
+ private:
+  void registerWithForeman();
+
+  /**
+   * @brief Process the Shiftboss initiate message and ack back.
+   *
+   * @param query_id The given query id.
+   * @param catalog_database_cache_proto The proto used to update
+   *        CatalogDatabaseCache.
+   * @param query_context_proto The QueryContext proto.
+   **/
+  void processQueryInitiateMessage(const std::size_t query_id,
+                                   const serialization::CatalogDatabase &catalog_database_cache_proto,
+                                   const serialization::QueryContext &query_context_proto);
+
+  /**
+   * @brief Process the RebuildWorkOrder initiate message and ack back.
+   *
+   * @param query_id The ID of the query to which this RebuildWorkOrder initiate
+   *        message belongs.
+   * @param op_index The index of the operator for rebuild work orders.
+   * @param dest_index The InsertDestination index in QueryContext to rebuild.
+   * @param rel_id The relation that needs to generate rebuild work orders.
+   **/
+  void processInitiateRebuildMessage(const std::size_t query_id,
+                                     const std::size_t op_index,
+                                     const QueryContext::insert_destination_id dest_index,
+                                     const relation_id rel_id);
+
+  // TODO(zuyu): Use two buses for the message communication between Foreman and Shiftboss,
+  // and Shiftboss and Worker thread pool.
+  tmb::MessageBus *bus_;
+
+  CatalogDatabaseCache database_cache_;
+  StorageManager *storage_manager_;
+  WorkerDirectory *workers_;
+
+  // The ID of the CPU that the Shiftboss thread can optionally be pinned to.
+  const int cpu_id_;
+
+  tmb::client_id shiftboss_client_id_, foreman_client_id_;
+
+  // TMB recipients for all workers managed by this Shiftboss.
+  tmb::Address worker_addresses_;
+
+  // During a single round of WorkOrder dispatch, a Worker should be allocated
+  // at most these many WorkOrders.
+  std::size_t max_msgs_per_worker_;
+
+  // The worker index for scheduling Work Order.
+  std::size_t start_worker_index_;
+
+  // QueryContexts per query.
+  std::unordered_map<std::size_t, std::unique_ptr<QueryContext>> query_contexts_;
+
+  DISALLOW_COPY_AND_ASSIGN(Shiftboss);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7415ee87/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index 50ddb0f..348018f 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -619,6 +619,7 @@ class StorageManager {
   FRIEND_TEST(BlockLocatorTest, BlockTest);
   FRIEND_TEST(BlockLocatorTest, BlobTest);
 
+  friend class Shiftboss;
   FRIEND_TEST(StorageManagerTest, DifferentNUMANodeBlobTestWithEviction);
   FRIEND_TEST(StorageManagerTest, EvictFromSameShardTest);
 


[08/10] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index be31fd9..6e3dc96 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -23,6 +23,7 @@
 #include <atomic>
 #include <cstddef>
 #include <cstdlib>
+#include <memory>
 #include <type_traits>
 #include <vector>
 
@@ -39,11 +40,14 @@
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/BloomFilter.hpp"
+#include "utility/BloomFilterAdapter.hpp"
 #include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+DECLARE_int64(bloom_adapter_batch_size);
+
 /** \addtogroup Storage
  *  @{
  */
@@ -1016,8 +1020,12 @@ class HashTable : public HashTableBase<resizable,
    *
    * @param bloom_filter The pointer to the bloom filter.
    **/
-  inline void setBuildSideBloomFilter(BloomFilter *bloom_filter) {
-    build_bloom_filter_ = bloom_filter;
+  inline void addBuildSideBloomFilter(BloomFilter *bloom_filter) {
+    build_bloom_filters_.emplace_back(bloom_filter);
+  }
+
+  inline void addBuildSideAttributeId(const attribute_id build_attribute_id) {
+    build_attribute_ids_.push_back(build_attribute_id);
   }
 
   /**
@@ -1042,8 +1050,8 @@ class HashTable : public HashTableBase<resizable,
    * @param probe_attribute_ids The vector of attribute ids to use for probing
    *        the bloom filter.
    **/
-  inline void addProbeSideAttributeIds(std::vector<attribute_id> &&probe_attribute_ids) {
-    probe_attribute_ids_.push_back(probe_attribute_ids);
+  inline void addProbeSideAttributeId(const attribute_id probe_attribute_id) {
+    probe_attribute_ids_.push_back(probe_attribute_id);
   }
 
  protected:
@@ -1329,9 +1337,10 @@ class HashTable : public HashTableBase<resizable,
   // Data structures used for bloom filter optimized semi-joins.
   bool has_build_side_bloom_filter_ = false;
   bool has_probe_side_bloom_filter_ = false;
-  BloomFilter *build_bloom_filter_;
+  std::vector<BloomFilter *> build_bloom_filters_;
+  std::vector<attribute_id> build_attribute_ids_;
   std::vector<const BloomFilter*> probe_bloom_filters_;
-  std::vector<std::vector<attribute_id>> probe_attribute_ids_;
+  std::vector<attribute_id> probe_attribute_ids_;
 
   DISALLOW_COPY_AND_ASSIGN(HashTable);
 };
@@ -1477,12 +1486,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                                         &prealloc_state);
       }
     }
-    std::unique_ptr<BloomFilter> thread_local_bloom_filter;
+
     if (has_build_side_bloom_filter_) {
-      thread_local_bloom_filter.reset(new BloomFilter(build_bloom_filter_->getRandomSeed(),
-                                                      build_bloom_filter_->getNumberOfHashes(),
-                                                      build_bloom_filter_->getBitArraySize()));
+      for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+        auto *build_bloom_filter = build_bloom_filters_[i];
+        std::unique_ptr<BloomFilter> thread_local_bloom_filter(
+            new BloomFilter(build_bloom_filter->getNumberOfHashes(),
+                            build_bloom_filter->getBitArraySize()));
+        const auto &build_attr = build_attribute_ids_[i];
+        const std::size_t attr_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
+        while (accessor->next()) {
+          thread_local_bloom_filter->insertUnSafe(
+              static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
+              attr_size);
+        }
+        build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
+        accessor->beginIteration();
+      }
     }
+
     if (resizable) {
       while (result == HashTablePutResult::kOutOfSpace) {
         {
@@ -1498,11 +1521,6 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                        variable_size,
                                        (*functor)(*accessor),
                                        using_prealloc ? &prealloc_state : nullptr);
-            // Insert into bloom filter, if enabled.
-            if (has_build_side_bloom_filter_) {
-              thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
-                                                      key.getDataSize());
-            }
             if (result == HashTablePutResult::kDuplicateKey) {
               DEBUG_ASSERT(!using_prealloc);
               return result;
@@ -1528,20 +1546,11 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                    variable_size,
                                    (*functor)(*accessor),
                                    using_prealloc ? &prealloc_state : nullptr);
-        // Insert into bloom filter, if enabled.
-        if (has_build_side_bloom_filter_) {
-          thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
-                                                  key.getDataSize());
-        }
         if (result != HashTablePutResult::kOK) {
           return result;
         }
       }
     }
-    // Update the build side bloom filter with thread local copy, if available.
-    if (has_build_side_bloom_filter_) {
-      build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get());
-    }
 
     return HashTablePutResult::kOK;
   });
@@ -1607,6 +1616,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                                         &prealloc_state);
       }
     }
+
+    if (has_build_side_bloom_filter_) {
+      for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+        auto *build_bloom_filter = build_bloom_filters_[i];
+        std::unique_ptr<BloomFilter> thread_local_bloom_filter(
+            new BloomFilter(build_bloom_filter->getNumberOfHashes(),
+                            build_bloom_filter->getBitArraySize()));
+        const auto &build_attr = build_attribute_ids_[i];
+        const std::size_t attr_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
+        while (accessor->next()) {
+          thread_local_bloom_filter->insertUnSafe(
+              static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
+              attr_size);
+        }
+        build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
+        accessor->beginIteration();
+      }
+    }
+
     if (resizable) {
       while (result == HashTablePutResult::kOutOfSpace) {
         {
@@ -2229,6 +2258,7 @@ inline std::size_t HashTable<ValueT, resizable, serializable, force_key_copy, al
   }
 }
 
+
 template <typename ValueT,
           bool resizable,
           bool serializable,
@@ -2246,42 +2276,85 @@ void HashTable<ValueT, resizable, serializable, force_key_copy, allow_duplicate_
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-    while (accessor->next()) {
-      // Probe any bloom filters, if enabled.
-      if (has_probe_side_bloom_filter_) {
-        DCHECK_EQ(probe_bloom_filters_.size(), probe_attribute_ids_.size());
-        // Check if the key is contained in the BloomFilters or not.
-        bool bloom_miss = false;
-        for (std::size_t i = 0; i < probe_bloom_filters_.size() && !bloom_miss; ++i) {
-          const BloomFilter *bloom_filter = probe_bloom_filters_[i];
-          for (const attribute_id &attr_id : probe_attribute_ids_[i]) {
-            TypedValue bloom_key = accessor->getTypedValue(attr_id);
-            if (!bloom_filter->contains(static_cast<const std::uint8_t*>(bloom_key.getDataPtr()),
-                                        bloom_key.getDataSize())) {
-              bloom_miss = true;
+    std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+    if (has_probe_side_bloom_filter_) {
+      // Find (and cache) the size of each attribute in the probe lists.
+      // NOTE(nav): This code uses the accessor to get the size,
+      // and hence only works if there's at least one tuple.
+      std::vector<std::size_t> attr_size_vector;
+      attr_size_vector.reserve(probe_attribute_ids_.size());
+      for (const auto &probe_attr : probe_attribute_ids_) {
+        auto val_and_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, probe_attr);
+//        std::cerr << "BF attr size = " << val_and_size.second << "\n";
+        attr_size_vector.emplace_back(val_and_size.second);
+      }
+
+      bloom_filter_adapter.reset(new BloomFilterAdapter(
+              probe_bloom_filters_, probe_attribute_ids_, attr_size_vector));
+
+      // We want to have large batch sizes for cache efficiency while probeing,
+      // but small batch sizes to ensure that the adaptation logic kicks in
+      // (and does early). We use exponentially increasing batch sizes to
+      // achieve a balance between the two.
+      //
+      // We also keep track of num_tuples_left in the block, to ensure that
+      // we don't reserve an unnecessarily large vector.
+      std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+      std::uint32_t num_tuples_left = accessor->getNumTuples();
+      std::vector<tuple_id> batch(num_tuples_left);
+
+      do {
+        std::uint32_t batch_size =
+            batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+        for (std::size_t i = 0; i < batch_size; ++i) {
+          accessor->next();
+          batch.push_back(accessor->getCurrentPosition());
+        }
+
+        std::size_t num_hits =
+            bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+
+        for (std::size_t t = 0; t < num_hits; ++t){
+          tuple_id probe_tid = batch[t];
+          TypedValue key = accessor->getTypedValueAtAbsolutePosition(key_attr_id, probe_tid);
+          if (check_for_null_keys && key.isNull()) {
+            continue;
+          }
+          const std::size_t true_hash = use_scalar_literal_hash_template ? key.getHashScalarLiteral()
+                                                                        : key.getHash();
+          const std::size_t adjusted_hash = adjust_hashes_template ? this->AdjustHash(true_hash)
+                                                                  : true_hash;
+          std::size_t entry_num = 0;
+          const ValueT *value;
+          while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
+            (*functor)(probe_tid, *value);
+            if (!allow_duplicate_keys)
               break;
-            }
           }
         }
-        if (bloom_miss) {
-          continue;  // On a bloom filter miss, probing the hash table can be skipped.
-        }
-      }
+        batch.clear();
+        num_tuples_left -= batch_size;
+        batch_size_try = batch_size * 2;
+      } while (!accessor->iterationFinished());
+    }
 
-      TypedValue key = accessor->getTypedValue(key_attr_id);
-      if (check_for_null_keys && key.isNull()) {
-        continue;
-      }
-      const std::size_t true_hash = use_scalar_literal_hash_template ? key.getHashScalarLiteral()
-                                                                     : key.getHash();
-      const std::size_t adjusted_hash = adjust_hashes_template ? this->AdjustHash(true_hash)
-                                                               : true_hash;
-      std::size_t entry_num = 0;
-      const ValueT *value;
-      while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
-        (*functor)(*accessor, *value);
-        if (!allow_duplicate_keys) {
-          break;
+    else { // no Bloom filters to probe
+      while(accessor->next()) {
+        TypedValue key = accessor->getTypedValue(key_attr_id);
+        if (check_for_null_keys && key.isNull()) {
+          continue;
+        }
+        const std::size_t true_hash = use_scalar_literal_hash_template ? key.getHashScalarLiteral()
+                                      : key.getHash();
+        const std::size_t adjusted_hash = adjust_hashes_template ? this->AdjustHash(true_hash)
+                                          : true_hash;
+        std::size_t entry_num = 0;
+        const ValueT *value;
+        while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
+          (*functor)(*accessor, *value);
+          if (!allow_duplicate_keys)
+            break;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index 7f00f29..6eabf60 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -34,10 +34,10 @@ message HashTable {
   required HashTableImplType hash_table_impl_type = 1;
   repeated Type key_types = 2;
   required uint64 estimated_num_entries = 3;
-  repeated uint32 build_side_bloom_filter_id = 4;
-  message ProbeSideBloomFilter {
-    required uint32 probe_side_bloom_filter_id = 1;
-    repeated uint32 probe_side_attr_ids = 2;
+  message BloomFilter {
+    required uint32 bloom_filter_id = 1;
+    required uint32 attr_id = 2;
   }
-  repeated ProbeSideBloomFilter probe_side_bloom_filters = 6;
+  repeated BloomFilter probe_side_bloom_filters = 4;
+  repeated BloomFilter build_side_bloom_filters = 5;  
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 34baaeb..fbb3d41 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -318,9 +318,15 @@ class HashTableFactory {
     //                 individual implementations of the hash table constructors.
 
     // Check if there are any build side bloom filter defined on the hash table.
-    if (proto.build_side_bloom_filter_id_size() > 0) {
+    if (proto.build_side_bloom_filters_size() > 0) {
       hash_table->enableBuildSideBloomFilter();
-      hash_table->setBuildSideBloomFilter(bloom_filters[proto.build_side_bloom_filter_id(0)].get());
+      for (int j = 0; j < proto.build_side_bloom_filters_size(); ++j) {
+        const auto build_side_bloom_filter = proto.build_side_bloom_filters(j);
+        hash_table->addBuildSideBloomFilter(
+            bloom_filters[build_side_bloom_filter.bloom_filter_id()].get());
+
+        hash_table->addBuildSideAttributeId(build_side_bloom_filter.attr_id());
+      }
     }
 
     // Check if there are any probe side bloom filters defined on the hash table.
@@ -330,15 +336,10 @@ class HashTableFactory {
       for (int j = 0; j < proto.probe_side_bloom_filters_size(); ++j) {
         // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
         const auto probe_side_bloom_filter = proto.probe_side_bloom_filters(j);
-        hash_table->addProbeSideBloomFilter(bloom_filters[probe_side_bloom_filter.probe_side_bloom_filter_id()].get());
-
-        // Add the attribute ids corresponding to this probe bloom filter.
-        std::vector<attribute_id> probe_attribute_ids;
-        for (int k = 0; k < probe_side_bloom_filter.probe_side_attr_ids_size(); ++k) {
-          const attribute_id probe_attribute_id = probe_side_bloom_filter.probe_side_attr_ids(k);
-          probe_attribute_ids.push_back(probe_attribute_id);
-        }
-        hash_table->addProbeSideAttributeIds(std::move(probe_attribute_ids));
+        hash_table->addProbeSideBloomFilter(
+            bloom_filters[probe_side_bloom_filter.bloom_filter_id()].get());
+
+        hash_table->addProbeSideAttributeId(probe_side_bloom_filter.attr_id());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
index 03a975e..cbd273e 100644
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ b/storage/PackedRowStoreValueAccessor.hpp
@@ -18,6 +18,8 @@
 #ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
 #define QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
 
+#include <utility>
+
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -40,7 +42,8 @@ class PackedRowStoreValueAccessorHelper {
       : relation_(relation),
         num_tuples_(num_tuples),
         tuple_storage_(tuple_storage),
-        null_bitmap_(null_bitmap) {
+        null_bitmap_(null_bitmap),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()) {
   }
 
   inline tuple_id numPackedTuples() const {
@@ -65,6 +68,25 @@ class PackedRowStoreValueAccessorHelper {
            + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
   }
 
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                        const attribute_id attr) const {
+    DEBUG_ASSERT(tuple < num_tuples_);
+    DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
+    if (check_null) {
+      const int nullable_idx = relation_.getNullableAttributeIndex(attr);
+      if ((nullable_idx != -1)
+          && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
+        return std::make_pair(nullptr, 0);
+      }
+    }
+
+    return std::make_pair(static_cast<const char*>(tuple_storage_)
+                              + (tuple * relation_.getFixedByteLength())
+                              + relation_.getFixedLengthAttributeOffset(attr),
+                          attr_max_lengths_[attr]);
+  }
+
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
                                            const attribute_id attr) const {
     const Type &attr_type = relation_.getAttributeById(attr)->getType();
@@ -79,6 +101,7 @@ class PackedRowStoreValueAccessorHelper {
   const tuple_id num_tuples_;
   const void *tuple_storage_;
   const BitVector<false> *null_bitmap_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   DISALLOW_COPY_AND_ASSIGN(PackedRowStoreValueAccessorHelper);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 9ea1a3a..19937f2 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -100,6 +100,11 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_position_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_position_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return getTypedValueAtAbsolutePosition(attr_id, current_position_);
   }
@@ -140,6 +145,44 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     }
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    DEBUG_ASSERT(occupancy_bitmap_.getBit(tid));
+    DEBUG_ASSERT(relation_.hasAttributeWithId(attr_id));
+    const char *tuple_slot = static_cast<const char*>(tuple_storage_)
+                             + tuple_slot_bytes_ * tid;
+    if (check_null) {
+      const int nullable_idx = relation_.getNullableAttributeIndex(attr_id);
+      if (nullable_idx != -1) {
+        // const_cast is safe here. We will only be using read-only methods of
+        // BitVector.
+        BitVector<true> tuple_null_bitmap(const_cast<void*>(static_cast<const void*>(tuple_slot)),
+                                          relation_.numNullableAttributes());
+        if (tuple_null_bitmap.getBit(nullable_idx)) {
+          return std::make_pair(nullptr, 0);
+        }
+      }
+    }
+
+    const int variable_length_idx = relation_.getVariableLengthAttributeIndex(attr_id);
+    if (variable_length_idx == -1) {
+      // Fixed-length, stored in-line in slot.
+      return std::make_pair(tuple_slot + per_tuple_null_bitmap_bytes_
+                                       + relation_.getFixedLengthAttributeOffset(attr_id),
+                            attr_max_lengths_[attr_id]);
+
+    } else {
+      // Variable-length, stored at back of block.
+      const std::uint32_t *pos_ptr = reinterpret_cast<const std::uint32_t*>(
+          tuple_slot + per_tuple_null_bitmap_bytes_
+                     + relation_.getFixedByteLength()
+                     + variable_length_idx * 2 * sizeof(std::uint32_t));
+      return std::make_pair(static_cast<const char*>(tuple_storage_) + pos_ptr[0],
+                            pos_ptr[1]);
+    }
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     DEBUG_ASSERT(occupancy_bitmap_.getBit(tid));
@@ -317,6 +360,7 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
         tuple_storage_(tuple_storage),
         tuple_slot_bytes_(tuple_slot_bytes),
         per_tuple_null_bitmap_bytes_(per_tuple_null_bitmap_bytes),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
         current_position_(std::numeric_limits<std::size_t>::max()) {
   }
 
@@ -327,6 +371,7 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
   const void *tuple_storage_;
   const std::size_t tuple_slot_bytes_;
   const std::size_t per_tuple_null_bitmap_bytes_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   std::size_t current_position_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index fdd438d..78aba7c 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -389,15 +389,7 @@ AggregationState* StorageBlock::aggregate(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
-    const Predicate *predicate,
     std::unique_ptr<TupleIdSequence> *reuse_matches) const {
-  // If there is a filter predicate that hasn't already been evaluated,
-  // evaluate it now and save the results for other aggregates on this same
-  // block.
-  if (predicate && !*reuse_matches) {
-    reuse_matches->reset(getMatchesForPredicate(predicate));
-  }
-
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   // If all the arguments to this aggregate are plain relation attributes,
   // aggregate directly on a ValueAccessor from this block to avoid a copy.
@@ -418,7 +410,6 @@ void StorageBlock::aggregateGroupBy(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
     AggregationStateHashTableBase *hash_table,
     std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
@@ -440,14 +431,7 @@ void StorageBlock::aggregateGroupBy(
   ColumnVectorsValueAccessor temp_result;
   {
     std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
+    if (reuse_matches) {
       // Create a filtered ValueAccessor that only iterates over predicate
       // matches.
       accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
@@ -499,7 +483,6 @@ void StorageBlock::aggregateDistinct(
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
     AggregationStateHashTableBase *distinctify_hash_table,
     std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
@@ -514,14 +497,7 @@ void StorageBlock::aggregateDistinct(
   ColumnVectorsValueAccessor temp_result;
   {
     std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
+    if (reuse_matches) {
       // Create a filtered ValueAccessor that only iterates over predicate
       // matches.
       accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 3ae3812..3217fa2 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -410,7 +410,6 @@ class StorageBlock : public StorageBlockBase {
       const AggregationHandle &handle,
       const std::vector<std::unique_ptr<const Scalar>> &arguments,
       const std::vector<attribute_id> *arguments_as_attributes,
-      const Predicate *predicate,
       std::unique_ptr<TupleIdSequence> *reuse_matches) const;
 
   /**
@@ -460,7 +459,6 @@ class StorageBlock : public StorageBlockBase {
   void aggregateGroupBy(const AggregationHandle &handle,
                         const std::vector<std::unique_ptr<const Scalar>> &arguments,
                         const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                        const Predicate *predicate,
                         AggregationStateHashTableBase *hash_table,
                         std::unique_ptr<TupleIdSequence> *reuse_matches,
                         std::vector<std::unique_ptr<ColumnVector>>
@@ -505,7 +503,6 @@ class StorageBlock : public StorageBlockBase {
                          const std::vector<std::unique_ptr<const Scalar>> &arguments,
                          const std::vector<attribute_id> *arguments_as_attributes,
                          const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                         const Predicate *predicate,
                          AggregationStateHashTableBase *distinctify_hash_table,
                          std::unique_ptr<TupleIdSequence> *reuse_matches,
                          std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
@@ -588,6 +585,8 @@ class StorageBlock : public StorageBlockBase {
    **/
   const std::size_t getNumTuples() const;
 
+  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
+
  private:
   static TupleStorageSubBlock* CreateTupleStorageSubBlock(
       const CatalogRelationSchema &relation,
@@ -627,8 +626,6 @@ class StorageBlock : public StorageBlockBase {
   // StorageBlock's header.
   bool rebuildIndexes(bool short_circuit);
 
-  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
-
   std::unordered_map<attribute_id, TypedValue>* generateUpdatedValues(
       const ValueAccessor &accessor,
       const tuple_id tuple,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index e2a898e..e9370cc 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -375,6 +375,11 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return accessor_->template getUntypedValueAtAbsolutePosition<check_null>(attr_id, *current_position_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, *current_position_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return accessor_->getTypedValueAtAbsolutePosition(attr_id, *current_position_);
   }
@@ -387,6 +392,13 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
   }
 
   // Pass-through.
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, tid);
+  }
+
+  // Pass-through.
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     return accessor_->getTypedValueAtAbsolutePosition(attr_id, tid);
@@ -560,6 +572,12 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
                                                                              id_sequence_[current_position_]);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(
+        attr_id, id_sequence_[current_position_]);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return accessor_->getTypedValueAtAbsolutePosition(attr_id, id_sequence_[current_position_]);
   }
@@ -571,6 +589,13 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
                 "OrderedTupleIdSequenceAdapterValueAccessor");
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    FATAL_ERROR("getUntypedValueAndByteLengthAtAbsolutePosition() not implemented in "
+                "OrderedTupleIdSequenceAdapterValueAccessor");
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     FATAL_ERROR("getTypedValueAtAbsolutePosition() not implemented in "
@@ -737,6 +762,11 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_tuple_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_tuple_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return getTypedValueAtAbsolutePosition(attr_id, current_tuple_);
   }
@@ -747,6 +777,12 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return helper_.template getAttributeValue<check_null>(tid, attr_id);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    return helper_.template getAttributeValueAndByteLength<check_null>(tid, attr_id);
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     return helper_.getAttributeValueTyped(tid, attr_id);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index 76968ba..a9349ee 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -193,6 +193,22 @@ class NativeColumnVector : public ColumnVector {
   }
 
   /**
+   * @brief Get the untyped pointer to a value as well as the value's byte length
+   *        in this NativeColumnVector as a pair.
+   *
+   * @param position The position of the value to get.
+   * @return A pair containing the untyped pointer to the value at position and
+   *         the value's byte length.
+   **/
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const std::size_t position) const {
+    DCHECK_LT(position, actual_length_);
+    return (check_null && null_bitmap_ && null_bitmap_->getBit(position))
+        ? std::make_pair(nullptr, 0)
+        : std::make_pair(static_cast<const char*>(values_) + (position * type_length_), type_length_);
+  }
+
+  /**
    * @brief Get a value in this NativeColumnVector as a TypedValue.
    *
    * @param position The position of the value to get.
@@ -453,6 +469,25 @@ class IndirectColumnVector : public ColumnVector {
   }
 
   /**
+   * @brief Get the untyped pointer to a value as well as the value's byte length
+   *        in this IndirectColumnVector as a pair.
+   *
+   * @param position The position of the value to get.
+   * @return A pair containing the untyped pointer to the value at position and
+   *         the value's byte length.
+   **/
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const std::size_t position) const {
+    DCHECK_LT(position, values_.size());
+    if (check_null && type_is_nullable_ && values_[position].isNull()) {
+      return std::make_pair(nullptr, 0);
+    } else {
+      const TypedValue &value = values_[position];
+      return std::make_pair(value.getDataPtr(), value.getDataSize());
+    }
+  }
+
+  /**
    * @brief Get a value in this IndirectColumnVector as a TypedValue.
    *
    * @param position The position of the value to get.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index f1d29a2..d69d1d8 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -124,6 +124,11 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_position_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_position_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return getTypedValueAtAbsolutePosition(attr_id, current_position_);
   }
@@ -140,6 +145,18 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     }
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    DCHECK(attributeIdInRange(attr_id));
+    DCHECK(tupleIdInRange(tid));
+    if (column_native_[attr_id]) {
+      return static_cast<const NativeColumnVector&>(*columns_[attr_id]).getUntypedValueAndByteLength<check_null>(tid);
+    } else {
+      return static_cast<const IndirectColumnVector&>(*columns_[attr_id]).getUntypedValueAndByteLength<check_null>(tid);
+    }
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     DCHECK(attributeIdInRange(attr_id));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/BloomFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.hpp b/utility/BloomFilter.hpp
index b93df84..973ca14 100644
--- a/utility/BloomFilter.hpp
+++ b/utility/BloomFilter.hpp
@@ -26,6 +26,7 @@
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <cstring>
 #include <memory>
 #include <utility>
 #include <vector>
@@ -44,11 +45,358 @@ namespace quickstep {
  *  @{
  */
 
+class BloomFilterOriginal;
+class BloomFilterBlocked;
+typedef BloomFilterBlocked BloomFilter;
+
+/**
+ * @brief A "blocked" version of Bloom Filter based on this paper:
+ *        Putze, Felix, Peter Sanders, and Johannes Singler.
+ *        "Cache-, hash-and space-efficient bloom filters."
+ *        International Workshop on Experimental and Efficient Algorithms.
+ *        Springer Berlin Heidelberg, 2007.
+ **/
+class BloomFilterBlocked {
+ public:
+  static const std::uint8_t kNumBitsPerByte = 8;
+  static const std::uint8_t kMaxNumHashFns = 4;
+
+  // This union allows us to read/write position in convenient fashion,
+  // through nested structs and their bitfield members
+  //
+  // A position can simply be a 32-bit hash
+  // Or it can be a cache line (block of 512 bits) and position within it
+  // Or it can be a byte (block of 8 bits) and position within it
+  union Position {
+    std::uint32_t hash;
+    struct CacheLinePosition {
+      unsigned index_in_line : 9;
+      unsigned line_num : 23;
+    } cache_line_pos;
+    struct BytePosition {
+      unsigned index_in_byte : 3;
+      unsigned byte_num : 29;
+    } byte_pos;
+  };
+
+  // This Bloom filter implementation requires the bit array to be a
+  // multiple of the cache-line size. So we either have to round up to a 
+  // multiple (default behavior) or round down to a multiple.
+  // Rounding up is usually preferable but rounding down is necessary when
+  // we are given a bit array that we don't control the size of, in the
+  // constructor.
+  static std::uint64_t getNearestAllowedSize(
+      const std::uint64_t approx_size,
+      bool round_down = false) {
+    if (round_down)
+      return (approx_size / kCacheLineBytes) * kCacheLineBytes;
+    return ((approx_size + kCacheLineBytes - 1)/ kCacheLineBytes) * kCacheLineBytes;
+  }
+
+
+  /**
+   * @brief Constructor.
+   * @note When no bit_array is being passed to the constructor,
+   *       then the bit_array is owned and managed by this class.
+   *
+   * @param hash_fn_count The number of hash functions used by this bloom filter.
+   * @param bit_array_size_in_bytes Size of the bit array.
+   **/
+  BloomFilterBlocked(const std::uint8_t hash_fn_count,
+              const std::uint64_t bit_array_size_in_bytes)
+      : hash_fn_count_(hash_fn_count),
+        array_size_in_bytes_(getNearestAllowedSize(bit_array_size_in_bytes)),
+        is_bit_array_owner_(true),
+        bit_array_(new std::uint8_t[array_size_in_bytes_]) {
+    reset();
+  }
+
+  /**
+   * @brief Constructor.
+   * @note When a bit_array is passed as an argument to the constructor,
+   *       then the ownership of the bit array lies with the caller.
+   *
+   * @param hash_fn_count The number of hash functions used by this bloom filter.
+   * @param bit_array_size_in_bytes Size of the bit array.
+   * @param bit_array A pointer to the memory region that is used to store bit array.
+   * @param is_initialized A boolean that indicates whether to zero-out the region
+   *                       before use or not.
+   **/
+  BloomFilterBlocked(const std::uint8_t hash_fn_count,
+              const std::uint64_t bit_array_size_in_bytes,
+              std::uint8_t *bit_array,
+              const bool is_initialized)
+      : hash_fn_count_(hash_fn_count),
+        array_size_in_bytes_(getNearestAllowedSize(bit_array_size_in_bytes, true)),
+        is_bit_array_owner_(false),
+        bit_array_(bit_array) {  // Owned by the calling method.
+    if (!is_initialized) {
+      reset();
+    }
+  }
+
+  /**
+   * @brief Constructor.
+   * @note When a bloom filter proto is passed as an initializer,
+   *       then the bit_array is owned and managed by this class.
+   *
+   * @param bloom_filter_proto The protobuf representation of a
+   *        bloom filter configuration.
+   **/
+  explicit BloomFilterBlocked(const serialization::BloomFilter &bloom_filter_proto)
+      : hash_fn_count_(bloom_filter_proto.number_of_hashes()),
+        array_size_in_bytes_(bloom_filter_proto.bloom_filter_size()),
+        is_bit_array_owner_(true),
+        bit_array_(new std::uint8_t[array_size_in_bytes_]) {
+    reset();
+  }
+
+  /**
+   * @brief Destructor.
+   **/
+  ~BloomFilterBlocked() {
+    if (is_bit_array_owner_) {
+      bit_array_.reset();
+    } else {
+      bit_array_.release();
+    }
+  }
+
+  static bool ProtoIsValid(const serialization::BloomFilter &bloom_filter_proto) {
+    return bloom_filter_proto.IsInitialized();
+  }
+
+  /**
+   * @brief Zeros out the contents of the bit array.
+   **/
+  inline void reset() {
+    // Initialize the bit_array with all zeros.
+    std::fill_n(bit_array_.get(), array_size_in_bytes_, 0x00);
+    inserted_element_count_ = 0;
+  }
+
+  /**
+   * @brief Get the number of hash functions used in this bloom filter.
+   *
+   * @return Returns the number of hash functions.
+   **/
+  inline std::uint8_t getNumberOfHashes() const {
+    return hash_fn_count_;
+  }
+
+  /**
+   * @brief Get the size of the bit array in bytes for this bloom filter.
+   *
+   * @return Returns the bit array size (in bytes).
+   **/
+  inline std::uint64_t getBitArraySize() const {
+    return array_size_in_bytes_;
+  }
+
+  /**
+   * @brief Get the constant pointer to the bit array for this bloom filter
+   *
+   * @return Returns constant pointer to the bit array.
+   **/
+  inline const std::uint8_t* getBitArray() const {
+    return bit_array_.get();
+  }
+
+  template <typename T>
+  void insert(const T &value) {
+    insert(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+  }
+
+  /**
+   * @brief Inserts a given value into the bloom filter in a thread-safe manner.
+   *
+   * @param key_begin A pointer to the value being inserted.
+   * @param length Size of the value being inserted in bytes.
+   */
+  inline void insert(const std::uint8_t *key_begin, const std::size_t length) {
+      SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
+      insertUnSafe(key_begin, length);
+  }
+
+  template <typename T>
+  void insertUnSafe(const T &value) {
+    insertUnSafe(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+  }
+
+  /**
+   * @brief Inserts a given value into the bloom filter.
+   * @Warning This is a faster thread-unsafe version of the insert() function.
+   *          The caller needs to ensure the thread safety.
+   *
+   * @param key_begin A pointer to the value being inserted.
+   * @param length Size of the value being inserted in bytes.
+   */
+  inline void insertUnSafe(const std::uint8_t *key_begin, const std::size_t length) {
+    Position first_pos = getFirstPosition(key_begin, length);
+    setBitAtPosition(first_pos);
+    Position other_pos;
+    for (std::uint8_t i = 1; i <hash_fn_count_; ++i) {
+      other_pos = getOtherPosition(key_begin, length, first_pos, i);
+      setBitAtPosition(other_pos);
+    }
+    ++inserted_element_count_;
+  }
+
+  template <typename T>
+  bool contains(const T &value) {
+    return contains(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+  }
+
+  /**
+   * @brief Test membership of a given value in the bloom filter.
+   *        If true is returned, then a value may or may not be present in the bloom filter.
+   *        If false is returned, a value is certainly not present in the bloom filter.
+   *
+   * @note The membersip test does not require any locks, because the assumption is that
+   *       the bloom filter will only be used after it has been built.
+   *
+   * @param key_begin A pointer to the value being tested for membership.
+   * @param length Size of the value being inserted in bytes.
+   */
+  inline bool contains(
+      const std::uint8_t *__restrict__ key_begin,
+      const std::size_t length) const {
+    Position first_pos = getFirstPosition(key_begin, length);
+    if (!getBitAtPosition(first_pos)) {
+      return false;
+    }
+    Position other_pos;
+    for (std::uint8_t i = 1; i < hash_fn_count_; ++i) {
+      other_pos = getOtherPosition(key_begin, length, first_pos, i);
+      if (!getBitAtPosition(other_pos)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * @brief Perform a bitwise-OR of the given Bloom filter with this bloom filter.
+   *        Essentially, it does a union of this bloom filter with the passed bloom filter.
+   *
+   * @param bloom_filter A const pointer to the bloom filter object to do bitwise-OR with.
+   */
+  inline void bitwiseOr(const BloomFilterBlocked *bloom_filter) {
+    SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
+    for (std::size_t byte_index = 0; byte_index < bloom_filter->getBitArraySize(); ++byte_index) {
+      (bit_array_.get())[byte_index] |= bloom_filter->getBitArray()[byte_index];
+    }
+  }
+
+  /**
+   * @brief Return the number of elements currently inserted into bloom filter.
+   *
+   * @return The number of elements inserted into bloom filter.
+   **/
+  inline std::uint32_t element_count() const {
+    return inserted_element_count_;
+  }
+
+ protected:
+  Position getFirstPosition(const std::uint8_t *begin, std::size_t length) const {
+    Position pos;
+    pos.hash = hash_identity(begin, length);
+    return pos;
+  }
+
+  Position getOtherPosition(
+      const std::uint8_t *begin,
+      std::size_t length,
+      const Position first_pos,
+      const std::uint8_t index) const {
+    Position pos;
+    pos.hash = hash_multiplicative(begin, length, hash_fn_[index-1]);
+    pos.cache_line_pos.line_num = first_pos.cache_line_pos.line_num;
+    return pos;
+  }
+
+  void fillPosition(
+      const std::uint8_t *begin,
+      std::size_t length,
+      const std::uint8_t index,
+      Position positions[]) const {
+    if (index == 0)
+      positions[0].hash = hash_identity(begin, length);
+    else {
+      positions[index].hash = hash_multiplicative(begin, length, hash_fn_[index-1]);
+      positions[index].cache_line_pos.line_num = positions[0].cache_line_pos.line_num;
+    }
+  }
+
+  void setBitAtPosition(const Position &pos) {
+    (bit_array_.get())[pos.byte_pos.byte_num] |= (1 << pos.byte_pos.index_in_byte);
+  }
+
+  bool getBitAtPosition(const Position &pos) const {
+    return (bit_array_.get())[pos.byte_pos.byte_num] & (1 << pos.byte_pos.index_in_byte);
+  }
+
+  inline std::uint32_t hash_identity(
+      const std::uint8_t *__restrict__ begin,
+      std::size_t length) const {
+    std::uint32_t hash;
+    if (length >= 4)
+      hash = *reinterpret_cast<const std::uint32_t*> (begin);
+    else
+      std::memcpy(&hash, begin, length);
+    return hash % (array_size_in_bytes_ * kNumBitsPerByte);
+  }
+
+  inline std::uint32_t hash_multiplicative(
+      const std::uint8_t *__restrict__ begin,
+      std::size_t length,
+      const std::uint64_t multiplier) const {
+    std::uint32_t hash = 0;
+    std::size_t bytes_hashed = 0;
+    if (length >= 4) {
+      while (bytes_hashed < length) {
+        auto val = *reinterpret_cast<const std::uint32_t *>(begin + bytes_hashed);
+        hash += (multiplier * val) >> 24;
+        bytes_hashed += 4;
+      }
+    }
+    while (bytes_hashed < length) {
+      std::uint8_t val = *(begin + bytes_hashed);
+      hash += (multiplier * val) >> 24;
+      bytes_hashed++;
+    }
+    return hash;//  % (array_size_in_bytes_ * kNumBitsPerByte);
+  }
+
+ private:
+  const std::uint32_t hash_fn_count_;
+  const std::uint64_t array_size_in_bytes_;
+  std::uint32_t inserted_element_count_;
+  const bool is_bit_array_owner_;
+
+  static constexpr std::uint64_t kKnuthGoldenRatioNumber = 2654435761;
+  const std::uint64_t hash_fn_[kMaxNumHashFns] = { // hash_fn_[i] is 2**(i+1) - 1
+    0x00000001 * kKnuthGoldenRatioNumber, // 0x00000003, 0x00000007, 0x0000000f,
+    // 0x0000001f * kKnuthGoldenRatioNumber, // 0x0000003f, 0x0000007f, 0x000000ff,
+    0x000001ff * kKnuthGoldenRatioNumber, // 0x000003ff, 0x000007ff, 0x00000fff,
+    // 0x00001fff * kKnuthGoldenRatioNumber, // 0x00003fff, 0x00007fff, 0x0000ffff,
+    0x0001ffff * kKnuthGoldenRatioNumber, // 0x0003ffff, 0x0007ffff, 0x000fffff,
+    // 0x001fffff * kKnuthGoldenRatioNumber, // 0x003fffff, 0x007fffff, 0x00ffffff,
+    0x01ffffff * kKnuthGoldenRatioNumber, // 0x03ffffff, 0x07ffffff, 0x0fffffff,
+    // 0x1fffffff * kKnuthGoldenRatioNumber  // 0x3fffffff, 0x7fffffff, 0xffffffff
+    };
+
+  alignas(kCacheLineBytes) std::unique_ptr<std::uint8_t> bit_array_;
+  alignas(kCacheLineBytes) mutable SpinSharedMutex<false> bloom_filter_insert_mutex_;
+
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterBlocked);
+};
+
 /**
  * @brief A simple Bloom Filter implementation with basic primitives
  *        based on Partow's Bloom Filter implementation.
  **/
-class BloomFilter {
+class BloomFilterOriginal {
  public:
   static const uint32_t kNumBitsPerByte = 8;
 
@@ -57,21 +405,17 @@ class BloomFilter {
    * @note When no bit_array is being passed to the constructor,
    *       then the bit_array is owned and managed by this class.
    *
-   * @param random_seed A random_seed that generates unique hash functions.
    * @param hash_fn_count The number of hash functions used by this bloom filter.
    * @param bit_array_size_in_bytes Size of the bit array.
    **/
-  BloomFilter(const std::uint64_t random_seed,
-              const std::size_t hash_fn_count,
+  BloomFilterOriginal(const std::size_t hash_fn_count,
               const std::uint64_t bit_array_size_in_bytes)
-      : random_seed_(random_seed),
-        hash_fn_count_(hash_fn_count),
+      : hash_fn_count_(hash_fn_count),
         array_size_in_bytes_(bit_array_size_in_bytes),
         array_size_(array_size_in_bytes_ * kNumBitsPerByte),
         bit_array_(new std::uint8_t[array_size_in_bytes_]),
         is_bit_array_owner_(true) {
     reset();
-    generate_unique_hash_fn();
   }
 
   /**
@@ -79,20 +423,17 @@ class BloomFilter {
    * @note When a bit_array is passed as an argument to the constructor,
    *       then the ownership of the bit array lies with the caller.
    *
-   * @param random_seed A random_seed that generates unique hash functions.
    * @param hash_fn_count The number of hash functions used by this bloom filter.
    * @param bit_array_size_in_bytes Size of the bit array.
    * @param bit_array A pointer to the memory region that is used to store bit array.
    * @param is_initialized A boolean that indicates whether to zero-out the region
    *                       before use or not.
    **/
-  BloomFilter(const std::uint64_t random_seed,
-              const std::size_t hash_fn_count,
+  BloomFilterOriginal(const std::size_t hash_fn_count,
               const std::uint64_t bit_array_size_in_bytes,
               std::uint8_t *bit_array,
               const bool is_initialized)
-      : random_seed_(random_seed),
-        hash_fn_count_(hash_fn_count),
+      : hash_fn_count_(hash_fn_count),
         array_size_in_bytes_(bit_array_size_in_bytes),
         array_size_(bit_array_size_in_bytes * kNumBitsPerByte),
         bit_array_(bit_array),  // Owned by the calling method.
@@ -100,7 +441,6 @@ class BloomFilter {
     if (!is_initialized) {
       reset();
     }
-    generate_unique_hash_fn();
   }
 
   /**
@@ -111,21 +451,19 @@ class BloomFilter {
    * @param bloom_filter_proto The protobuf representation of a
    *        bloom filter configuration.
    **/
-  explicit BloomFilter(const serialization::BloomFilter &bloom_filter_proto)
-      : random_seed_(bloom_filter_proto.bloom_filter_seed()),
-        hash_fn_count_(bloom_filter_proto.number_of_hashes()),
+  explicit BloomFilterOriginal(const serialization::BloomFilter &bloom_filter_proto)
+      : hash_fn_count_(bloom_filter_proto.number_of_hashes()),
         array_size_in_bytes_(bloom_filter_proto.bloom_filter_size()),
         array_size_(array_size_in_bytes_ * kNumBitsPerByte),
         bit_array_(new std::uint8_t[array_size_in_bytes_]),
         is_bit_array_owner_(true) {
     reset();
-    generate_unique_hash_fn();
   }
 
   /**
    * @brief Destructor.
    **/
-  ~BloomFilter() {
+  ~BloomFilterOriginal() {
     if (is_bit_array_owner_) {
       bit_array_.reset();
     } else {
@@ -147,15 +485,6 @@ class BloomFilter {
   }
 
   /**
-   * @brief Get the random seed that was used to initialize this bloom filter.
-   *
-   * @return Returns the random seed.
-   **/
-  inline std::uint64_t getRandomSeed() const {
-    return random_seed_;
-  }
-
-  /**
    * @brief Get the number of hash functions used in this bloom filter.
    *
    * @return Returns the number of hash functions.
@@ -198,7 +527,7 @@ class BloomFilter {
 
     // Determine all the bit positions that are required to be set.
     for (std::size_t i = 0; i < hash_fn_count_; ++i) {
-      compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+      compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
       modified_bit_positions.push_back(std::make_pair(bit_index, bit));
     }
 
@@ -243,7 +572,7 @@ class BloomFilter {
     std::size_t bit = 0;
 
     for (std::size_t i = 0; i < hash_fn_count_; ++i) {
-      compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+      compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
       (bit_array_.get())[bit_index / kNumBitsPerByte] |= (1 << bit);
     }
 
@@ -265,7 +594,7 @@ class BloomFilter {
     std::size_t bit_index = 0;
     std::size_t bit = 0;
     for (std::size_t i = 0; i < hash_fn_count_; ++i) {
-      compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+      compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
       if (((bit_array_.get())[bit_index / kNumBitsPerByte] & (1 << bit)) != (1 << bit)) {
         return false;
       }
@@ -279,7 +608,7 @@ class BloomFilter {
    *
    * @param bloom_filter A const pointer to the bloom filter object to do bitwise-OR with.
    */
-  inline void bitwiseOr(const BloomFilter *bloom_filter) {
+  inline void bitwiseOr(const BloomFilterOriginal *bloom_filter) {
     SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
     for (std::size_t byte_index = 0; byte_index < bloom_filter->getBitArraySize(); ++byte_index) {
       (bit_array_.get())[byte_index] |= bloom_filter->getBitArray()[byte_index];
@@ -301,95 +630,28 @@ class BloomFilter {
     *bit = *bit_index % kNumBitsPerByte;
   }
 
-  void generate_unique_hash_fn() {
-    hash_fn_.reserve(hash_fn_count_);
-    const std::uint32_t predef_hash_fn_count = 128;
-    static const std::uint32_t predef_hash_fn[predef_hash_fn_count] = {
-       0xAAAAAAAA, 0x55555555, 0x33333333, 0xCCCCCCCC,
-       0x66666666, 0x99999999, 0xB5B5B5B5, 0x4B4B4B4B,
-       0xAA55AA55, 0x55335533, 0x33CC33CC, 0xCC66CC66,
-       0x66996699, 0x99B599B5, 0xB54BB54B, 0x4BAA4BAA,
-       0xAA33AA33, 0x55CC55CC, 0x33663366, 0xCC99CC99,
-       0x66B566B5, 0x994B994B, 0xB5AAB5AA, 0xAAAAAA33,
-       0x555555CC, 0x33333366, 0xCCCCCC99, 0x666666B5,
-       0x9999994B, 0xB5B5B5AA, 0xFFFFFFFF, 0xFFFF0000,
-       0xB823D5EB, 0xC1191CDF, 0xF623AEB3, 0xDB58499F,
-       0xC8D42E70, 0xB173F616, 0xA91A5967, 0xDA427D63,
-       0xB1E8A2EA, 0xF6C0D155, 0x4909FEA3, 0xA68CC6A7,
-       0xC395E782, 0xA26057EB, 0x0CD5DA28, 0x467C5492,
-       0xF15E6982, 0x61C6FAD3, 0x9615E352, 0x6E9E355A,
-       0x689B563E, 0x0C9831A8, 0x6753C18B, 0xA622689B,
-       0x8CA63C47, 0x42CC2884, 0x8E89919B, 0x6EDBD7D3,
-       0x15B6796C, 0x1D6FDFE4, 0x63FF9092, 0xE7401432,
-       0xEFFE9412, 0xAEAEDF79, 0x9F245A31, 0x83C136FC,
-       0xC3DA4A8C, 0xA5112C8C, 0x5271F491, 0x9A948DAB,
-       0xCEE59A8D, 0xB5F525AB, 0x59D13217, 0x24E7C331,
-       0x697C2103, 0x84B0A460, 0x86156DA9, 0xAEF2AC68,
-       0x23243DA5, 0x3F649643, 0x5FA495A8, 0x67710DF8,
-       0x9A6C499E, 0xDCFB0227, 0x46A43433, 0x1832B07A,
-       0xC46AFF3C, 0xB9C8FFF0, 0xC9500467, 0x34431BDF,
-       0xB652432B, 0xE367F12B, 0x427F4C1B, 0x224C006E,
-       0x2E7E5A89, 0x96F99AA5, 0x0BEB452A, 0x2FD87C39,
-       0x74B2E1FB, 0x222EFD24, 0xF357F60C, 0x440FCB1E,
-       0x8BBE030F, 0x6704DC29, 0x1144D12F, 0x948B1355,
-       0x6D8FD7E9, 0x1C11A014, 0xADD1592F, 0xFB3C712E,
-       0xFC77642F, 0xF9C4CE8C, 0x31312FB9, 0x08B0DD79,
-       0x318FA6E7, 0xC040D23D, 0xC0589AA7, 0x0CA5C075,
-       0xF874B172, 0x0CF914D5, 0x784D3280, 0x4E8CFEBC,
-       0xC569F575, 0xCDB2A091, 0x2CC016B4, 0x5C5F4421
-    };
-    if (hash_fn_count_ <= predef_hash_fn_count) {
-      std::copy(predef_hash_fn, predef_hash_fn + hash_fn_count_, hash_fn_.begin());
-      for (std::uint32_t i = 0; i < hash_fn_.size(); ++i) {
-        hash_fn_[i] = hash_fn_[i] * hash_fn_[(i + 3) % hash_fn_count_] + static_cast<std::uint32_t>(random_seed_);
+  inline std::uint32_t hash_multiplicative(
+      const std::uint8_t *begin,
+      std::size_t remaining_length,
+      const std::uint64_t multiplier) const {
+    std::uint32_t hash = 0;
+    std::size_t bytes_hashed = 0;
+    if (remaining_length >= 4) {
+      while (bytes_hashed < remaining_length) {
+        auto val = *reinterpret_cast<const std::uint32_t *>(begin + bytes_hashed);
+        hash += (multiplier * val) >> 32;
+        bytes_hashed += 4;
       }
-    } else {
-      LOG(FATAL) << "Requested number of hash functions is too large.";
     }
-  }
-
-  inline std::uint32_t hash_ap(const std::uint8_t *begin, std::size_t remaining_length, std::uint32_t hash) const {
-    const std::uint8_t *itr = begin;
-    std::uint32_t loop = 0;
-    while (remaining_length >= 8) {
-      const std::uint32_t &i1 = *(reinterpret_cast<const std::uint32_t*>(itr)); itr += sizeof(std::uint32_t);
-      const std::uint32_t &i2 = *(reinterpret_cast<const std::uint32_t*>(itr)); itr += sizeof(std::uint32_t);
-      hash ^= (hash <<  7) ^  i1 * (hash >> 3) ^ (~((hash << 11) + (i2 ^ (hash >> 5))));
-      remaining_length -= 8;
-    }
-    if (remaining_length) {
-      if (remaining_length >= 4) {
-        const std::uint32_t &i = *(reinterpret_cast<const std::uint32_t*>(itr));
-        if (loop & 0x01) {
-          hash ^= (hash <<  7) ^  i * (hash >> 3);
-        } else {
-          hash ^= (~((hash << 11) + (i ^ (hash >> 5))));
-        }
-        ++loop;
-        remaining_length -= 4;
-        itr += sizeof(std::uint32_t);
-      }
-      if (remaining_length >= 2) {
-        const std::uint16_t &i = *(reinterpret_cast<const std::uint16_t*>(itr));
-        if (loop & 0x01) {
-          hash ^= (hash <<  7) ^  i * (hash >> 3);
-        } else {
-          hash ^= (~((hash << 11) + (i ^ (hash >> 5))));
-        }
-        ++loop;
-        remaining_length -= 2;
-        itr += sizeof(std::uint16_t);
-      }
-      if (remaining_length) {
-        hash += ((*itr) ^ (hash * 0xA5A5A5A5)) + loop;
-      }
+    while (bytes_hashed < remaining_length) {
+      std::uint8_t val = *(begin + bytes_hashed);
+      hash += (multiplier * val) >> 32;
+      bytes_hashed++;
     }
     return hash;
   }
 
  private:
-  const std::uint64_t random_seed_;
-  std::vector<std::uint32_t> hash_fn_;
   const std::uint32_t hash_fn_count_;
   std::uint64_t array_size_in_bytes_;
   std::uint64_t array_size_;
@@ -397,9 +659,21 @@ class BloomFilter {
   std::uint32_t inserted_element_count_;
   const bool is_bit_array_owner_;
 
+  static constexpr std::uint64_t kKnuthGoldenRatioNumber = 2654435761;
+  static constexpr std::size_t kMaxNumHashFns = 8;
+  const std::uint64_t hash_fn_[kMaxNumHashFns] = { // hash_fn_[i] is 2**(i+1) - 1
+    0x00000001 * kKnuthGoldenRatioNumber, // 0x00000003, 0x00000007, 0x0000000f,
+    0x0000001f * kKnuthGoldenRatioNumber, // 0x0000003f, 0x0000007f, 0x000000ff,
+    0x000001ff * kKnuthGoldenRatioNumber, // 0x000003ff, 0x000007ff, 0x00000fff,
+    0x00001fff * kKnuthGoldenRatioNumber, // 0x00003fff, 0x00007fff, 0x0000ffff,
+    0x0001ffff * kKnuthGoldenRatioNumber, // 0x0003ffff, 0x0007ffff, 0x000fffff,
+    0x001fffff * kKnuthGoldenRatioNumber, // 0x003fffff, 0x007fffff, 0x00ffffff,
+    0x01ffffff * kKnuthGoldenRatioNumber, // 0x03ffffff, 0x07ffffff, 0x0fffffff,
+    0x1fffffff * kKnuthGoldenRatioNumber  // 0x3fffffff, 0x7fffffff, 0xffffffff
+    };
   alignas(kCacheLineBytes) mutable SpinSharedMutex<false> bloom_filter_insert_mutex_;
 
-  DISALLOW_COPY_AND_ASSIGN(BloomFilter);
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterOriginal);
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/BloomFilter.proto
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.proto b/utility/BloomFilter.proto
index 8dd9163..b5d14a9 100644
--- a/utility/BloomFilter.proto
+++ b/utility/BloomFilter.proto
@@ -21,10 +21,8 @@ message BloomFilter {
   // The default values were determined from empirical experiments.
   // These values control the amount of false positivity that
   // is expected from Bloom Filter.
-  // - Default seed for initializing family of hashes = 0xA5A5A5A55A5A5A5A.
   // - Default bloom filter size = 10 KB.
   // - Default number of hash functions used in bloom filter = 5.
-  optional fixed64 bloom_filter_seed = 1 [default = 0xA5A5A5A55A5A5A5A];
-  optional uint32 bloom_filter_size = 2 [default = 10000];
-  optional uint32 number_of_hashes = 3 [default = 5];
+  optional uint32 bloom_filter_size = 1 [default = 10000];
+  optional uint32 number_of_hashes = 2 [default = 5];
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/BloomFilterAdapter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilterAdapter.hpp b/utility/BloomFilterAdapter.hpp
new file mode 100644
index 0000000..f094307
--- /dev/null
+++ b/utility/BloomFilterAdapter.hpp
@@ -0,0 +1,142 @@
+/**
+ *   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_UTILITY_BLOOM_FILTER_ADAPTER_HPP
+#define QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+class BloomFilterAdapter {
+ public:
+  BloomFilterAdapter(const std::vector<const BloomFilter*> &bloom_filters,
+                     const std::vector<attribute_id> &attribute_ids,
+                     const std::vector<std::size_t> &attr_sizes) {
+    DCHECK_EQ(bloom_filters.size(), attribute_ids.size());
+    DCHECK_EQ(bloom_filters.size(), attr_sizes.size());
+
+    bloom_filter_entries_.reserve(bloom_filters.size());
+    for (std::size_t i = 0; i < bloom_filters.size(); ++i) {
+      bloom_filter_entries_.emplace_back(
+          new BloomFilterEntry(
+              bloom_filters[i], attribute_ids[i], attr_sizes[i]));
+    }
+  }
+
+  ~BloomFilterAdapter() {
+    for (auto &entry : bloom_filter_entries_) {
+      delete entry;
+    }
+  }
+
+  template <bool adapt_filters, typename ValueAccessorT>
+  inline std::size_t bulkProbe(const ValueAccessorT *accessor,
+                               std::vector<tuple_id> &batch,
+                               const std::size_t batch_size) {
+    std::size_t out_size = batch_size;
+    for (auto &entry : bloom_filter_entries_) {
+      out_size = bulkProbeBloomFilterEntry<adapt_filters>(*entry, accessor, batch, out_size);
+    }
+    adaptEntryOrder();
+    return out_size;
+  }
+
+ private:
+  struct BloomFilterEntry {
+    BloomFilterEntry(const BloomFilter *in_bloom_filter,
+                     const attribute_id &in_attribute_id,
+                     const std::size_t &in_attribute_size)
+        : bloom_filter(in_bloom_filter),
+          attribute_id(in_attribute_id),
+          attribute_size(in_attribute_size),
+          miss(0),
+          cnt(0) {
+    }
+
+    static bool isBetterThan(const BloomFilterEntry *a,
+                             const BloomFilterEntry *b) {
+      return a->miss_rate > b->miss_rate;
+    }
+
+    const BloomFilter *bloom_filter;
+    const attribute_id attribute_id;
+    const std::size_t attribute_size;
+    std::uint32_t miss;
+    std::uint32_t cnt;
+    float miss_rate;
+  };
+
+  template <bool adapt_filters, typename ValueAccessorT>
+  inline std::size_t bulkProbeBloomFilterEntry(
+      BloomFilterEntry &entry,
+      const ValueAccessorT *accessor,
+      std::vector<tuple_id> &batch,
+      const std::size_t in_size) {
+    std::size_t out_size = 0;
+    const BloomFilter *bloom_filter = entry.bloom_filter;
+
+    for (std::size_t t = 0; t < in_size; ++t) {
+      const tuple_id tid = batch[t];
+      const auto value = static_cast<const std::uint8_t*>(
+          accessor->getUntypedValueAtAbsolutePosition(entry.attribute_id, tid));
+      if (bloom_filter->contains(value, entry.attribute_size)) {
+        batch[out_size] = tid;
+        ++out_size;
+      }
+    }
+    if (adapt_filters) {
+      entry.cnt += in_size;
+      entry.miss += (in_size - out_size);
+    }
+    return out_size;
+  }
+
+  inline void adaptEntryOrder() {
+    for (auto &entry : bloom_filter_entries_) {
+      entry->miss_rate = static_cast<float>(entry->miss) / entry->cnt;
+    }
+    std::sort(bloom_filter_entries_.begin(),
+              bloom_filter_entries_.end(),
+              BloomFilterEntry::isBetterThan);
+  }
+
+  std::vector<BloomFilterEntry *> bloom_filter_entries_;
+
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterAdapter);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 2d3db8f..5883470 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -159,6 +159,7 @@ add_library(quickstep_utility_Alignment ../empty_src.cpp Alignment.hpp)
 add_library(quickstep_utility_BitManipulation ../empty_src.cpp BitManipulation.hpp)
 add_library(quickstep_utility_BitVector ../empty_src.cpp BitVector.hpp)
 add_library(quickstep_utility_BloomFilter ../empty_src.cpp BloomFilter.hpp)
+add_library(quickstep_utility_BloomFilterAdapter ../empty_src.cpp BloomFilterAdapter.hpp)
 add_library(quickstep_utility_BloomFilter_proto
             ${quickstep_utility_BloomFilter_proto_srcs}
             ${quickstep_utility_BloomFilter_proto_hdrs})
@@ -166,6 +167,9 @@ add_library(quickstep_utility_CalculateInstalledMemory CalculateInstalledMemory.
 add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
+add_library(quickstep_utility_DAGVisualizer DAGVisualizer.cpp DAGVisualizer.hpp)
+add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
+add_library(quickstep_utility_EventProfiler EventProfiler.cpp EventProfiler.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
 add_library(quickstep_utility_Glob Glob.cpp Glob.hpp)
 add_library(quickstep_utility_HashPair ../empty_src.cpp HashPair.hpp)
@@ -216,6 +220,10 @@ target_link_libraries(quickstep_utility_BloomFilter
                       quickstep_threading_SpinSharedMutex
                       quickstep_utility_BloomFilter_proto
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_BloomFilterAdapter
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_utility_BloomFilter
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_BloomFilter_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_utility_CalculateInstalledMemory
@@ -225,6 +233,8 @@ target_link_libraries(quickstep_utility_CheckSnprintf
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_EventProfiler
+                      quickstep_threading_Mutex)
 target_link_libraries(quickstep_utility_Glob
                       glog)
 target_link_libraries(quickstep_utility_MemStream
@@ -243,6 +253,12 @@ target_link_libraries(quickstep_utility_PlanVisualizer
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_utility_Macros
                       quickstep_utility_StringUtil)
+target_link_libraries(quickstep_utility_DAGVisualizer
+                      quickstep_queryoptimizer_QueryPlan
+                      quickstep_utility_EventProfiler
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
+target_link_libraries(quickstep_utility_DisjointTreeForest)
 target_link_libraries(quickstep_utility_PtrList
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_PtrMap
@@ -297,11 +313,15 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_BitManipulation
                       quickstep_utility_BitVector
                       quickstep_utility_BloomFilter
+                      quickstep_utility_BloomFilterAdapter
                       quickstep_utility_BloomFilter_proto
                       quickstep_utility_CalculateInstalledMemory
                       quickstep_utility_Cast
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_DAG
+                      quickstep_utility_DAGVisualizer
+                      quickstep_utility_DisjointTreeForest
+                      quickstep_utility_EventProfiler
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Glob
                       quickstep_utility_HashPair

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/DAGVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/DAGVisualizer.cpp b/utility/DAGVisualizer.cpp
new file mode 100644
index 0000000..e62f948
--- /dev/null
+++ b/utility/DAGVisualizer.cpp
@@ -0,0 +1,167 @@
+/**
+ *   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 "utility/DAGVisualizer.hpp"
+#include "utility/EventProfiler.hpp"
+
+#include <cmath>
+#include <cstddef>
+#include <iomanip>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "query_optimizer/QueryPlan.hpp"
+#include "utility/EventProfiler.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+std::string DAGVisualizer::toDOT() {
+  std::set<std::string> no_display_op_names =
+      { "DestroyHashOperator", "DropTableOperator" };
+
+  const auto &dag = plan_.getQueryPlanDAG();
+  const std::size_t num_nodes = dag.size();
+
+  std::vector<double> time_elapsed(num_nodes, 0);
+  std::vector<double> time_percentage(num_nodes, 0);
+  std::vector<double> time_start(num_nodes, std::numeric_limits<double>::max());
+  std::vector<double> time_end(num_nodes, 0);
+  const auto &zero_time = relop_profiler.zero_time();
+  for (const auto &container : relop_profiler.containers()) {
+    for (const auto &line : container.second.events) {
+      const std::size_t relop_index = line.first;
+      for (const auto &event : line.second) {
+        time_elapsed[relop_index] +=
+            std::chrono::duration<double>(event.end_time - event.start_time).count();
+        time_start[relop_index] =
+            std::min(time_start[relop_index],
+                     std::chrono::duration<double>(event.start_time - zero_time).count());
+        time_end[relop_index] =
+            std::max(time_end[relop_index],
+                     std::chrono::duration<double>(event.end_time - zero_time).count());
+      }
+    }
+  }
+  const std::size_t num_threads = relop_profiler.containers().size();
+  double total_time_elapsed = 0;
+  double max_percentage = 0;
+  for (std::size_t i = 0; i < time_elapsed.size(); ++i) {
+    time_elapsed[i] /= num_threads;
+    total_time_elapsed += time_elapsed[i];
+  }
+  for (std::size_t i = 0; i < time_elapsed.size(); ++i) {
+    time_percentage[i] = time_elapsed[i] / total_time_elapsed;
+    max_percentage = std::max(max_percentage, time_percentage[i]);
+  }
+
+  std::vector<bool> display_ops(num_nodes, false);
+  for (std::size_t node_index = 0; node_index < num_nodes; ++node_index) {
+    const auto &node = dag.getNodePayload(node_index);
+    const std::string relop_name = node.getName();
+    if (no_display_op_names.find(relop_name) == no_display_op_names.end()) {
+      display_ops[node_index] = true;
+
+      nodes_.emplace_back();
+      NodeInfo &node_info = nodes_.back();
+      node_info.id = node_index;
+
+      std::string hue =
+          std::to_string(std::sqrt(time_percentage[node_index] / max_percentage));
+      node_info.color = hue + " " + hue + " 1.0";
+
+      node_info.labels.emplace_back(
+          "[" + std::to_string(node.getOperatorIndex()) + "] " + relop_name);
+      node_info.labels.emplace_back(
+          std::to_string(std::lround(time_elapsed[node_index] * 1000)) +
+          "ms (" + PercentageToString(time_percentage[node_index] * 100) + "%)");
+      node_info.labels.emplace_back(
+          "span: [" +
+          std::to_string(std::lround(time_start[node_index] * 1000)) + "ms, " +
+          std::to_string(std::lround(time_end[node_index] * 1000)) + "ms]");
+    }
+  }
+  for (std::size_t node_index = 0; node_index < num_nodes; ++node_index) {
+    if (display_ops[node_index]) {
+      for (const auto &link : dag.getDependents(node_index)) {
+        if (display_ops[link.first]) {
+          edges_.emplace_back();
+          EdgeInfo &edge_info = edges_.back();
+          edge_info.src_node_id = node_index;
+          edge_info.dst_node_id = link.first;
+          edge_info.is_pipeline_breaker = link.second;
+        }
+      }
+    }
+  }
+
+  // Format output graph
+  std::ostringstream graph_oss;
+  graph_oss << "digraph g {\n";
+  graph_oss << "  rankdir=BT\n";
+  graph_oss << "  node [penwidth=2]\n";
+  graph_oss << "  edge [fontsize=16 fontcolor=gray penwidth=2]\n\n";
+
+  // Format nodes
+  for (const NodeInfo &node_info : nodes_) {
+    graph_oss << "  " << node_info.id << " [ ";
+    if (!node_info.labels.empty()) {
+      graph_oss << "label=\""
+                << EscapeSpecialChars(JoinToString(node_info.labels, "&#10;"))
+                << "\" ";
+    }
+    if (!node_info.color.empty()) {
+      graph_oss << "style=filled fillcolor=\"" << node_info.color << "\" ";
+    }
+    graph_oss << "]\n";
+  }
+  graph_oss << "\n";
+
+  // Format edges
+  for (const EdgeInfo &edge_info : edges_) {
+    graph_oss << "  " << edge_info.src_node_id << " -> "
+              << edge_info.dst_node_id << " [ ";
+    if (edge_info.is_pipeline_breaker) {
+      graph_oss << "style=dashed ";
+    }
+    if (!edge_info.labels.empty()) {
+      graph_oss << "label=\""
+                << EscapeSpecialChars(JoinToString(edge_info.labels, "&#10;"))
+                << "\" ";
+    }
+    graph_oss << "]\n";
+  }
+
+  graph_oss << "}\n";
+
+  return graph_oss.str();
+}
+
+std::string DAGVisualizer::PercentageToString(double percentage) {
+  std::ostringstream oss;
+  oss << static_cast<std::uint32_t>(percentage) << ".";
+  int digits = std::lround(percentage * 10000) % 100;
+  oss << digits / 10 << digits % 10;
+  return oss.str();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/DAGVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/DAGVisualizer.hpp b/utility/DAGVisualizer.hpp
new file mode 100644
index 0000000..5c81d22
--- /dev/null
+++ b/utility/DAGVisualizer.hpp
@@ -0,0 +1,85 @@
+/**
+ *   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_UTILITY_DAG_VISUALIZER_HPP_
+#define QUICKSTEP_UTILITY_DAG_VISUALIZER_HPP_
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class QueryPlan;
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A visualizer that converts an execution plan DAG into a graph in
+ *        DOT format. Note that DOT is a plain text graph description language.
+ *
+ * @note This utility tool can be further extended to be more generic.
+ */
+class DAGVisualizer {
+ public:
+  DAGVisualizer(const QueryPlan &plan)
+      : plan_(plan) {}
+
+  ~DAGVisualizer() {}
+
+  std::string toDOT();
+
+ private:
+  static std::string PercentageToString(double percentage);
+
+  /**
+   * @brief Information of a graph node.
+   */
+  struct NodeInfo {
+    std::size_t id;
+    std::vector<std::string> labels;
+    std::string color;
+  };
+
+  /**
+   * @brief Information of a graph edge.
+   */
+  struct EdgeInfo {
+    std::size_t src_node_id;
+    std::size_t dst_node_id;
+    std::vector<std::string> labels;
+    bool is_pipeline_breaker;
+  };
+
+  const QueryPlan &plan_;
+
+  std::vector<NodeInfo> nodes_;
+  std::vector<EdgeInfo> edges_;
+
+  DISALLOW_COPY_AND_ASSIGN(DAGVisualizer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_UTILITY_DAG_VISUALIZER_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/DisjointTreeForest.hpp
----------------------------------------------------------------------
diff --git a/utility/DisjointTreeForest.hpp b/utility/DisjointTreeForest.hpp
new file mode 100644
index 0000000..f5722ba
--- /dev/null
+++ b/utility/DisjointTreeForest.hpp
@@ -0,0 +1,116 @@
+/**
+ *   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_UTILITY_DISJOINT_TREE_FOREST_HPP_
+#define QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
+
+#include <cstddef>
+#include <limits>
+#include <utility>
+#include <unordered_map>
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A.k.a. union-find set.
+ */
+template <typename ElementT,
+          class MapperT = std::unordered_map<ElementT, std::size_t>>
+class DisjointTreeForest {
+ public:
+  inline bool hasElement(const ElementT &element) const {
+    return elements_map_.find(element) != elements_map_.end();
+  }
+
+  inline void makeSet(const ElementT &element) {
+    if (!hasElement(element)) {
+      std::size_t loc = nodes_.size();
+      nodes_.emplace_back(0, loc);
+      elements_map_.emplace(element, loc);
+    }
+  }
+
+  inline std::size_t find(const ElementT &element) {
+    const std::size_t node_id = elements_map_.at(element);
+    std::size_t root_id = node_id;
+    std::size_t parent_id;
+    while ((parent_id = nodes_[root_id].parent) != root_id) {
+      root_id = parent_id;
+    }
+    compress_path(node_id, root_id);
+    return root_id;
+  }
+
+  inline void merge(const ElementT &element1, const ElementT &element2) {
+    std::size_t root_id1 = find(element1);
+    std::size_t root_id2 = find(element2);
+    if (root_id1 != root_id2) {
+      Node &n1 = nodes_[root_id1];
+      Node &n2 = nodes_[root_id2];
+      if (n1.rank > n2.rank) {
+        n2.parent = root_id1;
+      } else if (n1.rank < n2.rank) {
+        n1.parent = root_id2;
+      } else {
+        n1.parent = root_id2;
+        n2.rank += 1;
+      }
+    }
+  }
+
+  inline bool isConnected(const ElementT &element1, const ElementT &element2) {
+    return find(element1) == find(element2);
+  }
+
+ private:
+  struct Node {
+    Node(const std::size_t rank_in, const std::size_t parent_in)
+        : rank(rank_in), parent(parent_in) {
+    }
+    std::size_t rank;
+    std::size_t parent;
+  };
+
+  inline void compress_path(const std::size_t leaf_node_id, const std::size_t root_node_id) {
+    std::size_t node_id = leaf_node_id;
+    std::size_t max_rank = 0;
+    while (node_id != root_node_id) {
+      const Node &node = nodes_[node_id];
+      max_rank = std::max(max_rank, node.rank);
+
+      const std::size_t parent_id = node.parent;
+      nodes_[node_id].parent = root_node_id;
+      node_id = parent_id;
+    }
+    nodes_[root_node_id].rank = max_rank + 1;
+  }
+
+  std::vector<Node> nodes_;
+  MapperT elements_map_;
+
+  static constexpr std::size_t kInvalid = std::numeric_limits<std::size_t>::max();
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/utility/EventProfiler.cpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.cpp b/utility/EventProfiler.cpp
new file mode 100644
index 0000000..728ebff
--- /dev/null
+++ b/utility/EventProfiler.cpp
@@ -0,0 +1,29 @@
+/**
+ *   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 "utility/EventProfiler.hpp"
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+namespace quickstep {
+
+EventProfiler<int, std::size_t> simple_profiler;
+EventProfiler<std::size_t> relop_profiler;
+
+}  // namespace quickstep


[03/10] incubator-quickstep git commit: Minor bug fix in QueryManager base.

Posted by ji...@apache.org.
Minor bug fix in QueryManager base.

- Modified the order in which we check the completion of query and
  completion of an operator in the queryStatus function.


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

Branch: refs/heads/LIP-for-tpch
Commit: 9f9e3b7b1957e774a982f260c93968a9617c5354
Parents: c0bb462
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Jul 25 12:23:34 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Jul 26 16:21:42 2016 -0500

----------------------------------------------------------------------
 query_execution/QueryManagerBase.cpp | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9f9e3b7b/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index 37beb02..d2a3341 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -71,15 +71,16 @@ QueryManagerBase::QueryManagerBase(QueryHandle *query_handle)
 
 QueryManagerBase::QueryStatusCode QueryManagerBase::queryStatus(
     const dag_node_index op_index) {
-  if (query_exec_state_->hasExecutionFinished(op_index)) {
-    return QueryStatusCode::kOperatorExecuted;
-  }
-
-  // As kQueryExecuted takes precedence over kOperatorExecuted, we check again.
+  // As kQueryExecuted takes precedence over kOperatorExecuted, we first check
+  // whether the query has finished its execution.
   if (query_exec_state_->hasQueryExecutionFinished()) {
     return QueryStatusCode::kQueryExecuted;
   }
 
+  if (query_exec_state_->hasExecutionFinished(op_index)) {
+    return QueryStatusCode::kOperatorExecuted;
+  }
+
   return QueryStatusCode::kNone;
 }
 


[05/10] incubator-quickstep git commit: Added README for types module.

Posted by ji...@apache.org.
Added README for types module.


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

Branch: refs/heads/LIP-for-tpch
Commit: 33554c3edcac6becb84bfcdcdb8a60b9dd6a3f0b
Parents: 7415ee8
Author: Craig Chasseur <sp...@gmail.com>
Authored: Wed Jul 27 20:01:53 2016 -0700
Committer: Craig Chasseur <sp...@gmail.com>
Committed: Wed Jul 27 20:01:53 2016 -0700

----------------------------------------------------------------------
 types/README.md | 102 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 102 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/33554c3e/types/README.md
----------------------------------------------------------------------
diff --git a/types/README.md b/types/README.md
new file mode 100644
index 0000000..baf01aa
--- /dev/null
+++ b/types/README.md
@@ -0,0 +1,102 @@
+# The Quickstep Type System
+
+The types module is used across Quickstep and handles details of how date values
+are stored and represented, how they are parsed from and printed to
+human-readable text, and low-level operations on values that form the building
+blocks for more complex [expressions](../expressions).
+
+## The Type Class
+
+Every distinct concrete type in Quickstep is represented by a single object of
+a class derived from the base `quickstep::Type` class. All types have some
+common properties, including the following:
+
+  * A `TypeID` - an enum identifying the type, e.g. `kInt` for 32-bit integers,
+    or `kVarChar` for variable-length strings.
+  * Nullability - whether the type allows NULL values. All types have both a
+    nullable and a non-nullable flavor, except for NullType, a special type that
+    can ONLY store NULLs and has no non-nullable version.
+  * Storage size - minimum and maximum byte length. For fixed-length types like
+    basic numeric types and fixed length `CHAR(X)` strings, these lengths are
+    the same. For variable-length types like `VARCHAR(X)`, they can be
+    different (and the `Type` class has a method `estimateAverageByteLength()`
+    that can be used to make educated guesses when allocating storage). Note
+    that storage requirements really only apply to uncompressed, non-NULL
+    values. The actual bytes needed to store the values in the
+    [storage system](../storage) may be different if
+    [compression](../compression) is used, and some storage formats might store
+    NULLs differently.
+
+Some categories of types have additional properties (e.g. `CharType` and
+`VarCharType` also have a length parameter that indicates the maximum length of
+string that can be stored).
+
+### Getting a Type
+
+Each distinct, concrete Type is represented by a single object in the entire
+Quickstep process. To actually get a reference to usable `Type`, most code will
+go through the `TypeFactory`. `TypeFactory` provides static methods to access
+specific types by `TypeID` and other parameters. It can also deserialize a type
+from its protobuf representation (a `quickstep::serialization::Type` message).
+Finally, it also provides methods that can determine a `Type` that two different
+types can be cast to.
+
+### More on the `Type` Interface
+
+In addition to methods that allow inspection of a type's properties (e.g. those
+listed above), the Type class defines an interface with useful functionality
+common to all types:
+
+  * Serialization (of the type itself) - the `getProto()` method produces a
+    protobuf message that can be serialized and deserialized and later passed to
+    the TypeFactory to get back the same type.
+  * Relationship to other types - `equals()` determines if two types are exactly
+    the same, while `isCoercibleFrom()` determines if it is possible to convert
+    from another type to a given type (e.g. with a `CAST`), and
+    `isSafelyCoercibleFrom()` determines if such a conversion can always be done
+    without loss of precision.
+  * Printing to human-readable format - `printValueToString()` and
+    `printValueToFile()` can print out values of a type (see `TypedValue` below)
+    in human-readable format.
+  * Parsing from human-readable format - Similarly, `parseValueFromString()`
+    produces a `TypedValue` that is parsed from a string in human-readable
+    format.
+  * Making values - `makeValue()` creates a `TypedValue` from a bare pointer to
+    a value's representation in storage. For nullable types, `makeNullValue()`
+    makes a NULL value, and for numeric types, `makeZeroValue()` makes a zero
+    of that type.
+  * Coercing values - `coerceValue()` takes a value of another type and converts
+    it to the given type (e.g. as part of a `CAST`).
+
+## The TypedValue Class
+
+An individual typed value in Quickstep is represented by an instance of the
+`TypedValue` class. TypedValues can be created by methods of the `Type` class,
+by operation and expression classes that operate on values, or simply by calling
+one of several constructors provided in the class itself for convenience.
+TypedValues have C++ value semantics (i.e. they are copyable, assignable, and
+movable). A TypedValue may own its own data, or it may be a lightweight
+reference to data that is stored elsewhere in memory (this can be checked with
+`isReference()`, and any reference can be upgraded to own its own data copy by
+calling `ensureNotReference()`).
+
+Here are some of the things you can do with a TypedValue:
+
+  * NULL checks - calling `isNull()` determines if the TypedValue represents a
+    NULL. Several methods of TypedValue are usable only for non-NULL values, so
+    it is often important to check this first if in doubt.
+  * Access to underlying data - `getDataPtr()` returns an untyped `void*`
+    pointer to the underlying data, and `getDataSize()` returns the size of the
+    underlying data in bytes. Depending on the type of the value, the templated
+    method `getLiteral()` can be used to get the underlying data as a literal
+    scalar, or `getAsciiStringLength()` can be used to get the string length of
+    a `CHAR(X)` or `VARCHAR(X)` without counting null-terminators.
+  * Hashing - `getHash()` returns a hash of the value, which is suitable for
+    use in the HashTables of the [storage system](../storage), or in generic
+    hash-based C++ containers. `fastEqualCheck()` is provided to quickly check
+    whether two TypedValues of the same type (e.g. in the same hash table) are
+    actually equal.
+  * Serialization/Deserialization - `getProto()` serializes a TypedValue to a
+    `serialization::TypedValue` protobuf. The static method `ProtoIsValid()`
+    checks whether a serialized TypedValue is valid, and
+    `ReconstructFromProto()` rebuilds a TypedValue from its serialized form.


[06/10] incubator-quickstep git commit: Renamed a QueryExecutionMessage.

Posted by ji...@apache.org.
Renamed a QueryExecutionMessage.


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

Branch: refs/heads/LIP-for-tpch
Commit: aaecc76b7bea85f46bf06dc2e63fccf43636d7eb
Parents: 33554c3
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Thu Jul 28 11:11:33 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Thu Jul 28 11:11:33 2016 -0700

----------------------------------------------------------------------
 query_execution/QueryExecutionMessages.proto |  4 +-
 query_execution/QueryExecutionTypedefs.hpp   |  4 +-
 query_execution/Shiftboss.cpp                | 45 +++++++++++------------
 query_execution/Shiftboss.hpp                |  4 +-
 4 files changed, 28 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aaecc76b/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 591ca6c..308d736 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -113,12 +113,12 @@ message InitiateRebuildResponseMessage {
   required uint64 num_rebuild_work_orders = 3;
 }
 
-message QueryResultRelationMessage {
+message SaveQueryResultMessage {
   required int32 relation_id = 1;
   repeated fixed64 blocks = 2 [packed=true];
 }
 
-message QueryResultRelationResponseMessage {
+message SaveQueryResultResponseMessage {
   required int32 relation_id = 1;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aaecc76b/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index d73d4ee..b67209f 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -81,8 +81,8 @@ enum QueryExecutionMessageType : message_type_id {
   kInitiateRebuildMessage,  // From Foreman to Shiftboss.
   kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
 
-  kQueryResultRelationMessage,  // From Foreman to Shiftboss.
-  kQueryResultRelationResponseMessage,  // From Shiftboss to Foreman.
+  kSaveQueryResultMessage,  // From Foreman to Shiftboss.
+  kSaveQueryResultResponseMessage,  // From Shiftboss to Foreman.
 
   // BlockLocator related messages, sorted in a life cycle of StorageManager
   // with a unique block domain.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aaecc76b/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index af56306..7f655c6 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -149,11 +149,10 @@ void Shiftboss::run() {
                                            move(annotated_message.tagged_message));
         break;
       }
-      case kQueryResultRelationMessage: {
-        // TODO(zuyu): Rename to kSaveQueryResultMessage.
+      case kSaveQueryResultMessage: {
         const TaggedMessage &tagged_message = annotated_message.tagged_message;
 
-        serialization::QueryResultRelationMessage proto;
+        serialization::SaveQueryResultMessage proto;
         CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 
         for (int i = 0; i < proto.blocks_size(); ++i) {
@@ -168,25 +167,25 @@ void Shiftboss::run() {
           }
         }
 
-        serialization::QueryResultRelationResponseMessage ack_proto;
-        ack_proto.set_relation_id(proto.relation_id());
+        serialization::SaveQueryResultResponseMessage proto_response;
+        proto_response.set_relation_id(proto.relation_id());
 
-        const size_t ack_proto_length = ack_proto.ByteSize();
-        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
-        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
+        const size_t proto_response_length = proto_response.ByteSize();
+        char *proto_response_bytes = static_cast<char*>(malloc(proto_response_length));
+        CHECK(proto_response.SerializeToArray(proto_response_bytes, proto_response_length));
 
-        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
-                                  ack_proto_length,
-                                  kQueryResultRelationResponseMessage);
-        free(ack_proto_bytes);
+        TaggedMessage message_response(static_cast<const void*>(proto_response_bytes),
+                                       proto_response_length,
+                                       kSaveQueryResultResponseMessage);
+        free(proto_response_bytes);
 
         LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
-                  << ") to Foreman";
+                  << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
+                  << "') to Foreman";
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            shiftboss_client_id_,
                                            foreman_client_id_,
-                                           move(ack_message));
+                                           move(message_response));
         break;
       }
       case kPoisonMessage: {
@@ -280,15 +279,15 @@ void Shiftboss::processQueryInitiateMessage(
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
   CHECK(proto.SerializeToArray(proto_bytes, proto_length));
 
-  TaggedMessage ack_message(static_cast<const void*>(proto_bytes),
-                            proto_length,
-                            kQueryInitiateResponseMessage);
+  TaggedMessage message_response(static_cast<const void*>(proto_bytes),
+                                 proto_length,
+                                 kQueryInitiateResponseMessage);
   free(proto_bytes);
 
   QueryExecutionUtil::SendTMBMessage(bus_,
                                      shiftboss_client_id_,
                                      foreman_client_id_,
-                                     move(ack_message));
+                                     move(message_response));
 }
 
 void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -317,15 +316,15 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
   CHECK(proto.SerializeToArray(proto_bytes, proto_length));
 
-  TaggedMessage ack_message(static_cast<const void*>(proto_bytes),
-                            proto_length,
-                            kInitiateRebuildResponseMessage);
+  TaggedMessage message_response(static_cast<const void*>(proto_bytes),
+                                 proto_length,
+                                 kInitiateRebuildResponseMessage);
   free(proto_bytes);
 
   QueryExecutionUtil::SendTMBMessage(bus_,
                                      shiftboss_client_id_,
                                      foreman_client_id_,
-                                     move(ack_message));
+                                     move(message_response));
 
   for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
     // NOTE(zuyu): Worker releases the memory after the execution of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aaecc76b/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 096ab74..9e24d62 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -112,8 +112,8 @@ class Shiftboss : public Thread {
     bus_->RegisterClientAsReceiver(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
     bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
 
-    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kQueryResultRelationMessage);
-    bus_->RegisterClientAsSender(shiftboss_client_id_, kQueryResultRelationResponseMessage);
+    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kSaveQueryResultMessage);
+    bus_->RegisterClientAsSender(shiftboss_client_id_, kSaveQueryResultResponseMessage);
 
     // Stop itself.
     bus_->RegisterClientAsReceiver(shiftboss_client_id_, kPoisonMessage);


[02/10] incubator-quickstep git commit: - Supported ROWS mode for AVG window aggregation. - Created WindowAggregateFunctions in expressions/window_aggregation. - Created WindowAggregationHandle for AVG to actually do the calculation. - Other functions wi

Posted by ji...@apache.org.
- Supported ROWS mode for AVG window aggregation.
- Created WindowAggregateFunctions in expressions/window_aggregation.
- Created WindowAggregationHandle for AVG to actually do the calculation.
- Other functions will be supported in future PRs.
- RANGE mode is not supported yet.


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

Branch: refs/heads/LIP-for-tpch
Commit: c0bb4620ca7d3e2ba6334efc009376164d2cd6f8
Parents: 2b78380
Author: shixuan-fan <sh...@apache.org>
Authored: Fri Jul 8 18:23:47 2016 +0000
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Jul 25 13:21:40 2016 -0700

----------------------------------------------------------------------
 expressions/CMakeLists.txt                      |   1 +
 expressions/window_aggregation/CMakeLists.txt   | 206 ++++++++++
 .../WindowAggregateFunction.cpp                 |  58 +++
 .../WindowAggregateFunction.hpp                 | 149 +++++++
 .../WindowAggregateFunction.proto               |  32 ++
 .../WindowAggregateFunctionAvg.cpp              |  85 ++++
 .../WindowAggregateFunctionAvg.hpp              |  75 ++++
 .../WindowAggregateFunctionCount.cpp            |  59 +++
 .../WindowAggregateFunctionCount.hpp            |  75 ++++
 .../WindowAggregateFunctionFactory.cpp          | 106 +++++
 .../WindowAggregateFunctionFactory.hpp          |  96 +++++
 .../WindowAggregateFunctionMax.cpp              |  67 ++++
 .../WindowAggregateFunctionMax.hpp              |  75 ++++
 .../WindowAggregateFunctionMin.cpp              |  66 ++++
 .../WindowAggregateFunctionMin.hpp              |  75 ++++
 .../WindowAggregateFunctionSum.cpp              |  82 ++++
 .../WindowAggregateFunctionSum.hpp              |  75 ++++
 .../WindowAggregationHandle.hpp                 | 137 +++++++
 .../WindowAggregationHandleAvg.cpp              | 241 ++++++++++++
 .../WindowAggregationHandleAvg.hpp              | 101 +++++
 .../window_aggregation/WindowAggregationID.hpp  |  44 +++
 .../WindowAggregationHandleAvg_unittest.cpp     | 387 +++++++++++++++++++
 query_optimizer/CMakeLists.txt                  |   2 +
 query_optimizer/ExecutionGenerator.cpp          |   5 +-
 query_optimizer/expressions/CMakeLists.txt      |   2 +-
 .../expressions/WindowAggregateFunction.cpp     |   4 +-
 .../expressions/WindowAggregateFunction.hpp     |  10 +-
 query_optimizer/resolver/CMakeLists.txt         |   2 +
 query_optimizer/resolver/Resolver.cpp           |  40 +-
 query_optimizer/resolver/Resolver.hpp           |   5 +-
 .../tests/execution_generator/Select.test       |  78 +++-
 relational_operators/CMakeLists.txt             |   1 +
 .../WindowAggregationOperator.cpp               |  15 +-
 .../WindowAggregationOperator.hpp               |  30 +-
 relational_operators/WorkOrder.proto            |   3 +-
 storage/CMakeLists.txt                          |  17 +-
 storage/WindowAggregationOperationState.cpp     | 183 +++++++--
 storage/WindowAggregationOperationState.hpp     |  68 +---
 storage/WindowAggregationOperationState.proto   |   6 +-
 ...WindowAggregationOperationState_unittest.cpp |  14 +-
 40 files changed, 2613 insertions(+), 164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/CMakeLists.txt b/expressions/CMakeLists.txt
index 53ad5d4..6ef3c24 100644
--- a/expressions/CMakeLists.txt
+++ b/expressions/CMakeLists.txt
@@ -17,6 +17,7 @@ add_subdirectory(aggregation)
 add_subdirectory(predicate)
 add_subdirectory(scalar)
 add_subdirectory(table_generator)
+add_subdirectory(window_aggregation)
 
 QS_PROTOBUF_GENERATE_CPP(expressions_Expressions_proto_srcs
                          expressions_Expressions_proto_hdrs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/CMakeLists.txt b/expressions/window_aggregation/CMakeLists.txt
new file mode 100644
index 0000000..6a16fcc
--- /dev/null
+++ b/expressions/window_aggregation/CMakeLists.txt
@@ -0,0 +1,206 @@
+#   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.
+
+QS_PROTOBUF_GENERATE_CPP(expressions_windowaggregation_WindowAggregateFunction_proto_srcs
+                         expressions_windowaggregation_WindowAggregateFunction_proto_hdrs
+                         WindowAggregateFunction.proto)
+
+# Declare micro-libs:
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunction
+            WindowAggregateFunction.cpp
+            WindowAggregateFunction.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
+            ${expressions_windowaggregation_WindowAggregateFunction_proto_srcs})
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunctionAvg
+            WindowAggregateFunctionAvg.cpp
+            WindowAggregateFunctionAvg.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunctionCount
+            WindowAggregateFunctionCount.cpp
+            WindowAggregateFunctionCount.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunctionFactory
+            WindowAggregateFunctionFactory.cpp
+            WindowAggregateFunctionFactory.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunctionMax
+            WindowAggregateFunctionMax.cpp
+            WindowAggregateFunctionMax.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunctionMin
+            WindowAggregateFunctionMin.cpp
+            WindowAggregateFunctionMin.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregateFunctionSum
+            WindowAggregateFunctionSum.cpp
+            WindowAggregateFunctionSum.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregationHandle
+            ../../empty_src.cpp
+            WindowAggregationHandle.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregationHandleAvg
+            WindowAggregationHandleAvg.cpp
+            WindowAggregationHandleAvg.hpp)
+add_library(quickstep_expressions_windowaggregation_WindowAggregationID
+            ../../empty_src.cpp
+            WindowAggregationID.hpp)
+
+# Link dependencies:
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
+                      ${PROTOBUF_LIBRARY})
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionAvg
+                      glog
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregationHandleAvg
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
+                      quickstep_types_operations_binaryoperations_BinaryOperationID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionCount
+                      glog
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionFactory
+                      glog
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionAvg
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionCount
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionMax
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionMin
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionSum
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionMax
+                      glog
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_types_Type
+                      quickstep_types_operations_comparisons_Comparison
+                      quickstep_types_operations_comparisons_ComparisonFactory
+                      quickstep_types_operations_comparisons_ComparisonID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionMin
+                      glog
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_types_Type
+                      quickstep_types_operations_comparisons_Comparison
+                      quickstep_types_operations_comparisons_ComparisonFactory
+                      quickstep_types_operations_comparisons_ComparisonID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionSum
+                      glog
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
+                      quickstep_types_operations_binaryoperations_BinaryOperationID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      glog
+                      quickstep_catalog_CatalogRelationSchema
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_types_operations_comparisons_Comparison
+                      quickstep_types_operations_comparisons_ComparisonFactory
+                      quickstep_types_operations_comparisons_ComparisonID
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregationHandleAvg
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_scalar_Scalar
+                      quickstep_expressions_scalar_ScalarAttribute
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_storage_ValueAccessor
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
+                      quickstep_types_operations_binaryoperations_BinaryOperationID
+                      quickstep_types_operations_comparisons_Comparison
+                      quickstep_utility_Macros)
+
+# Submodule all-in-one library:
+add_library(quickstep_expressions_windowaggregation ../../empty_src.cpp)
+target_link_libraries(quickstep_expressions_windowaggregation
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction_proto
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionAvg
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionCount
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionFactory
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionMax
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionMin
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionSum
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationHandleAvg
+                      quickstep_expressions_windowaggregation_WindowAggregationID)
+
+# Tests:
+
+# Unified executable to ammortize cost of linking.
+add_executable(WindowAggregationHandle_tests
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/WindowAggregationHandleAvg_unittest.cpp")
+target_link_libraries(WindowAggregationHandle_tests
+                      gtest
+                      gtest_main
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_windowaggregation_WindowAggregateFunction
+                      quickstep_expressions_windowaggregation_WindowAggregateFunctionFactory
+                      quickstep_expressions_windowaggregation_WindowAggregationHandle
+                      quickstep_expressions_windowaggregation_WindowAggregationHandleAvg
+                      quickstep_expressions_windowaggregation_WindowAggregationID
+                      quickstep_storage_ValueAccessor
+                      quickstep_types_CharType
+                      quickstep_types_DateOperatorOverloads
+                      quickstep_types_DatetimeIntervalType
+                      quickstep_types_DatetimeType
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
+                      quickstep_types_IntType
+                      quickstep_types_IntervalLit
+                      quickstep_types_LongType
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_VarCharType
+                      quickstep_types_YearMonthIntervalType
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor)
+add_test(WindowAggregationHandle_tests WindowAggregationHandle_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunction.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunction.cpp b/expressions/window_aggregation/WindowAggregateFunction.cpp
new file mode 100644
index 0000000..3911e1c
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunction.cpp
@@ -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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+
+#include <type_traits>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.pb.h"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+serialization::WindowAggregateFunction WindowAggregateFunction::getProto() const {
+  serialization::WindowAggregateFunction proto;
+  switch (win_agg_id_) {
+    case WindowAggregationID::kAvg:
+      proto.set_window_aggregation_id(serialization::WindowAggregateFunction::AVG);
+      break;
+    case WindowAggregationID::kCount:
+      proto.set_window_aggregation_id(serialization::WindowAggregateFunction::COUNT);
+      break;
+    case WindowAggregationID::kMax:
+      proto.set_window_aggregation_id(serialization::WindowAggregateFunction::MAX);
+      break;
+    case WindowAggregationID::kMin:
+      proto.set_window_aggregation_id(serialization::WindowAggregateFunction::MIN);
+      break;
+    case WindowAggregationID::kSum:
+      proto.set_window_aggregation_id(serialization::WindowAggregateFunction::SUM);
+      break;
+    default: {
+      LOG(FATAL) << "Unrecognized WindowAggregationID: "
+                 << static_cast<std::underlying_type<WindowAggregationID>::type>(win_agg_id_);
+    }
+  }
+
+  return proto;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunction.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunction.hpp b/expressions/window_aggregation/WindowAggregateFunction.hpp
new file mode 100644
index 0000000..e40479b
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunction.hpp
@@ -0,0 +1,149 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_HPP_
+
+#include <string>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunction.pb.h"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class CatalogRelationSchema;
+class Type;
+class WindowAggregationHandle;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief A class representing a particular window aggregate function in the
+ *        abstract sense. Each named aggregate function is represented by a
+ *        singleton subclass of WindowAggregateFunction.
+ *
+ * WindowAggregateFunction provides informational methods about the
+ * applicability of a particular window aggregate function to particular
+ * argument Type(s). The actual implementation of the window aggregate
+ * functions' logic is in the WindowAggregationHandle class hierarchy, and can
+ * be different depending on the particular argument Type(s) given to the window
+ * aggregate. To perform a window aggregation, a caller should first call
+ * WindowAggregateFunction::createHandle() to instantiate an
+ * WindowAggregationHandle object, then use the methods of
+ * WindowAggregationHandle to do the actual window aggregation. Finally, delete
+ * the WindowAggregationHandle after finished.
+ * 
+ * See WindowAggregationHandle for more detailed information about how
+ * window aggregates are actually computed.
+ **/
+class WindowAggregateFunction {
+ public:
+  /**
+   * @brief Get the ID of this window aggregate (i.e. its unique ID amongst all
+   *        the WindowAggregateFunctions).
+   *
+   * @return The WindowAggregationID of this WindowAggregateFunction.
+   **/
+  inline WindowAggregationID getWindowAggregationID() const {
+    return win_agg_id_;
+  }
+
+  /**
+   * @brief Get the human-readable name of this WindowAggregateFunction.
+   *
+   * @return The human-readable name of this WindowAggregateFunction.
+   **/
+  virtual std::string getName() const = 0;
+
+  /**
+   * @brief Get the serialized protocol buffer representation of this
+   *        WindowAggregateFunction.
+   *
+   * @return A serialized protocol buffer representation of this
+   *         WindowAggregateFunction.
+   **/
+  virtual serialization::WindowAggregateFunction getProto() const;
+
+  /**
+   * @brief Determine if this WindowAggregateFunction can be applied to
+   *        arguments of particular Type(s).
+   *
+   * @param argument_types A list of zero or more Types (in order) for
+   *        arguments to this WindowAggregateFunction.
+   * @return Whether this WindowAggregateFunction is applicable to the given
+   *         argument_types.
+   **/
+  virtual bool canApplyToTypes(
+      const std::vector<const Type*> &argument_types) const = 0;
+
+  /**
+   * @brief Determine the result Type for this WindowAggregateFunction given
+   *        arguments of particular Type(s).
+   *
+   * @param argument_types A list of zero or more Types (in order) for
+   *        arguments to this WindowAggregateFunction.
+   * @return The result Type for this WindowAggregateFunction applied to the
+   *         specified argument_types, or nullptr if this
+   *         WindowAggregateFunction is not applicable to the specified Type(s).
+   **/
+  virtual const Type* resultTypeForArgumentTypes(
+      const std::vector<const Type*> &argument_types) const = 0;
+
+  /**
+   * @brief Create a WindowAggregationHandle to compute aggregates.
+   *
+   * @warning It is an error to call this method for argument_types which this
+   *          WindowAggregateFunction can not apply to. For safety, check
+   *          canApplyToTypes() first.
+   *
+   * @param argument_types A list of zero or more Types (in order) for
+   *        arguments to this WindowAggregateFunction.
+   * @param partition_key_types A list or zero or more Types for partition keys
+   *                            to this WindowAggregateFunction.
+   * 
+   * @return A new WindowAggregationHandle that can be used to compute this
+   *         WindowAggregateFunction over the specified argument_types. Caller
+   *         is responsible for deleting the returned object.
+   **/
+  virtual WindowAggregationHandle* createHandle(
+      const std::vector<const Type*> &argument_types,
+      const std::vector<const Type*> &partition_key_types) const = 0;
+
+ protected:
+  explicit WindowAggregateFunction(const WindowAggregationID win_agg_id)
+      : win_agg_id_(win_agg_id) {
+  }
+
+ private:
+  const WindowAggregationID win_agg_id_;
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunction);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunction.proto
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunction.proto b/expressions/window_aggregation/WindowAggregateFunction.proto
new file mode 100644
index 0000000..fe8d799
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunction.proto
@@ -0,0 +1,32 @@
+//   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.
+
+syntax = "proto2";
+
+package quickstep.serialization;
+
+message WindowAggregateFunction {
+  enum WindowAggregationID {
+    AVG = 0;
+    COUNT = 1;
+    MAX = 2;
+    MIN = 3;
+    SUM = 4;    
+  }
+
+  required WindowAggregationID window_aggregation_id = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp b/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
new file mode 100644
index 0000000..bc31a53
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
@@ -0,0 +1,85 @@
+/**
+ *   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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunctionAvg.hpp"
+
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregationHandleAvg.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
+#include "types/operations/binary_operations/BinaryOperationID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+bool WindowAggregateFunctionAvg::canApplyToTypes(
+    const std::vector<const Type*> &argument_types) const {
+  // AVG is unary.
+  if (argument_types.size() != 1) {
+    return false;
+  }
+
+  // Argument must be addable and divisible.
+  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+             .canApplyToTypes(*argument_types.front(), *argument_types.front()) &&
+         BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+             .canApplyToTypes(*argument_types.front(), TypeFactory::GetType(kDouble));
+}
+
+const Type* WindowAggregateFunctionAvg::resultTypeForArgumentTypes(
+    const std::vector<const Type*> &argument_types) const {
+  if (!canApplyToTypes(argument_types)) {
+    return nullptr;
+  }
+
+  // The type used to sum values is nullable, and we automatically widen int to
+  // long and float to double to have more headroom when adding up many values.
+  const Type *sum_type = &(argument_types.front()->getNullableVersion());
+  switch (sum_type->getTypeID()) {
+    case kInt:
+      sum_type = &TypeFactory::GetType(kLong, true);
+      break;
+    case kFloat:
+      sum_type = &TypeFactory::GetType(kDouble, true);
+      break;
+    default:
+      break;
+  }
+
+  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+             .resultTypeForArgumentTypes(*sum_type, TypeFactory::GetType(kDouble));
+}
+
+WindowAggregationHandle* WindowAggregateFunctionAvg::createHandle(
+    const std::vector<const Type*> &argument_types,
+    const std::vector<const Type*> &partition_key_types) const {
+  DCHECK(canApplyToTypes(argument_types))
+      << "Attempted to create an WindowAggregationHandleAvg for argument Type(s)"
+      << " that AVG can not be applied to.";
+
+  return new WindowAggregationHandleAvg(partition_key_types,
+                                        *argument_types.front());
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp b/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
new file mode 100644
index 0000000..32fd9d5
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionAvg.hpp
@@ -0,0 +1,75 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_AVG_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_AVG_HPP_
+
+#include <string>
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class Type;
+class WindowAggregationHandle;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief WindowAggregateFunction representing SQL AVG() OVER term.
+ **/
+class WindowAggregateFunctionAvg : public WindowAggregateFunction {
+ public:
+  static const WindowAggregateFunctionAvg& Instance() {
+    static WindowAggregateFunctionAvg instance;
+    return instance;
+  }
+
+  std::string getName() const override {
+    return "AVG";
+  }
+
+  bool canApplyToTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  const Type* resultTypeForArgumentTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  WindowAggregationHandle* createHandle(
+      const std::vector<const Type*> &argument_types,
+      const std::vector<const Type*> &partition_key_types) const override;
+
+ private:
+  WindowAggregateFunctionAvg()
+      : WindowAggregateFunction(WindowAggregationID::kAvg) {
+  }
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunctionAvg);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_AVG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionCount.cpp b/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
new file mode 100644
index 0000000..504e000
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionCount.cpp
@@ -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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunctionCount.hpp"
+
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+bool WindowAggregateFunctionCount::canApplyToTypes(
+    const std::vector<const Type*> &argument_types) const {
+  // COUNT may be nullary (i.e. COUNT(*)) or unary.
+  return argument_types.size() <= 1;
+}
+
+const Type* WindowAggregateFunctionCount::resultTypeForArgumentTypes(
+    const std::vector<const Type*> &argument_types) const {
+  if (!canApplyToTypes(argument_types)) {
+    return nullptr;
+  }
+
+  return &TypeFactory::GetType(kLong);
+}
+
+WindowAggregationHandle* WindowAggregateFunctionCount::createHandle(
+    const std::vector<const Type*> &argument_types,
+    const std::vector<const Type*> &partition_key_types) const {
+  DCHECK(canApplyToTypes(argument_types))
+      << "Attempted to create a WindowAggregationHandleCount for argument Types "
+      << "that COUNT can not be applied to (> 1 argument).";
+
+  // TODO(Shixuan): Add handle for Count.
+  return nullptr;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionCount.hpp b/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
new file mode 100644
index 0000000..1b40fdd
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionCount.hpp
@@ -0,0 +1,75 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_COUNT_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_COUNT_HPP_
+
+#include <string>
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class Type;
+class WindowAggregationHandle;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief WindowAggregateFunction representing SQL COUNT() OVER term.
+ **/
+class WindowAggregateFunctionCount : public WindowAggregateFunction {
+ public:
+  static const WindowAggregateFunctionCount& Instance() {
+    static WindowAggregateFunctionCount instance;
+    return instance;
+  }
+
+  std::string getName() const override {
+    return "COUNT";
+  }
+
+  bool canApplyToTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  const Type* resultTypeForArgumentTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  WindowAggregationHandle* createHandle(
+      const std::vector<const Type*> &argument_types,
+      const std::vector<const Type*> &partition_key_types) const override;
+
+ private:
+  WindowAggregateFunctionCount()
+      : WindowAggregateFunction(WindowAggregationID::kCount) {
+  }
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunctionCount);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_COUNT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp b/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
new file mode 100644
index 0000000..65247f2
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionFactory.cpp
@@ -0,0 +1,106 @@
+/**
+ *   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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunctionFactory.hpp"
+
+#include <string>
+#include <type_traits>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.pb.h"
+#include "expressions/window_aggregation/WindowAggregateFunctionAvg.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionCount.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionMax.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionMin.hpp"
+#include "expressions/window_aggregation/WindowAggregateFunctionSum.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+const WindowAggregateFunction& WindowAggregateFunctionFactory::Get(
+    const WindowAggregationID agg_id) {
+  switch (agg_id) {
+    case WindowAggregationID::kAvg:
+      return WindowAggregateFunctionAvg::Instance();
+    case WindowAggregationID::kCount:
+      return WindowAggregateFunctionCount::Instance();
+    case WindowAggregationID::kMax:
+      return WindowAggregateFunctionMax::Instance();
+    case WindowAggregationID::kMin:
+      return WindowAggregateFunctionMin::Instance();
+    case WindowAggregationID::kSum:
+      return WindowAggregateFunctionSum::Instance();
+    default: {
+      LOG(FATAL) << "Unrecognized WindowAggregationID: "
+                 << static_cast<std::underlying_type<WindowAggregationID>::type>(agg_id);
+    }
+  }
+}
+
+const WindowAggregateFunction* WindowAggregateFunctionFactory::GetByName(
+    const std::string &name) {
+  if (name == "avg") {
+    return &WindowAggregateFunctionAvg::Instance();
+  } else if (name == "count") {
+    return &WindowAggregateFunctionCount::Instance();
+  } else if (name == "max") {
+    return &WindowAggregateFunctionMax::Instance();
+  } else if (name == "min") {
+    return &WindowAggregateFunctionMin::Instance();
+  } else if (name == "sum") {
+    return &WindowAggregateFunctionSum::Instance();
+  } else {
+    return nullptr;
+  }
+}
+
+bool WindowAggregateFunctionFactory::ProtoIsValid(
+    const serialization::WindowAggregateFunction &proto) {
+  return proto.IsInitialized() &&
+         serialization::WindowAggregateFunction::WindowAggregationID_IsValid(proto.window_aggregation_id());
+}
+
+const WindowAggregateFunction& WindowAggregateFunctionFactory::ReconstructFromProto(
+    const serialization::WindowAggregateFunction &proto) {
+  DCHECK(ProtoIsValid(proto))
+      << "Attempted to reconstruct an WindowAggregateFunction from an invalid proto:\n"
+      << proto.DebugString();
+
+  switch (proto.window_aggregation_id()) {
+    case serialization::WindowAggregateFunction::AVG:
+      return WindowAggregateFunctionAvg::Instance();
+    case serialization::WindowAggregateFunction::COUNT:
+      return WindowAggregateFunctionCount::Instance();
+    case serialization::WindowAggregateFunction::MAX:
+      return WindowAggregateFunctionMax::Instance();
+    case serialization::WindowAggregateFunction::MIN:
+      return WindowAggregateFunctionMin::Instance();
+    case serialization::WindowAggregateFunction::SUM:
+      return WindowAggregateFunctionSum::Instance();
+    default: {
+      LOG(FATAL) << "Unrecognized serialization::WindowAggregateFunction::WindowAggregationID: "
+                 << proto.window_aggregation_id()
+                 << "\nFull proto debug string:\n"
+                 << proto.DebugString();
+    }
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp b/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
new file mode 100644
index 0000000..1d59e93
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionFactory.hpp
@@ -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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_FACTORY_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_FACTORY_HPP_
+
+#include <string>
+
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class WindowAggregateFunction;
+namespace serialization { class WindowAggregateFunction; }
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief All-static factory with methods that provide access to the various
+ *        concrete implementations of WindowAggregateFunction.
+ *
+ * WindowAggregateFunctionFactory allows client code to use any
+ * WindowAggregateFunction in Quickstep in a generic way without having to know
+ * about all the specific subclasses of WindowAggregateFunction. In particular,
+ * it is used to deserialize WindowAggregateFunctions used in
+ * WindowAggregationOperationState from their protobuf representations
+ * (originally created by the optimizer) when deserializing a QueryContext.
+ **/
+namespace WindowAggregateFunctionFactory {
+  /**
+   * @brief Get a particular WindowAggregateFunction by its ID.
+   *
+   * @param agg_id The ID of the desired WindowAggregateFunction.
+   * @return A reference to the singleton instance of the
+   *         WindowAggregateFunction specified by agg_id.
+   **/
+  const WindowAggregateFunction& Get(const WindowAggregationID agg_id);
+
+  /**
+   * @brief Get a particular WindowAggregateFunction by its name in SQL syntax.
+   *
+   * @param name The name of the desired WindowAggregateFunction in lower case.
+   * @return A pointer to the WindowAggregateFunction specified by name, or NULL
+   *         if name does not match any known WindowAggregateFunction.
+   **/
+  const WindowAggregateFunction* GetByName(const std::string &name);
+
+  /**
+   * @brief Determine if a serialized protobuf representation of a
+   *        WindowAggregateFunction is fully-formed and valid.
+   *
+   * @param proto A serialized protobuf representation of a
+   *              WindowAggregateFunction to check for validity.
+   * @return Whether proto is fully-formed and valid.
+   **/
+  bool ProtoIsValid(const serialization::WindowAggregateFunction &proto);
+
+  /**
+   * @brief Get the WindowAggregateFunction represented by a proto.
+   *
+   * @warning It is an error to call this method with an invalid proto.
+   *          ProtoIsValid() should be called first to check.
+   *
+   * @param proto A serialized protobuf representation of a
+   *              WindowAggregateFunction.
+   * @return The WindowAggregateFunction represented by proto.
+   **/
+  const WindowAggregateFunction& ReconstructFromProto(
+      const serialization::WindowAggregateFunction &proto);
+
+}  // namespace WindowAggregateFunctionFactory
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMax.cpp b/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
new file mode 100644
index 0000000..f3997c7
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionMax.cpp
@@ -0,0 +1,67 @@
+/**
+ *   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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunctionMax.hpp"
+
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "types/Type.hpp"
+#include "types/operations/comparisons/Comparison.hpp"
+#include "types/operations/comparisons/ComparisonFactory.hpp"
+#include "types/operations/comparisons/ComparisonID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+bool WindowAggregateFunctionMax::canApplyToTypes(
+    const std::vector<const Type*> &argument_types) const {
+  // MAX is unary.
+  if (argument_types.size() != 1) {
+    return false;
+  }
+
+  // Argument must be comparable by '>'.
+  return ComparisonFactory::GetComparison(ComparisonID::kGreater).canCompareTypes(
+      *argument_types.front(),
+      *argument_types.front());
+}
+
+const Type* WindowAggregateFunctionMax::resultTypeForArgumentTypes(
+    const std::vector<const Type*> &argument_types) const {
+  if (!canApplyToTypes(argument_types)) {
+    return nullptr;
+  }
+
+  return &(argument_types.front()->getNullableVersion());
+}
+
+WindowAggregationHandle* WindowAggregateFunctionMax::createHandle(
+    const std::vector<const Type*> &argument_types,
+    const std::vector<const Type*> &partition_key_types) const {
+  DCHECK(canApplyToTypes(argument_types))
+      << "Attempted to create a WindowAggregationHandleMax for argument Type(s) "
+      << "that MAX can not be applied to.";
+
+  // TODO(Shixuan): Add handle for Max.
+  return nullptr;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMax.hpp b/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
new file mode 100644
index 0000000..00c788e
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionMax.hpp
@@ -0,0 +1,75 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MAX_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MAX_HPP_
+
+#include <string>
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class Type;
+class WindowAggregationHandle;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief WindowAggregateFunction representing SQL MAX() OVER term.
+ **/
+class WindowAggregateFunctionMax : public WindowAggregateFunction {
+ public:
+  static const WindowAggregateFunctionMax& Instance() {
+    static WindowAggregateFunctionMax instance;
+    return instance;
+  }
+
+  std::string getName() const override {
+    return "MAX";
+  }
+
+  bool canApplyToTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  const Type* resultTypeForArgumentTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  WindowAggregationHandle* createHandle(
+      const std::vector<const Type*> &argument_types,
+      const std::vector<const Type*> &partition_key_types) const override;
+
+ private:
+  WindowAggregateFunctionMax()
+      : WindowAggregateFunction(WindowAggregationID::kMax) {
+  }
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunctionMax);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MAX_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMin.cpp b/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
new file mode 100644
index 0000000..a13e28e
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionMin.cpp
@@ -0,0 +1,66 @@
+/**
+ *   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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunctionMin.hpp"
+
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "types/Type.hpp"
+#include "types/operations/comparisons/Comparison.hpp"
+#include "types/operations/comparisons/ComparisonFactory.hpp"
+#include "types/operations/comparisons/ComparisonID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+bool WindowAggregateFunctionMin::canApplyToTypes(
+    const std::vector<const Type*> &argument_types) const {
+  // MIN is unary.
+  if (argument_types.size() != 1) {
+    return false;
+  }
+
+  // Argument must be comparable by '<'.
+  return ComparisonFactory::GetComparison(ComparisonID::kLess).canCompareTypes(
+      *argument_types.front(),
+      *argument_types.front());
+}
+
+const Type* WindowAggregateFunctionMin::resultTypeForArgumentTypes(
+    const std::vector<const Type*> &argument_types) const {
+  if (!canApplyToTypes(argument_types)) {
+    return nullptr;
+  }
+
+  return &(argument_types.front()->getNullableVersion());
+}
+
+WindowAggregationHandle* WindowAggregateFunctionMin::createHandle(
+    const std::vector<const Type*> &argument_types,
+    const std::vector<const Type*> &partition_key_types) const {
+  DCHECK(canApplyToTypes(argument_types))
+      << "Attempted to create a WindowAggregationHandleMin for argument Type(s) "
+      << "that MIN can not be applied to.";
+
+  return nullptr;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionMin.hpp b/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
new file mode 100644
index 0000000..aeba539
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionMin.hpp
@@ -0,0 +1,75 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MIN_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MIN_HPP_
+
+#include <string>
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class Type;
+class WindowAggregationHandle;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief WindowAggregateFunction representing SQL MIN() OVER term.
+ **/
+class WindowAggregateFunctionMin : public WindowAggregateFunction {
+ public:
+  static const WindowAggregateFunctionMin& Instance() {
+    static WindowAggregateFunctionMin instance;
+    return instance;
+  }
+
+  std::string getName() const override {
+    return "MIN";
+  }
+
+  bool canApplyToTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  const Type* resultTypeForArgumentTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  WindowAggregationHandle* createHandle(
+      const std::vector<const Type*> &argument_types,
+      const std::vector<const Type*> &partition_key_types) const override;
+
+ private:
+  WindowAggregateFunctionMin()
+      : WindowAggregateFunction(WindowAggregationID::kMin) {
+  }
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunctionMin);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_MIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionSum.cpp b/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
new file mode 100644
index 0000000..636c53a
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
@@ -0,0 +1,82 @@
+/**
+ *   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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregateFunctionSum.hpp"
+
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
+#include "types/operations/binary_operations/BinaryOperationID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+bool WindowAggregateFunctionSum::canApplyToTypes(
+    const std::vector<const Type*> &argument_types) const {
+  // SUM is unary.
+  if (argument_types.size() != 1) {
+    return false;
+  }
+
+  // Argument must be addable.
+  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+             .canApplyToTypes(*argument_types.front(), *argument_types.front());
+}
+
+const Type* WindowAggregateFunctionSum::resultTypeForArgumentTypes(
+    const std::vector<const Type*> &argument_types) const {
+  if (!canApplyToTypes(argument_types)) {
+    return nullptr;
+  }
+
+  // SUM may return NULL if there are no input rows, and we automatically widen
+  // int to long and float to double to have more headroom when adding up many
+  // values.
+  const Type *sum_type = &(argument_types.front()->getNullableVersion());
+  switch (sum_type->getTypeID()) {
+    case kInt:
+      sum_type = &TypeFactory::GetType(kLong, true);
+      break;
+    case kFloat:
+      sum_type = &TypeFactory::GetType(kDouble, true);
+      break;
+    default:
+      break;
+  }
+
+  return sum_type;
+}
+
+WindowAggregationHandle* WindowAggregateFunctionSum::createHandle(
+    const std::vector<const Type*> &argument_types,
+    const std::vector<const Type*> &partition_key_types) const {
+  DCHECK(canApplyToTypes(argument_types))
+      << "Attempted to create a WindowAggregationHandleSum for argument Type(s) "
+      << "that SUM can not be applied to.";
+
+  return nullptr;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionSum.hpp b/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
new file mode 100644
index 0000000..047113c
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregateFunctionSum.hpp
@@ -0,0 +1,75 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_SUM_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_SUM_HPP_
+
+#include <string>
+#include <vector>
+
+#include "expressions/window_aggregation/WindowAggregateFunction.hpp"
+#include "expressions/window_aggregation/WindowAggregationID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class Type;
+class WindowAggregationHandle;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief WindowAggregateFunction representing SQL SUM() OVER term.
+ **/
+class WindowAggregateFunctionSum : public WindowAggregateFunction {
+ public:
+  static const WindowAggregateFunctionSum& Instance() {
+    static WindowAggregateFunctionSum instance;
+    return instance;
+  }
+
+  std::string getName() const override {
+    return "SUM";
+  }
+
+  bool canApplyToTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  const Type* resultTypeForArgumentTypes(
+      const std::vector<const Type*> &argument_types) const override;
+
+  WindowAggregationHandle* createHandle(
+      const std::vector<const Type*> &argument_types,
+      const std::vector<const Type*> &partition_key_types) const override;
+
+ private:
+  WindowAggregateFunctionSum()
+      : WindowAggregateFunction(WindowAggregationID::kSum) {
+  }
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregateFunctionSum);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATE_FUNCTION_SUM_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandle.hpp b/expressions/window_aggregation/WindowAggregationHandle.hpp
new file mode 100644
index 0000000..65f95d9
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregationHandle.hpp
@@ -0,0 +1,137 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogRelationSchema.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "types/operations/comparisons/Comparison.hpp"
+#include "types/operations/comparisons/ComparisonFactory.hpp"
+#include "types/operations/comparisons/ComparisonID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class InsertDestinationInterface;
+class Scalar;
+class StorageManager;
+class Type;
+class ValueAccessor;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief WindowAggregationHandle encapsulates logic for actually computing
+ *        window aggregates with particular argument(s).
+ * @note See also WindowAggregateFunction, which represents a SQL aggregate
+ *       function in the abstract sense.
+ *
+ * A WindowAggregationHandle is created by calling
+ * WindowAggregateFunction::createHandle(). The WindowAggregationHandle object
+ * provides methods that are used to actually compute the window aggregate,
+ * storing intermediate results in WindowAggregationState objects.
+ *
+ * The work flow for computing a window aggregate is:
+ *     1. Create an initial state by createInitialState().
+ *     2. One thread will handle all the computation, iterating from the first
+ *        tuple to the last tuple. Note there will be two modes that could be
+ *        used upon different situations:
+ *        a. If the window aggregate is defined as accumulative, which are:
+ *           i.  Functions applied to whole partition, such as rank(), ntile()
+ *               and dense_rank().
+ *           ii. The window frame is defined as "BETWEEN UNBOUNDED PRECEDING
+ *               AND CURRENT ROW" or "BETWEEN CURRENT ROW AND UNBOUNDED
+ *               FOLLOWING".
+ *           Then, for functions except median, we could store some global
+ *           values in the state without keeping all the tuple values around.
+ *        b. If the window frame is sliding, such as "BETWEEN 3 PRECEDING AND
+ *           3 FOLLOWING", we have to store all the tuples in the state so that
+ *           we could know which values should be dropped as the window slides.
+ *        For each computed value, generate a tuple store in the column vector.
+ *     3. Insert the new column into the original relation and return.
+ *
+ * TODO(Shixuan): Currently we don't support parallelization. The basic idea for
+ * parallelization is to calculate the partial result inside each block. Each
+ * block could visit the following blocks as long as the block's last partition
+ * is not finished. WindowAggregationOperationState will be used for handling
+ * the global state of the calculation.
+ **/
+
+class WindowAggregationHandle {
+ public:
+  /**
+   * @brief Destructor.
+   **/
+  virtual ~WindowAggregationHandle() {}
+
+  /**
+   * @brief Calculate the window aggregate result.
+   *
+   * @param block_accessors A pointer to the value accessor of block attributes.
+   * @param arguments The ColumnVectors of arguments
+   * @param partition_by_ids The ids of partition keys.
+   * @param is_row True if the frame mode is ROWS, false if it is RANGE.
+   * @param num_preceding The number of rows/range that precedes the current row.
+   * @param num_following The number of rows/range that follows the current row.
+   *
+   * @return A ColumnVector of the calculated window aggregates.
+   **/
+  virtual ColumnVector* calculate(ColumnVectorsValueAccessor* block_accessors,
+                                  std::vector<ColumnVector*> &&arguments,
+                                  const std::vector<attribute_id> &partition_by_ids,
+                                  const bool is_row,
+                                  const std::int64_t num_preceding,
+                                  const std::int64_t num_following) const = 0;
+
+ protected:
+  /**
+   * @brief Constructor.
+   *
+   * @param partition_key_types The Types of the partition key.
+   **/
+  explicit WindowAggregationHandle(
+      const std::vector<const Type*> &partition_key_types) {
+    // Comparison operators for checking if two tuples belong to the same partition.
+    for (const Type *partition_key_type : partition_key_types) {
+      equal_comparators_.emplace_back(
+          ComparisonFactory::GetComparison(ComparisonID::kEqual)
+              .makeUncheckedComparatorForTypes(*partition_key_type, *partition_key_type));
+    }
+  }
+
+  std::vector<std::unique_ptr<UncheckedComparator>> equal_comparators_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregationHandle);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandleAvg.cpp b/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
new file mode 100644
index 0000000..a6a10d4
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
@@ -0,0 +1,241 @@
+/**
+ *   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.
+ **/
+
+#include "expressions/window_aggregation/WindowAggregationHandleAvg.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/scalar/Scalar.hpp"
+#include "expressions/scalar/ScalarAttribute.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
+#include "types/operations/binary_operations/BinaryOperationID.hpp"
+#include "types/operations/comparisons/Comparison.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+WindowAggregationHandleAvg::WindowAggregationHandleAvg(
+    const std::vector<const Type*> &partition_key_types,
+    const Type &type)
+    : WindowAggregationHandle(partition_key_types),
+      argument_type_(type) {
+  // We sum Int as Long and Float as Double so that we have more headroom when
+  // adding many values.
+  TypeID type_id;
+  switch (type.getTypeID()) {
+    case kInt:
+    case kLong:
+      type_id = kLong;
+      break;
+    case kFloat:
+    case kDouble:
+      type_id = kDouble;
+      break;
+    default:
+      type_id = type.getTypeID();
+      break;
+  }
+
+  sum_type_ = &(TypeFactory::GetType(type_id));
+
+  // Result is nullable, because AVG() over 0 values (or all NULL values) is
+  // NULL.
+  result_type_
+      = &(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+              .resultTypeForArgumentTypes(*sum_type_, TypeFactory::GetType(kDouble))
+                  ->getNullableVersion());
+
+  // Make operators to do arithmetic:
+  // Add operator for summing argument values.
+  fast_add_operator_.reset(
+      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+          .makeUncheckedBinaryOperatorForTypes(*sum_type_, argument_type_));
+  // Divide operator for dividing sum by count to get final average.
+  divide_operator_.reset(
+      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+          .makeUncheckedBinaryOperatorForTypes(*sum_type_, TypeFactory::GetType(kDouble)));
+}
+
+ColumnVector* WindowAggregationHandleAvg::calculate(
+    ColumnVectorsValueAccessor *tuple_accessor,
+    std::vector<ColumnVector*> &&arguments,
+    const std::vector<attribute_id> &partition_by_ids,
+    const bool is_row,
+    const std::int64_t num_preceding,
+    const std::int64_t num_following) const {
+  DCHECK_EQ(1u, arguments.size());
+  DCHECK(arguments[0]->isNative());
+  DCHECK_EQ(static_cast<std::size_t>(tuple_accessor->getNumTuples()),
+            static_cast<const NativeColumnVector*>(arguments[0])->size());
+
+  // Initialize the output column and argument accessor.
+  NativeColumnVector *window_aggregates =
+      new NativeColumnVector(*result_type_, tuple_accessor->getNumTuples());
+  ColumnVectorsValueAccessor* argument_accessor = new ColumnVectorsValueAccessor();
+  argument_accessor->addColumn(arguments[0]);
+
+  // Create a window for each tuple and calculate the window aggregate.
+  tuple_accessor->beginIteration();
+  argument_accessor->beginIteration();
+
+  while (tuple_accessor->next() && argument_accessor->next()) {
+    const TypedValue window_aggregate = this->calculateOneWindow(tuple_accessor,
+                                                                 argument_accessor,
+                                                                 partition_by_ids,
+                                                                 is_row,
+                                                                 num_preceding,
+                                                                 num_following);
+    window_aggregates->appendTypedValue(window_aggregate);
+  }
+
+  return window_aggregates;
+}
+
+TypedValue WindowAggregationHandleAvg::calculateOneWindow(
+    ColumnVectorsValueAccessor *tuple_accessor,
+    ColumnVectorsValueAccessor *argument_accessor,
+    const std::vector<attribute_id> &partition_by_ids,
+    const bool is_row,
+    const std::int64_t num_preceding,
+    const std::int64_t num_following) const {
+  // Initialize.
+  TypedValue sum = sum_type_->makeZeroValue();
+  TypedValue current_value = argument_accessor->getTypedValue(0);
+  std::uint64_t count = 0;
+
+  // Ignore the value if null.
+  if (!current_value.isNull()) {
+    sum = fast_add_operator_->applyToTypedValues(sum, current_value);
+    count++;
+  }
+
+  // Get the partition key for the current row.
+  std::vector<TypedValue> current_row_partition_key;
+  for (attribute_id partition_by_id : partition_by_ids) {
+    current_row_partition_key.push_back(
+        tuple_accessor->getTypedValue(partition_by_id));
+  }
+
+  // Get current position.
+  tuple_id current_tuple_id = tuple_accessor->getCurrentPositionVirtual();
+
+  // Find preceding tuples.
+  int count_preceding = 0;
+  tuple_id preceding_tuple_id = current_tuple_id;
+  while (num_preceding == -1 || count_preceding < num_preceding) {
+    preceding_tuple_id--;
+
+    // No more preceding tuples.
+    if (preceding_tuple_id < 0) {
+      break;
+    }
+
+    // Get the partition keys and compare. If not the same partition as the
+    // current row, stop searching preceding tuples.
+    if (!samePartition(tuple_accessor,
+                       current_row_partition_key,
+                       preceding_tuple_id,
+                       partition_by_ids)) {
+      break;
+    }
+
+    // Actually count the element and do the calculation.
+    count_preceding++;
+    TypedValue preceding_value =
+        argument_accessor->getTypedValueAtAbsolutePosition(0, preceding_tuple_id);
+
+    // Ignore the value if null.
+    if (!preceding_value.isNull()) {
+      sum = fast_add_operator_->applyToTypedValues(sum, preceding_value);
+      count++;
+    }
+  }
+
+  // Find following tuples.
+  int count_following = 0;
+  tuple_id following_tuple_id = current_tuple_id;
+  while (num_following == -1 || count_following < num_following) {
+    following_tuple_id++;
+
+    // No more following tuples.
+    if (following_tuple_id == tuple_accessor->getNumTuples()) {
+      break;
+    }
+
+    // Get the partition keys and compare. If not the same partition as the
+    // current row, stop searching preceding tuples.
+    if (!samePartition(tuple_accessor,
+                       current_row_partition_key,
+                       following_tuple_id,
+                       partition_by_ids)) {
+      break;
+    }
+
+    // Actually count the element and do the calculation.
+    count_following++;
+    TypedValue following_value =
+        argument_accessor->getTypedValueAtAbsolutePosition(0, following_tuple_id);
+
+    // Ignore the value if null.
+    if (!following_value.isNull()) {
+      sum = fast_add_operator_->applyToTypedValues(sum, following_value);
+      count++;
+    }
+  }
+
+  // If all values are NULLs, return NULL; Otherwise, return the quotient.
+  if (count == 0) {
+    return result_type_->makeNullValue();
+  } else {
+    return divide_operator_->applyToTypedValues(sum,
+                                                TypedValue(static_cast<double>(count)));
+  }
+}
+
+bool WindowAggregationHandleAvg::samePartition(
+    const ColumnVectorsValueAccessor *tuple_accessor,
+    const std::vector<TypedValue> &current_row_partition_key,
+    const tuple_id boundary_tuple_id,
+    const std::vector<attribute_id> &partition_by_ids) const {
+  for (std::size_t partition_by_index = 0;
+       partition_by_index < partition_by_ids.size();
+       ++partition_by_index) {
+    if (!equal_comparators_[partition_by_index]->compareTypedValues(
+            current_row_partition_key[partition_by_index],
+            tuple_accessor->getTypedValueAtAbsolutePosition(
+                partition_by_ids[partition_by_index], boundary_tuple_id))) {
+      return false;
+    }
+  }
+
+  return true;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandleAvg.hpp b/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
new file mode 100644
index 0000000..5b41779
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregationHandleAvg.hpp
@@ -0,0 +1,101 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_AVG_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_AVG_HPP_
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <queue>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/window_aggregation/WindowAggregationHandle.hpp"
+#include "types/Type.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/comparisons/Comparison.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class ColumnVector;
+class ColumnVectorsValueAccessor;
+class ValueAccessor;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief A WindowAggregationHandle for average.
+ **/
+class WindowAggregationHandleAvg : public WindowAggregationHandle {
+ public:
+  ~WindowAggregationHandleAvg() override {}
+
+  ColumnVector* calculate(ColumnVectorsValueAccessor* block_accessors,
+                          std::vector<ColumnVector*> &&arguments,
+                          const std::vector<attribute_id> &partition_by_ids,
+                          const bool is_row,
+                          const std::int64_t num_preceding,
+                          const std::int64_t num_following) const override;
+
+ private:
+  friend class WindowAggregateFunctionAvg;
+
+  /**
+   * @brief Constructor.
+   *
+   * @param partition_key_types The Types of the partition key.
+   * @param type Type of the avg value.
+   **/
+  WindowAggregationHandleAvg(const std::vector<const Type*> &partition_key_types,
+                             const Type &type);
+
+  TypedValue calculateOneWindow(
+      ColumnVectorsValueAccessor *tuple_accessor,
+      ColumnVectorsValueAccessor *argument_accessor,
+      const std::vector<attribute_id> &partition_by_ids,
+      const bool is_row,
+      const std::int64_t num_preceding,
+      const std::int64_t num_following) const;
+
+  bool samePartition(const ColumnVectorsValueAccessor *tuple_accessor,
+                     const std::vector<TypedValue> &current_row_partition_key,
+                     const tuple_id boundary_tuple_id,
+                     const std::vector<attribute_id> &partition_by_ids) const;
+
+  const Type &argument_type_;
+  const Type *sum_type_;
+  const Type *result_type_;
+  std::unique_ptr<UncheckedBinaryOperator> fast_add_operator_;
+  std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
+
+  DISALLOW_COPY_AND_ASSIGN(WindowAggregationHandleAvg);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_HANDLE_AVG_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0bb4620/expressions/window_aggregation/WindowAggregationID.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationID.hpp b/expressions/window_aggregation/WindowAggregationID.hpp
new file mode 100644
index 0000000..8122df3
--- /dev/null
+++ b/expressions/window_aggregation/WindowAggregationID.hpp
@@ -0,0 +1,44 @@
+/**
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_ID_HPP_
+#define QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_ID_HPP_
+
+namespace quickstep {
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief The possible types of window aggregations.
+ **/
+enum class WindowAggregationID {
+  kAvg,
+  kCount,
+  kMin,
+  kMax,
+  kSum
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_WINDOW_AGGREGATION_WINDOW_AGGREGATION_ID_HPP_



[09/10] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
new file mode 100644
index 0000000..03a42a0
--- /dev/null
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -0,0 +1,308 @@
+/**
+ *   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/rules/AttachBloomFilters.hpp"
+
+#include <memory>
+#include <set>
+#include <unordered_set>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr AttachBloomFilters::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(
+          std::static_pointer_cast<const P::TopLevelPlan>(input)->shared_subplans()));
+
+  visitProducer(input, 0);
+  visitConsumer(input);
+
+//  for (const auto &info_vec_pair : consumers_) {
+//    std::cerr << "--------\n"
+//              << "Node " << info_vec_pair.first->getName()
+//              << " " << info_vec_pair.first << "\n";
+//
+//    for (const auto &info : info_vec_pair.second) {
+//      std::cerr << info.attribute->attribute_alias();
+//      if (info.attribute->id() != info.source_attribute->id()) {
+//        std::cerr << "{FROM " << info.source_attribute->attribute_alias() << "}";
+//      }
+//      if (info.from_sibling) {
+//        std::cerr << " sibling";
+//      }
+//      std::cerr << " @" << info.source << "[" << info.depth << "]"
+//                << ": " << info.selectivity << "\n";
+//    }
+//    std::cerr << "********\n";
+//  }
+
+  return visitAndAttach(input);
+}
+
+void AttachBloomFilters::visitProducer(const P::PhysicalPtr &node, const int depth) {
+  for (const P::PhysicalPtr &child : node->children()) {
+    visitProducer(child, depth+1);
+  }
+
+  std::vector<BloomFilterInfo> bloom_filters;
+
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    const P::HashJoinPtr &hash_join =
+        std::static_pointer_cast<const P::HashJoin>(node);
+    const P::PhysicalPtr &build_node = hash_join->right();
+    double selectivity = cost_model_->estimateSelectivity(build_node);
+    if (selectivity < 1.0) {
+      auto &build_node_info = producers_[build_node];
+      for (const auto &attr : hash_join->right_join_attributes()) {
+        build_node_info.emplace_back(node, attr, depth, selectivity, false);
+      }
+    }
+  }
+
+  const std::vector<E::AttributeReferencePtr> output_attributes(
+      node->getOutputAttributes());
+  std::unordered_set<E::ExprId> output_attribute_ids;
+  for (const auto &attr : output_attributes) {
+    output_attribute_ids.emplace(attr->id());
+  }
+
+  // First check inherited bloom filters
+  std::vector<const BloomFilterInfo*> candidates;
+  switch (node->getPhysicalType()) {
+    case P::PhysicalType::kAggregate:
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kHashJoin: {
+      for (const P::PhysicalPtr &child : node->children()) {
+        for (const BloomFilterInfo &info : producers_[child]) {
+          candidates.emplace_back(&info);
+        }
+      }
+    }
+    default:
+      break;
+  }
+
+  for (const BloomFilterInfo *info : candidates) {
+    if (output_attribute_ids.find(info->attribute->id()) != output_attribute_ids.end()) {
+      bloom_filters.emplace_back(
+          info->source, info->attribute, info->depth, info->selectivity, false);
+    }
+  }
+
+  // Self-produced bloom filters
+//  double selectivity = cost_model_->estimateSelectivity(node);
+//  if (selectivity < 1.0) {
+//    for (const auto &attr : output_attributes) {
+//      bloom_filters.emplace_back(node, attr, depth, selectivity, false);
+//    }
+//  }
+
+  producers_.emplace(node, std::move(bloom_filters));
+}
+
+void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
+  std::vector<BloomFilterInfo> bloom_filters;
+
+  // Bloom filters from parent
+  const auto &parent_bloom_filters = consumers_[node];
+  if (!parent_bloom_filters.empty()) {
+    for (const auto &child : node->children()) {
+      std::unordered_set<E::ExprId> child_output_attribute_ids;
+      for (const auto &attr : child->getOutputAttributes()) {
+        child_output_attribute_ids.emplace(attr->id());
+      }
+
+      std::vector<BloomFilterInfo> bloom_filters;
+      for (const auto &info : parent_bloom_filters) {
+        if (child_output_attribute_ids.find(info.attribute->id())
+                != child_output_attribute_ids.end()) {
+          bloom_filters.emplace_back(info.source,
+                                     info.attribute,
+                                     info.depth,
+                                     info.selectivity,
+                                     false,
+                                     info.source_attribute);
+        }
+      }
+      consumers_.emplace(child, std::move(bloom_filters));
+    }
+  }
+
+  // Bloom filters from build side to probe side via HashJoin
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    const P::HashJoinPtr hash_join =
+        std::static_pointer_cast<const P::HashJoin>(node);
+    if (hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin ||
+        hash_join->join_type() == P::HashJoin::JoinType::kLeftSemiJoin) {
+      const P::PhysicalPtr &producer_child = hash_join->right();
+      const P::PhysicalPtr &consumer_child = hash_join->left();
+      std::unordered_map<E::ExprId, E::AttributeReferencePtr> join_attribute_pairs;
+      for (std::size_t i = 0; i < hash_join->left_join_attributes().size(); ++i) {
+        const E::AttributeReferencePtr probe_join_attribute =
+            hash_join->left_join_attributes()[i];
+        const E::AttributeReferencePtr build_join_attribute =
+            hash_join->right_join_attributes()[i];
+        join_attribute_pairs.emplace(build_join_attribute->id(),
+                                     probe_join_attribute);
+      }
+
+      auto &consumer_bloom_filters = consumers_[consumer_child];
+      for (const auto &info : producers_[producer_child]) {
+        const auto pair_it = join_attribute_pairs.find(info.attribute->id());
+        if (pair_it != join_attribute_pairs.end()) {
+          consumer_bloom_filters.emplace_back(info.source,
+                                              pair_it->second,
+                                              info.depth,
+                                              info.selectivity,
+                                              true,
+                                              info.attribute);
+        }
+      }
+    }
+  }
+
+  P::PhysicalPtr consumer_child = nullptr;
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
+  }
+  if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
+    consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
+  }
+
+  if (consumer_child != nullptr) {
+    // Decide attaches
+    auto &consumer_bloom_filters = consumers_[consumer_child];
+    if (cost_model_->estimateCardinality(consumer_child) > 10000000 &&
+        !consumer_bloom_filters.empty()) {
+      std::map<E::AttributeReferencePtr, const BloomFilterInfo*> filters;
+      for (const auto &info : consumer_bloom_filters) {
+        auto it = filters.find(info.attribute);
+        if (it == filters.end()) {
+          filters.emplace(info.attribute, &info);
+        } else {
+          if (BloomFilterInfo::isBetterThan(&info, it->second)) {
+            it->second = &info;
+          }
+        }
+      }
+
+      auto &probe_attaches = getBloomFilterConfig(node);
+      for (const auto &pair : filters) {
+        auto &build_attaches = getBloomFilterConfig(pair.second->source);
+        build_attaches.addBuildSideBloomFilter(
+            pair.second->source_attribute);
+        probe_attaches.addProbeSideBloomFilter(
+            pair.first,
+            pair.second->source_attribute,
+            pair.second->source);
+      }
+    }
+  }
+
+  for (const auto &child : node->children()) {
+    visitConsumer(child);
+  }
+}
+
+P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &node) {
+  std::vector<P::PhysicalPtr> new_children;
+  bool has_changed = false;
+  for (const auto &child : node->children()) {
+    P::PhysicalPtr new_child = visitAndAttach(child);
+    if (new_child != child) {
+      has_changed = true;
+    }
+    new_children.emplace_back(new_child);
+  }
+
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    const auto attach_it = attaches_.find(node);
+    if (attach_it != attaches_.end()) {
+//      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+//        std::cout << "Attach probe from " << item.builder
+//                  << " to " << node << "\n";
+//      }
+
+      const P::HashJoinPtr hash_join =
+          std::static_pointer_cast<const P::HashJoin>(node);
+      return P::HashJoin::Create(
+          new_children[0],
+          new_children[1],
+          hash_join->left_join_attributes(),
+          hash_join->right_join_attributes(),
+          hash_join->residual_predicate(),
+          hash_join->project_expressions(),
+          hash_join->join_type(),
+          attach_it->second);
+    }
+  }
+
+  if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
+    const auto attach_it = attaches_.find(node);
+    if (attach_it != attaches_.end()) {
+//      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+//        std::cout << "Attach probe from " << item.builder
+//                  << " to " << node << "\n";
+//      }
+
+      const P::AggregatePtr aggregate =
+          std::static_pointer_cast<const P::Aggregate>(node);
+      return P::Aggregate::Create(
+          aggregate->input(),
+          aggregate->grouping_expressions(),
+          aggregate->aggregate_expressions(),
+          aggregate->filter_predicate(),
+          attach_it->second);
+    }
+  }
+
+  if (has_changed) {
+    return node->copyWithNewChildren(new_children);
+  }
+
+  return node;
+}
+
+P::BloomFilterConfig& AttachBloomFilters::getBloomFilterConfig(const physical::PhysicalPtr &node) {
+  if (attaches_.find(node) == attaches_.end()) {
+    attaches_.emplace(node, node);
+  }
+  return attaches_[node];
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/rules/AttachBloomFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.hpp b/query_optimizer/rules/AttachBloomFilters.hpp
new file mode 100644
index 0000000..e4437f7
--- /dev/null
+++ b/query_optimizer/rules/AttachBloomFilters.hpp
@@ -0,0 +1,118 @@
+/**
+ *   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_RULES_ATTACH_BLOOM_FILTERS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief TODO
+ */
+class AttachBloomFilters : public Rule<physical::Physical> {
+ public:
+  AttachBloomFilters() {}
+
+  ~AttachBloomFilters() override {}
+
+  std::string getName() const override {
+    return "AttachBloomFilters";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct BloomFilterInfo {
+    BloomFilterInfo(const physical::PhysicalPtr &source_in,
+                    const expressions::AttributeReferencePtr &attribute_in,
+                    const int depth_in,
+                    const double selectivity_in,
+                    const bool from_sibling_in,
+                    const expressions::AttributeReferencePtr &source_attribute_in = nullptr)
+        : source(source_in),
+          attribute(attribute_in),
+          depth(depth_in),
+          selectivity(selectivity_in),
+          from_sibling(from_sibling_in),
+          source_attribute(
+              source_attribute_in == nullptr
+                  ? attribute_in
+                  : source_attribute_in) {
+
+    }
+    static bool isBetterThan(const BloomFilterInfo *a,
+                             const BloomFilterInfo *b) {
+      if (a->selectivity == b->selectivity) {
+        return a->depth > b->depth;
+      } else {
+        return a->selectivity < b->selectivity;
+      }
+    }
+    physical::PhysicalPtr source;
+    expressions::AttributeReferencePtr attribute;
+    int depth;
+    double selectivity;
+    bool from_sibling;
+    expressions::AttributeReferencePtr source_attribute;
+  };
+
+  void visitProducer(const physical::PhysicalPtr &node, const int depth);
+
+  void visitConsumer(const physical::PhysicalPtr &node);
+
+  physical::PhysicalPtr visitAndAttach(const physical::PhysicalPtr &node);
+
+  physical::BloomFilterConfig &getBloomFilterConfig(const physical::PhysicalPtr &node);
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  std::map<physical::PhysicalPtr, std::vector<BloomFilterInfo>> producers_;
+  std::map<physical::PhysicalPtr, std::vector<BloomFilterInfo>> consumers_;
+  std::map<physical::PhysicalPtr, physical::BloomFilterConfig> attaches_;
+
+  DISALLOW_COPY_AND_ASSIGN(AttachBloomFilters);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 1990174..a943ef7 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -18,6 +18,7 @@
 add_subdirectory(tests)
 
 # Declare micro-libs:
+add_library(quickstep_queryoptimizer_rules_AttachBloomFilters AttachBloomFilters.cpp AttachBloomFilters.hpp)
 add_library(quickstep_queryoptimizer_rules_BottomUpRule ../../empty_src.cpp BottomUpRule.hpp)
 add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp CollapseProject.hpp)
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
@@ -35,6 +36,20 @@ add_library(quickstep_queryoptimizer_rules_UnnestSubqueries UnnestSubqueries.cpp
 
 
 # Link dependencies:
+target_link_libraries(quickstep_queryoptimizer_rules_AttachBloomFilters
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_BottomUpRule
                       glog
                       quickstep_queryoptimizer_rules_Rule
@@ -126,6 +141,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOpti
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_DisjointTreeForest
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_Rule
@@ -176,6 +192,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_UpdateExpression
 # Module all-in-one library:
 add_library(quickstep_queryoptimizer_rules ../../empty_src.cpp OptimizerRulesModule.hpp)
 target_link_libraries(quickstep_queryoptimizer_rules
+                      quickstep_queryoptimizer_rules_AttachBloomFilters
                       quickstep_queryoptimizer_rules_BottomUpRule
                       quickstep_queryoptimizer_rules_CollapseProject
                       quickstep_queryoptimizer_rules_GenerateJoins

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 9770606..cfbb5d1 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -31,6 +31,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "utility/DisjointTreeForest.hpp"
 
 #include "glog/logging.h"
 
@@ -72,6 +73,9 @@ P::PhysicalPtr StarSchemaHashJoinOrderOptimization::applyInternal(const P::Physi
     JoinGroupInfo *join_group = nullptr;
     if (parent_join_group == nullptr || !is_valid_cascading_hash_join) {
       new_join_group.reset(new JoinGroupInfo());
+      for (const auto &attr : input->getReferencedAttributes()) {
+        new_join_group->referenced_attributes.emplace(attr->id());
+      }
       join_group = new_join_group.get();
     } else {
       join_group = parent_join_group;
@@ -144,7 +148,10 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
         i,
         tables[i],
         cost_model_->estimateCardinality(tables[i]),
-        cost_model_->estimateSelectivity(tables[i]));
+        cost_model_->estimateSelectivity(tables[i]),
+        CountSharedAttributes(join_group.referenced_attributes,
+                              tables[i]->getOutputAttributes()),
+        tables[i]->getPhysicalType() == physical::PhysicalType::kAggregate);
   }
 
   // Auxiliary mapping info.
@@ -161,9 +168,19 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
     }
   }
 
-  // Create a join graph where tables are vertices, and add an edge between vertices
-  // t1 and t2 for each join predicate t1.x = t2.y
-  std::vector<std::unordered_set<std::size_t>> join_graph(table_info_storage.size());
+  std::set<TableInfo*> remaining_tables;
+  for (auto &table_info : table_info_storage) {
+    remaining_tables.emplace(&table_info);
+  }
+
+  DisjointTreeForest<E::ExprId> join_attribute_forest;
+  for (const auto &attr_id_pair : join_group.join_attribute_pairs) {
+    join_attribute_forest.makeSet(attr_id_pair.first);
+    join_attribute_forest.makeSet(attr_id_pair.second);
+    join_attribute_forest.merge(attr_id_pair.first, attr_id_pair.second);
+  }
+
+  std::map<std::size_t, std::map<std::size_t, E::ExprId>> join_attribute_groups;
   for (const auto &attr_id_pair : join_group.join_attribute_pairs) {
     DCHECK(attribute_id_to_table_info_index_map.find(attr_id_pair.first)
                != attribute_id_to_table_info_index_map.end());
@@ -176,128 +193,169 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
         attribute_id_to_table_info_index_map[attr_id_pair.second];
     DCHECK_NE(first_table_idx, second_table_idx);
 
-    table_info_storage[first_table_idx].join_attribute_pairs.emplace(
-        attr_id_pair.first, attr_id_pair.second);
-    table_info_storage[second_table_idx].join_attribute_pairs.emplace(
-        attr_id_pair.second, attr_id_pair.first);
-
-    join_graph[first_table_idx].emplace(second_table_idx);
-    join_graph[second_table_idx].emplace(first_table_idx);
-  }
-
-  std::set<TableInfo*, TableInfoPtrLessComparator> table_info_ordered_by_priority;
-  for (std::size_t i = 0; i < table_info_storage.size(); ++i) {
-    table_info_ordered_by_priority.emplace(&table_info_storage[i]);
+    DCHECK_EQ(join_attribute_forest.find(attr_id_pair.first),
+              join_attribute_forest.find(attr_id_pair.second));
+    const std::size_t attr_group_id = join_attribute_forest.find(attr_id_pair.first);
+    auto &attr_group = join_attribute_groups[attr_group_id];
+    attr_group.emplace(first_table_idx, attr_id_pair.first);
+    attr_group.emplace(second_table_idx, attr_id_pair.second);
   }
 
-  // Contruct hash join tree.
   while (true) {
-    TableInfo *first_table_info = *table_info_ordered_by_priority.begin();
-    table_info_ordered_by_priority.erase(
-        table_info_ordered_by_priority.begin());
-    const std::size_t first_table_info_id = first_table_info->table_info_id;
-
-    TableInfo *second_table_info = nullptr;
-    std::set<TableInfo*, TableInfoPtrLessComparator>::iterator second_table_info_it;
-    for (auto candidate_table_info_it = table_info_ordered_by_priority.begin();
-         candidate_table_info_it != table_info_ordered_by_priority.end();
-         ++candidate_table_info_it) {
-      TableInfo *candidate_table_info = *candidate_table_info_it;
-      const std::size_t candidate_table_info_id = candidate_table_info->table_info_id;
-
-      if (join_graph[first_table_info_id].find(candidate_table_info_id)
-              == join_graph[first_table_info_id].end() &&
-          join_graph[candidate_table_info_id].find(first_table_info_id)
-              == join_graph[candidate_table_info_id].end()) {
-        continue;
-      } else if (second_table_info == nullptr) {
-        second_table_info = candidate_table_info;
-        second_table_info_it = candidate_table_info_it;
-      }
-
-      bool is_likely_many_to_many_join = false;
-      for (const auto join_attr_pair : first_table_info->join_attribute_pairs) {
-        if (candidate_table_info->joined_attribute_set.find(join_attr_pair.second)
-                != candidate_table_info->joined_attribute_set.end()) {
-          is_likely_many_to_many_join = true;
-          break;
-        }
-      }
-      for (const auto join_attr_pair : candidate_table_info->join_attribute_pairs) {
-        if (first_table_info->joined_attribute_set.find(join_attr_pair.second)
-                != first_table_info->joined_attribute_set.end()) {
-          is_likely_many_to_many_join = true;
-          break;
+    // TODO(jianqiao): design better data structure to improve efficiency here.
+    std::unique_ptr<JoinPair> best_join = nullptr;
+    for (TableInfo *probe_table_info : remaining_tables) {
+      for (TableInfo *build_table_info : remaining_tables) {
+        if (probe_table_info != build_table_info) {
+          std::vector<E::AttributeReferencePtr> build_attrs;
+          const std::size_t probe_table_id = probe_table_info->table_info_id;
+          const std::size_t build_table_id = build_table_info->table_info_id;
+          for (const auto &attr_group_pair : join_attribute_groups) {
+            const auto &attr_group = attr_group_pair.second;
+            auto probe_it = attr_group.find(probe_table_id);
+            auto build_it = attr_group.find(build_table_id);
+            if (probe_it != attr_group.end() && build_it != attr_group.end()) {
+              build_attrs.emplace_back(
+                  attribute_id_to_reference_map.at(build_it->second));
+            }
+          }
+          if (!build_attrs.empty()
+              && build_table_info->table->impliesUniqueAttributes(build_attrs)) {
+            std::unique_ptr<JoinPair> new_join(
+                new JoinPair(probe_table_info, build_table_info));
+            if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
+//              if (best_join != nullptr) {
+//                std::cerr << "(" << best_join->probe->estimated_selectivity
+//                          << ", " << best_join->probe->estimated_cardinality << ")"
+//                          << " -- "
+//                          << "(" << best_join->build->estimated_selectivity
+//                          << ", " << best_join->build->estimated_cardinality << ")"
+//                          << "\n";
+//                std::cerr << "REPLACED WITH\n";
+//              }
+//              std::cerr << "(" << new_join->probe->estimated_selectivity
+//                        << ", " << new_join->probe->estimated_cardinality << ")"
+//                        << " -- "
+//                        << "(" << new_join->build->estimated_selectivity
+//                        << ", " << new_join->build->estimated_cardinality << ")"
+//                        << "\n****\n";
+              best_join.reset(new_join.release());
+            }
+          }
         }
       }
-      if (!is_likely_many_to_many_join) {
-        second_table_info = candidate_table_info;
-        second_table_info_it = candidate_table_info_it;
-        break;
-      }
     }
-    DCHECK(second_table_info != nullptr);
-    table_info_ordered_by_priority.erase(second_table_info_it);
 
-    const P::PhysicalPtr &left_child = first_table_info->table;
-    const P::PhysicalPtr &right_child = second_table_info->table;
+    TableInfo *selected_probe_table_info = nullptr;
+    TableInfo *selected_build_table_info = nullptr;
+
+    if (best_join != nullptr) {
+      selected_probe_table_info = best_join->probe;
+      selected_build_table_info = best_join->build;
+    }
+
+    // TODO(jianqiao): Handle the case when there is no primary key-foreign key information available.
+    CHECK(selected_probe_table_info != nullptr);
+    CHECK(selected_build_table_info != nullptr);
+
+//    std::cerr << selected_probe_table_info->estimated_selectivity
+//              << " -- "
+//              << selected_build_table_info->estimated_selectivity
+//              << "\n";
+
+//    std::cerr << selected_probe_table_info->estimated_num_output_attributes
+//              << " -- "
+//              << selected_build_table_info->estimated_num_output_attributes
+//              << "\n";
+
+    remaining_tables.erase(selected_probe_table_info);
+    remaining_tables.erase(selected_build_table_info);
+
+    const P::PhysicalPtr &probe_child = selected_probe_table_info->table;
+    const P::PhysicalPtr &build_child = selected_build_table_info->table;
     std::vector<E::NamedExpressionPtr> output_attributes;
-    for (const E::AttributeReferencePtr &left_attr : left_child->getOutputAttributes()) {
-      output_attributes.emplace_back(left_attr);
+    for (const E::AttributeReferencePtr &probe_attr : probe_child->getOutputAttributes()) {
+      output_attributes.emplace_back(probe_attr);
     }
-    for (const E::AttributeReferencePtr &right_attr : right_child->getOutputAttributes()) {
-      output_attributes.emplace_back(right_attr);
+    for (const E::AttributeReferencePtr &build_attr : build_child->getOutputAttributes()) {
+      output_attributes.emplace_back(build_attr);
     }
 
-    std::vector<E::AttributeReferencePtr> left_join_attributes;
-    std::vector<E::AttributeReferencePtr> right_join_attributes;
-    std::unordered_set<expressions::ExprId> new_joined_attribute_set;
-    for (const auto &join_attr_pair : first_table_info->join_attribute_pairs) {
-      if (second_table_info->join_attribute_pairs.find(join_attr_pair.second)
-              != second_table_info->join_attribute_pairs.end()) {
-        left_join_attributes.emplace_back(
-            attribute_id_to_reference_map[join_attr_pair.first]);
-        right_join_attributes.emplace_back(
-            attribute_id_to_reference_map[join_attr_pair.second]);
-
-        new_joined_attribute_set.emplace(join_attr_pair.first);
-        new_joined_attribute_set.emplace(join_attr_pair.second);
+    std::vector<E::AttributeReferencePtr> probe_attributes;
+    std::vector<E::AttributeReferencePtr> build_attributes;
+    const std::size_t probe_table_id = selected_probe_table_info->table_info_id;
+    const std::size_t build_table_id = selected_build_table_info->table_info_id;
+    for (const auto &attr_group_pair : join_attribute_groups) {
+      const auto &attr_group = attr_group_pair.second;
+      auto probe_it = attr_group.find(probe_table_id);
+      auto build_it = attr_group.find(build_table_id);
+      if (probe_it != attr_group.end() && build_it != attr_group.end()) {
+        probe_attributes.emplace_back(
+            attribute_id_to_reference_map.at(probe_it->second));
+        build_attributes.emplace_back(
+            attribute_id_to_reference_map.at(build_it->second));
       }
     }
-    DCHECK_GE(left_join_attributes.size(), static_cast<std::size_t>(1));
 
-    if (table_info_ordered_by_priority.size() > 0) {
+    if (remaining_tables.size() > 0) {
       P::PhysicalPtr output =
-          P::HashJoin::Create(left_child,
-                              right_child,
-                              left_join_attributes,
-                              right_join_attributes,
+          P::HashJoin::Create(probe_child,
+                              build_child,
+                              probe_attributes,
+                              build_attributes,
                               nullptr,
                               output_attributes,
                               P::HashJoin::JoinType::kInnerJoin);
 
-      second_table_info->table = output;
+//      P::PhysicalPtr output;
+//      if (selected_build_table_info->estimated_num_output_attributes >= 4 &&
+//          selected_probe_table_info->estimated_num_output_attributes < 4) {
+//        output = P::HashJoin::Create(build_child,
+//                                     probe_child,
+//                                     build_attributes,
+//                                     probe_attributes,
+//                                     nullptr,
+//                                     output_attributes,
+//                                     P::HashJoin::JoinType::kInnerJoin);
+//      } else {
+//        output = P::HashJoin::Create(probe_child,
+//                                     build_child,
+//                                     probe_attributes,
+//                                     build_attributes,
+//                                     nullptr,
+//                                     output_attributes,
+//                                     P::HashJoin::JoinType::kInnerJoin);
+//      }
+
+      selected_probe_table_info->table = output;
 
       // TODO(jianqiao): Cache the estimated cardinality for each plan in cost
       // model to avoid duplicated estimation.
-      second_table_info->estimated_cardinality = cost_model_->estimateCardinality(output);
-
-      second_table_info->join_attribute_pairs.insert(first_table_info->join_attribute_pairs.begin(),
-                                                     first_table_info->join_attribute_pairs.end());
-      second_table_info->joined_attribute_set.insert(first_table_info->joined_attribute_set.begin(),
-                                                     first_table_info->joined_attribute_set.end());
-      second_table_info->joined_attribute_set.insert(new_joined_attribute_set.begin(),
-                                                     new_joined_attribute_set.end());
-      table_info_ordered_by_priority.emplace(second_table_info);
-
-      join_graph[second_table_info->table_info_id].insert(join_graph[first_table_info_id].begin(),
-                                                          join_graph[first_table_info_id].end());
-
+      selected_probe_table_info->estimated_cardinality = cost_model_->estimateCardinality(output);
+      selected_probe_table_info->estimated_selectivity = cost_model_->estimateSelectivity(output);
+
+      selected_probe_table_info->estimated_num_output_attributes =
+          CountSharedAttributes(join_group.referenced_attributes,
+                                output->getOutputAttributes());
+      selected_probe_table_info->is_aggregation = false;
+
+      remaining_tables.emplace(selected_probe_table_info);
+
+      // Update join attribute groups.
+      for (auto &attr_group_pair : join_attribute_groups) {
+        auto &attr_group = attr_group_pair.second;
+        auto build_it = attr_group.find(build_table_id);
+        if (build_it != attr_group.end()) {
+          const E::ExprId attr_id = build_it->second;
+          attr_group.erase(build_it);
+          attr_group.emplace(probe_table_id, attr_id);
+        }
+      }
     } else {
-      return P::HashJoin::Create(left_child,
-                                 right_child,
-                                 left_join_attributes,
-                                 right_join_attributes,
+      return P::HashJoin::Create(probe_child,
+                                 build_child,
+                                 probe_attributes,
+                                 build_attributes,
                                  residual_predicate,
                                  project_expressions,
                                  P::HashJoin::JoinType::kInnerJoin);
@@ -305,5 +363,18 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
   }
 }
 
+std::size_t StarSchemaHashJoinOrderOptimization::CountSharedAttributes(
+    const std::unordered_set<expressions::ExprId> &attr_set1,
+    const std::vector<expressions::AttributeReferencePtr> &attr_set2) {
+  std::size_t cnt = 0;
+  for (const auto &attr : attr_set2) {
+    if (attr_set1.find(attr->id()) != attr_set1.end()) {
+      ++cnt;
+    }
+  }
+  return cnt;
+}
+
+
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index deddffd..33d95a5 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -62,6 +62,7 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
    * @brief A group of tables to form a hash join tree.
    */
   struct JoinGroupInfo {
+    std::unordered_set<expressions::ExprId> referenced_attributes;
     std::vector<physical::PhysicalPtr> tables;
     std::vector<std::pair<expressions::ExprId, expressions::ExprId>> join_attribute_pairs;
   };
@@ -70,49 +71,84 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
    * @brief Auxiliary information of a table for the optimizer.
    */
   struct TableInfo {
-    TableInfo(const std::size_t in_table_info_id,
-              const physical::PhysicalPtr &in_table,
-              const std::size_t in_estimated_cardinality,
-              const double in_estimated_selectivity)
-        : table_info_id(in_table_info_id),
-          table(in_table),
-          estimated_cardinality(in_estimated_cardinality),
-          estimated_selectivity(in_estimated_selectivity) {
+    TableInfo(const std::size_t table_info_id_in,
+              const physical::PhysicalPtr &table_in,
+              const std::size_t estimated_cardinality_in,
+              const double estimated_selectivity_in,
+              const std::size_t estimated_num_output_attributes_in,
+              const bool is_aggregation_in)
+        : table_info_id(table_info_id_in),
+          table(table_in),
+          estimated_cardinality(estimated_cardinality_in),
+          estimated_selectivity(estimated_selectivity_in),
+          estimated_num_output_attributes(estimated_num_output_attributes_in),
+          is_aggregation(is_aggregation_in) {
     }
 
     const std::size_t table_info_id;
     physical::PhysicalPtr table;
     std::size_t estimated_cardinality;
     double estimated_selectivity;
-    std::unordered_multimap<expressions::ExprId, expressions::ExprId> join_attribute_pairs;
-    std::unordered_set<expressions::ExprId> joined_attribute_set;
+    std::size_t estimated_num_output_attributes;
+    bool is_aggregation;
   };
 
-  /**
-   * @brief Comparator that compares the join priorities between two tables.
-   */
-  struct TableInfoPtrLessComparator {
-    inline bool operator() (const TableInfo *lhs, const TableInfo *rhs) {
-      bool swapped = false;
-      if (lhs->estimated_cardinality > rhs->estimated_cardinality) {
-        std::swap(lhs, rhs);
-        swapped = true;
+  struct JoinPair {
+    JoinPair(TableInfo *probe_in, TableInfo *build_in)
+        : probe(probe_in), build(build_in) {
+    }
+
+    inline bool isBetterThan(const JoinPair &rhs) const {
+      const auto &lhs = *this;
+      const bool lhs_has_large_output =
+          lhs.build->estimated_num_output_attributes
+              + lhs.probe->estimated_num_output_attributes > 5;
+      const bool rhs_has_large_output =
+          rhs.build->estimated_num_output_attributes
+              + rhs.probe->estimated_num_output_attributes > 5;
+      if (lhs_has_large_output || rhs_has_large_output) {
+        if (lhs_has_large_output != rhs_has_large_output) {
+          return rhs_has_large_output;
+        }
+        double lhs_selectivity =
+            lhs.build->estimated_selectivity * lhs.probe->estimated_selectivity;
+        double rhs_selectivity =
+            rhs.build->estimated_selectivity * rhs.probe->estimated_selectivity;
+        if (lhs_selectivity != rhs_selectivity) {
+          return lhs_selectivity < rhs_selectivity;
+        }
       }
 
-      if (lhs->estimated_selectivity < rhs->estimated_selectivity) {
-        return !swapped;
-      } else if (lhs->estimated_cardinality < 1000u &&
-                 rhs->estimated_cardinality > 10000u &&
-                 lhs->estimated_selectivity < rhs->estimated_selectivity * 1.5) {
-        return !swapped;
-      } else if (lhs->estimated_selectivity > rhs->estimated_selectivity) {
-        return swapped;
-      } else if (lhs->estimated_cardinality != rhs->estimated_cardinality) {
-        return !swapped;
+      const bool lhs_has_small_build =
+          !lhs_has_large_output && lhs.build->estimated_cardinality < 0x100;
+      const bool rhs_has_small_build =
+          !rhs_has_large_output && rhs.build->estimated_cardinality < 0x100;
+      if (lhs_has_small_build != rhs_has_small_build) {
+        return lhs_has_small_build;
+      }
+
+      if (lhs.probe->is_aggregation != rhs.probe->is_aggregation) {
+        return lhs.probe->is_aggregation;
+      }
+
+      if (lhs.probe->estimated_cardinality != rhs.probe->estimated_cardinality) {
+        return lhs.probe->estimated_cardinality < rhs.probe->estimated_cardinality;
+      }
+      if (lhs.build->estimated_selectivity != rhs.build->estimated_selectivity) {
+        return lhs.build->estimated_selectivity < rhs.build->estimated_selectivity;
+      }
+      if (lhs.build->estimated_cardinality != rhs.build->estimated_cardinality) {
+        return lhs.build->estimated_cardinality < rhs.build->estimated_cardinality;
+      }
+      if (lhs.probe->table != rhs.probe->table) {
+        return lhs.probe->table < rhs.probe->table;
       } else {
-        return swapped ^ (lhs->table < rhs->table);
+        return lhs.build->table < rhs.build->table;
       }
     }
+
+    TableInfo *probe;
+    TableInfo *build;
   };
 
   physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
@@ -123,6 +159,10 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
       const expressions::PredicatePtr &residual_predicate,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions);
 
+  static std::size_t CountSharedAttributes(
+      const std::unordered_set<expressions::ExprId> &attr_set1,
+      const std::vector<expressions::AttributeReferencePtr> &attr_set2);
+
   std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
 
   DISALLOW_COPY_AND_ASSIGN(StarSchemaHashJoinOrderOptimization);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index 815c13e..ac0adea 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -70,7 +70,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
                                           probe_relation,
                                           std::move(build_attribute_ids),
                                           std::move(probe_attribute_ids),
-                                          join_hash_table_id);
+                                          join_hash_table_id,
+                                          build_relation->estimateTupleCardinality());
   }
 
   QueryPlan::DAGNodeIndex createDummyBuildHashOperator(QueryPlan *query_plan,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 4bcbcf6..c46ba2c 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -77,6 +77,10 @@ class AggregationOperator : public RelationalOperator {
 
   ~AggregationOperator() override {}
 
+  std::string getName() const override {
+    return "AggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 464bbf8..952c7ac 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -93,6 +93,10 @@ class BuildHashOperator : public RelationalOperator {
 
   ~BuildHashOperator() override {}
 
+  std::string getName() const override {
+    return "BuildHashOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 18ca656..4e05448 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -69,6 +69,10 @@ class CreateIndexOperator : public RelationalOperator {
 
   ~CreateIndexOperator() override {}
 
+  std::string getName() const override {
+    return "CreateIndexOperator";
+  }
+
   /**
    * @note No WorkOrder generated for this operator.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 6d91142..b7b707b 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -66,6 +66,10 @@ class CreateTableOperator : public RelationalOperator {
 
   ~CreateTableOperator() override {}
 
+  std::string getName() const override {
+    return "CreateTableOperator";
+  }
+
   /**
    * @note No WorkOrder generated for this operator.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 74da8c1..abfe4a9 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -81,6 +81,10 @@ class DeleteOperator : public RelationalOperator {
 
   ~DeleteOperator() override {}
 
+  std::string getName() const override {
+    return "DeleteOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 181386f..ae65de5 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -58,6 +58,10 @@ class DestroyHashOperator : public RelationalOperator {
 
   ~DestroyHashOperator() override {}
 
+  std::string getName() const override {
+    return "DestroyHashOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index 6c7fca3..f854b4f 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -74,6 +74,10 @@ class DropTableOperator : public RelationalOperator {
 
   ~DropTableOperator() override {}
 
+  std::string getName() const override {
+    return "DropTableOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 158a637..0dcfc9e 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -74,6 +74,10 @@ class FinalizeAggregationOperator : public RelationalOperator {
 
   ~FinalizeAggregationOperator() override {}
 
+  std::string getName() const override {
+    return "FinalizeAggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 667df1e..16c0d82 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -59,6 +59,11 @@ using std::vector;
 
 namespace quickstep {
 
+DEFINE_int64(bloom_adapter_batch_size, 64,
+             "Number of tuples to probe in bulk in Bloom filter adapter.");
+DEFINE_bool(adapt_bloom_filters, true,
+            "Whether to adaptively adjust the ordering of bloom filters.");
+
 namespace {
 
 // Functor passed to HashTable::getAllFromValueAccessor() to collect matching
@@ -75,6 +80,11 @@ class MapBasedJoinedTupleCollector {
     joined_tuples_[tref.block].emplace_back(tref.tuple, accessor.getCurrentPosition());
   }
 
+  inline void operator()(const tuple_id probe_tid,
+                         const TupleReference &build_tref) {
+    joined_tuples_[build_tref.block].emplace_back(build_tref.tuple, probe_tid);
+  }
+
   // Get a mutable pointer to the collected map of joined tuple ID pairs. The
   // key is inner block_id, values are vectors of joined tuple ID pairs with
   // tuple ID from the inner block on the left and the outer block on the

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 5d3d7da..36b8929 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -157,6 +157,21 @@ class HashJoinOperator : public RelationalOperator {
 
   ~HashJoinOperator() override {}
 
+  std::string getName() const override {
+    switch (join_type_) {
+      case JoinType::kInnerJoin:
+        return "HashJoinOperator";
+      case JoinType::kLeftSemiJoin:
+        return "HashJoinOperator(LeftSemi)";
+      case JoinType::kLeftAntiJoin:
+        return "HashJoinOperator(LeftAnti)";
+      case JoinType::kLeftOuterJoin:
+        return "HashJoinOperator(LeftOuter)";
+      default: break;
+    }
+    LOG(FATAL) << "Unknown join type in HashJoinOperator::getName()";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,
@@ -283,8 +298,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
@@ -330,8 +346,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
@@ -411,8 +428,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
@@ -458,8 +476,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
@@ -535,8 +554,9 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 78f5199..2c6aca7 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -73,6 +73,10 @@ class InsertOperator : public RelationalOperator {
 
   ~InsertOperator() override {}
 
+  std::string getName() const override {
+    return "InsertOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 992e76d..cf190fe 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -116,6 +116,10 @@ class NestedLoopsJoinOperator : public RelationalOperator {
 
   ~NestedLoopsJoinOperator() override {}
 
+  std::string getName() const override {
+    return "NestedLoopsJoinOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 116727b..65cd213 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -55,6 +55,13 @@ class RelationalOperator {
   virtual ~RelationalOperator() {}
 
   /**
+   * @brief Get the name of this relational operator.
+   *
+   * @return The name of this relational operator.
+   */
+  virtual std::string getName() const = 0;
+
+  /**
    * @brief Generate all the next WorkOrders for this RelationalOperator.
    *
    * @note If a RelationalOperator has blocking dependencies, it should not
@@ -226,6 +233,15 @@ class RelationalOperator {
     op_index_ = operator_index;
   }
 
+  /**
+   * @brief Get the index of this operator in the query plan DAG.
+   *
+   * @return The index of this operator in the query plan DAG.
+   */
+  std::size_t getOperatorIndex() const {
+    return op_index_;
+  }
+
  protected:
   /**
    * @brief Constructor

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index f8fe5f6..08f08c8 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -93,6 +93,10 @@ class SampleOperator : public RelationalOperator {
 
   ~SampleOperator() override {}
 
+  std::string getName() const override {
+    return "SampleOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 50032b6..ebc5ffc 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -64,6 +64,10 @@ class SaveBlocksOperator : public RelationalOperator {
 
   ~SaveBlocksOperator() override {}
 
+  std::string getName() const override {
+    return "SaveBlocksOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 0c10686..ee25886 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -189,6 +189,10 @@ class SelectOperator : public RelationalOperator {
 
   ~SelectOperator() override {}
 
+  std::string getName() const override {
+    return "SelectOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index 177836f..9b07ad6 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -129,6 +129,10 @@ class SortMergeRunOperator : public RelationalOperator {
    **/
   ~SortMergeRunOperator() {}
 
+  std::string getName() const override {
+    return "SortMergeRunOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 96a3ce1..54c7feb 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -109,6 +109,10 @@ class SortRunGenerationOperator : public RelationalOperator {
 
   ~SortRunGenerationOperator() {}
 
+  std::string getName() const override {
+    return "SortRunGenerationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 1b791a6..15e7052 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -76,6 +76,10 @@ class TableGeneratorOperator : public RelationalOperator {
 
   ~TableGeneratorOperator() override {}
 
+  std::string getName() const override {
+    return "TableGeneratorOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 1a62ded..6890d7d 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -134,6 +134,10 @@ class TextScanOperator : public RelationalOperator {
 
   ~TextScanOperator() override {}
 
+  std::string getName() const override {
+    return "TextScanOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index 4471a17..d021844 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -94,6 +94,10 @@ class UpdateOperator : public RelationalOperator {
 
   ~UpdateOperator() override {}
 
+  std::string getName() const override {
+    return "UpdateOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/WindowAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WindowAggregationOperator.hpp b/relational_operators/WindowAggregationOperator.hpp
index bd83248..ee9b9b2 100644
--- a/relational_operators/WindowAggregationOperator.hpp
+++ b/relational_operators/WindowAggregationOperator.hpp
@@ -78,6 +78,10 @@ class WindowAggregationOperator : public RelationalOperator {
 
   ~WindowAggregationOperator() override {}
 
+  std::string getName() const override {
+    return "WindowAggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index df195cc..4eb6b3a 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -299,16 +299,23 @@ class WorkOrder {
     return query_id_;
   }
 
+  inline const int getOperatorIndex() const {
+    return op_index_;
+  }
+
  protected:
   /**
    * @brief Constructor.
    *
    * @param query_id The ID of the query to which this WorkOrder belongs.
    **/
-  explicit WorkOrder(const std::size_t query_id)
-      : query_id_(query_id) {}
+  explicit WorkOrder(const std::size_t query_id,
+                     const int op_index = -1)
+      : query_id_(query_id),
+        op_index_(op_index) {}
 
   const std::size_t query_id_;
+  const int op_index_;
   // A vector of preferred NUMA node IDs where this workorder should be executed.
   // These node IDs typically indicate the NUMA node IDs of the input(s) of the
   // workorder. Derived classes should ensure that there are no duplicate entries

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 4878cf1..668164c 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -46,10 +46,13 @@
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/containers/Tuple.hpp"
+#include "utility/BloomFilterAdapter.hpp"
 
 #include "glog/logging.h"
 
@@ -57,6 +60,8 @@ using std::unique_ptr;
 
 namespace quickstep {
 
+DECLARE_int64(bloom_adapter_batch_size);
+
 AggregationOperationState::AggregationOperationState(
     const CatalogRelationSchema &input_relation,
     const std::vector<const AggregateFunction*> &aggregate_functions,
@@ -64,12 +69,16 @@ AggregationOperationState::AggregationOperationState(
     std::vector<bool> &&is_distinct,
     std::vector<std::unique_ptr<const Scalar>> &&group_by,
     const Predicate *predicate,
+    std::vector<const BloomFilter *> &&bloom_filters,
+    std::vector<attribute_id> &&bloom_filter_attribute_ids,
     const std::size_t estimated_num_entries,
     const HashTableImplType hash_table_impl_type,
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
     : input_relation_(input_relation),
       predicate_(predicate),
+      bloom_filters_(std::move(bloom_filters)),
+      bloom_filter_attribute_ids_(std::move(bloom_filter_attribute_ids)),
       group_by_list_(std::move(group_by)),
       arguments_(std::move(arguments)),
       is_distinct_(std::move(is_distinct)),
@@ -183,7 +192,8 @@ AggregationOperationState::AggregationOperationState(
 AggregationOperationState* AggregationOperationState::ReconstructFromProto(
     const serialization::AggregationOperationState &proto,
     const CatalogDatabaseLite &database,
-    StorageManager *storage_manager) {
+    StorageManager *storage_manager,
+    const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters) {
   DCHECK(ProtoIsValid(proto, database));
 
   // Rebuild contructor arguments from their representation in 'proto'.
@@ -232,12 +242,24 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
                                                database));
   }
 
+  std::vector<const BloomFilter*> bloom_filter_vector;
+  std::vector<attribute_id> bloom_filter_attribute_ids;
+  for (int i = 0; i < proto.bloom_filters_size(); ++i) {
+    // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
+    const auto bloom_filter_proto = proto.bloom_filters(i);
+    bloom_filter_vector.emplace_back(
+        bloom_filters[bloom_filter_proto.bloom_filter_id()].get());
+    bloom_filter_attribute_ids.emplace_back(bloom_filter_proto.attr_id());
+  }
+
   return new AggregationOperationState(database.getRelationSchemaById(proto.relation_id()),
                                        aggregate_functions,
                                        std::move(arguments),
                                        std::move(is_distinct),
                                        std::move(group_by_expressions),
                                        predicate.release(),
+                                       std::move(bloom_filter_vector),
+                                       std::move(bloom_filter_attribute_ids),
                                        proto.estimated_num_entries(),
                                        HashTableImplTypeFromProto(proto.hash_table_impl_type()),
                                        distinctify_hash_table_impl_types,
@@ -340,6 +362,10 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
   // tuples so that it can be reused across multiple aggregates (i.e. we only
   // pay the cost of evaluating the predicate once).
   std::unique_ptr<TupleIdSequence> reuse_matches;
+  if (predicate_) {
+    reuse_matches.reset(block->getMatchesForPredicate(predicate_.get()));
+  }
+
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
@@ -358,7 +384,6 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
                                arguments_[agg_idx],
                                local_arguments_as_attributes,
                                {}, /* group_by */
-                               predicate_.get(),
                                distinctify_hashtables_[agg_idx].get(),
                                &reuse_matches,
                                nullptr /* reuse_group_by_vectors */);
@@ -369,7 +394,6 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
           block->aggregate(*handles_[agg_idx],
                            arguments_[agg_idx],
                            local_arguments_as_attributes,
-                           predicate_.get(),
                            &reuse_matches));
     }
   }
@@ -391,6 +415,72 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
   // GROUP BY expressions once).
   std::vector<std::unique_ptr<ColumnVector>> reuse_group_by_vectors;
 
+  if (predicate_) {
+    reuse_matches.reset(block->getMatchesForPredicate(predicate_.get()));
+  }
+
+  if (bloom_filters_.size() > 0) {
+    const std::size_t num_tuples = block->getNumTuples();
+//    std::cerr << "Before: " << num_tuples << " -- "
+//              << (reuse_matches ? reuse_matches->numTuples() : num_tuples)
+//              << "\n";
+    std::unique_ptr<ValueAccessor> accessor;
+    if (reuse_matches) {
+      accessor.reset(
+          block->getTupleStorageSubBlock().createValueAccessor(reuse_matches.get()));
+    } else {
+      accessor.reset(
+          block->getTupleStorageSubBlock().createValueAccessor());
+    }
+    InvokeOnAnyValueAccessor(
+        accessor.get(),
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      std::unique_ptr<TupleIdSequence> filtered(new TupleIdSequence(num_tuples));
+
+      std::vector<std::size_t> attr_size_vector;
+      attr_size_vector.reserve(bloom_filter_attribute_ids_.size());
+      for (const auto &attr : bloom_filter_attribute_ids_) {
+        auto val_and_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+        attr_size_vector.emplace_back(val_and_size.second);
+      }
+
+      std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+      bloom_filter_adapter.reset(new BloomFilterAdapter(
+          bloom_filters_, bloom_filter_attribute_ids_, attr_size_vector));
+
+      std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+      std::uint32_t num_tuples_left = accessor->getNumTuples();
+      std::vector<tuple_id> batch(num_tuples_left);
+
+      do {
+        std::uint32_t batch_size =
+            batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+        for (std::size_t i = 0; i < batch_size; ++i) {
+          accessor->next();
+          batch.push_back(accessor->getCurrentPosition());
+        }
+
+        std::size_t num_hits =
+            bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+        for (std::size_t t = 0; t < num_hits; ++t){
+          filtered->set(batch[t], true);
+        }
+
+        batch.clear();
+        num_tuples_left -= batch_size;
+        batch_size_try = batch_size * 2;
+      } while (num_tuples_left > 0);
+
+      if (reuse_matches) {
+        reuse_matches->intersectWith(*filtered);
+      } else {
+        reuse_matches.reset(filtered.release());
+      }
+    });
+//    std::cerr << "After: " << reuse_matches->numTuples() << "\n";
+  }
+
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
@@ -402,7 +492,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
                                arguments_[agg_idx],
                                nullptr, /* arguments_as_attributes */
                                group_by_list_,
-                               predicate_.get(),
                                distinctify_hashtables_[agg_idx].get(),
                                &reuse_matches,
                                &reuse_group_by_vectors);
@@ -416,7 +505,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
       block->aggregateGroupBy(*handles_[agg_idx],
                               arguments_[agg_idx],
                               group_by_list_,
-                              predicate_.get(),
                               agg_hash_table,
                               &reuse_matches,
                               &reuse_group_by_vectors);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 0199749..5db7325 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -33,6 +33,7 @@
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTablePool.hpp"
 #include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilter.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -108,6 +109,8 @@ class AggregationOperationState {
                             std::vector<bool> &&is_distinct,
                             std::vector<std::unique_ptr<const Scalar>> &&group_by,
                             const Predicate *predicate,
+                            std::vector<const BloomFilter *> &&bloom_filters,
+                            std::vector<attribute_id> &&bloom_filter_attribute_ids,
                             const std::size_t estimated_num_entries,
                             const HashTableImplType hash_table_impl_type,
                             const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
@@ -131,7 +134,8 @@ class AggregationOperationState {
   static AggregationOperationState* ReconstructFromProto(
       const serialization::AggregationOperationState &proto,
       const CatalogDatabaseLite &database,
-      StorageManager *storage_manager);
+      StorageManager *storage_manager,
+      const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters);
 
   /**
    * @brief Check whether a serialization::AggregationOperationState is
@@ -181,6 +185,10 @@ class AggregationOperationState {
   // filter predicate (if any), and the list of GROUP BY expressions (if any).
   const CatalogRelationSchema &input_relation_;
   std::unique_ptr<const Predicate> predicate_;
+
+  std::vector<const BloomFilter*> bloom_filters_;
+  std::vector<attribute_id> bloom_filter_attribute_ids_;
+
   std::vector<std::unique_ptr<const Scalar>> group_by_list_;
 
   // Each individual aggregate in this operation has an AggregationHandle and

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/AggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.proto b/storage/AggregationOperationState.proto
index bf78e3a..165148e 100644
--- a/storage/AggregationOperationState.proto
+++ b/storage/AggregationOperationState.proto
@@ -42,4 +42,10 @@ message AggregationOperationState {
 
   // Each DISTINCT aggregation has its distinctify hash table impl type.
   repeated HashTableImplType distinctify_hash_table_impl_types = 7;
+
+  message BloomFilter {
+    required uint32 bloom_filter_id = 1;
+    required uint32 attr_id = 2;
+  }
+  repeated BloomFilter bloom_filters = 8;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/BasicColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreValueAccessor.hpp b/storage/BasicColumnStoreValueAccessor.hpp
index 759e187..7907fd5 100644
--- a/storage/BasicColumnStoreValueAccessor.hpp
+++ b/storage/BasicColumnStoreValueAccessor.hpp
@@ -18,6 +18,8 @@
 #ifndef QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_
 #define QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_
 
+#include <cstddef>
+#include <utility>
 #include <vector>
 
 #include "catalog/CatalogRelationSchema.hpp"
@@ -43,7 +45,8 @@ class BasicColumnStoreValueAccessorHelper {
       : relation_(relation),
         num_tuples_(num_tuples),
         column_stripes_(column_stripes),
-        column_null_bitmaps_(column_null_bitmaps) {
+        column_null_bitmaps_(column_null_bitmaps),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()) {
   }
 
   inline tuple_id numPackedTuples() const {
@@ -61,9 +64,23 @@ class BasicColumnStoreValueAccessorHelper {
       return nullptr;
     }
 
-    // TODO(chasseur): Consider cacheing the byte lengths of attributes.
-    return static_cast<const char*>(column_stripes_[attr])
-           + (tuple * relation_.getAttributeById(attr)->getType().maximumByteLength());
+    return static_cast<const char*>(column_stripes_[attr]) + (tuple * attr_max_lengths_[attr]);
+  }
+
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                            const attribute_id attr) const {
+    DEBUG_ASSERT(tuple < num_tuples_);
+    DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
+    if (check_null
+        && (!column_null_bitmaps_.elementIsNull(attr))
+        && column_null_bitmaps_[attr].getBit(tuple)) {
+      return std::make_pair(nullptr, 0);
+    }
+
+    const std::size_t attr_length = attr_max_lengths_[attr];
+    return std::make_pair(static_cast<const char*>(column_stripes_[attr]) + (tuple * attr_length),
+                          attr_length);
   }
 
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
@@ -80,6 +97,7 @@ class BasicColumnStoreValueAccessorHelper {
   const tuple_id num_tuples_;
   const std::vector<void*> &column_stripes_;
   const PtrVector<BitVector<false>, true> &column_null_bitmaps_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   DISALLOW_COPY_AND_ASSIGN(BasicColumnStoreValueAccessorHelper);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/BloomFilterIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.cpp b/storage/BloomFilterIndexSubBlock.cpp
index e806217..a40f69f 100644
--- a/storage/BloomFilterIndexSubBlock.cpp
+++ b/storage/BloomFilterIndexSubBlock.cpp
@@ -55,7 +55,6 @@ BloomFilterIndexSubBlock::BloomFilterIndexSubBlock(const TupleStorageSubBlock &t
                     sub_block_memory_size),
       is_initialized_(false),
       is_consistent_(false),
-      random_seed_(kBloomFilterSeed),
       bit_array_size_in_bytes_(description.GetExtension(
                                    BloomFilterIndexSubBlockDescription::bloom_filter_size)) {
   CHECK(DescriptionIsValid(relation_, description_))
@@ -74,8 +73,7 @@ BloomFilterIndexSubBlock::BloomFilterIndexSubBlock(const TupleStorageSubBlock &t
   const std::uint32_t salt_count = description.GetExtension(BloomFilterIndexSubBlockDescription::number_of_hashes);
 
   // Initialize the bloom_filter_ data structure to operate on bit_array.
-  bloom_filter_.reset(new BloomFilter(random_seed_,
-                                      salt_count,
+  bloom_filter_.reset(new BloomFilter(salt_count,
                                       bit_array_size_in_bytes_,
                                       bit_array_.get(),
                                       is_bloom_filter_initialized));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/BloomFilterIndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.hpp b/storage/BloomFilterIndexSubBlock.hpp
index 4925673..8c81156 100644
--- a/storage/BloomFilterIndexSubBlock.hpp
+++ b/storage/BloomFilterIndexSubBlock.hpp
@@ -65,11 +65,6 @@ class BloomFilterIndexSubBlock : public IndexSubBlock {
     kSelectivityNone
   };
 
-  /**
-   * @brief A random seed to initialize the bloom filter hash functions.
-   **/
-  static const std::uint64_t kBloomFilterSeed = 0xA5A5A5A55A5A5A5AULL;
-
   BloomFilterIndexSubBlock(const TupleStorageSubBlock &tuple_store,
                            const IndexSubBlockDescription &description,
                            const bool new_block,
@@ -179,7 +174,6 @@ class BloomFilterIndexSubBlock : public IndexSubBlock {
  private:
   bool is_initialized_;
   bool is_consistent_;
-  const std::uint64_t random_seed_;
   const std::uint64_t bit_array_size_in_bytes_;
   std::vector<attribute_id> indexed_attribute_ids_;
   std::unique_ptr<unsigned char> bit_array_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 582effd..777a888 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -677,6 +677,8 @@ target_link_libraries(quickstep_storage_HashTable
                       quickstep_types_Type
                       quickstep_types_TypedValue
                       quickstep_utility_BloomFilter
+                      quickstep_utility_BloomFilterAdapter
+                      quickstep_utility_EventProfiler
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTableBase

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/CompressedColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreValueAccessor.hpp b/storage/CompressedColumnStoreValueAccessor.hpp
index 64eb315..984dea3 100644
--- a/storage/CompressedColumnStoreValueAccessor.hpp
+++ b/storage/CompressedColumnStoreValueAccessor.hpp
@@ -52,6 +52,7 @@ class CompressedColumnStoreValueAccessorHelper {
       const PtrVector<BitVector<false>, true> &uncompressed_column_null_bitmaps)
       : relation_(relation),
         num_tuples_(num_tuples),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
         compression_info_(compression_info),
         dictionary_coded_attributes_(dictionary_coded_attributes),
         truncated_attributes_(truncated_attributes),
@@ -84,6 +85,26 @@ class CompressedColumnStoreValueAccessorHelper {
     }
   }
 
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                            const attribute_id attr) const {
+    if (dictionary_coded_attributes_[attr]) {
+      return dictionaries_.atUnchecked(attr).getUntypedValueAndByteLengthForCode<check_null>(
+          getCode(tuple, attr));
+    } else if (truncated_attributes_[attr]) {
+      if (truncated_attribute_is_int_[attr]) {
+        int_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&int_buffer_, sizeof(int_buffer_));
+      } else {
+        long_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&long_buffer_, sizeof(long_buffer_));
+      }
+    } else {
+      return std::make_pair(getAttributePtr<check_null>(tuple, attr),
+                            attr_max_lengths_[attr]);
+    }
+  }
+
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
                                            const attribute_id attr) const {
     if (dictionary_coded_attributes_[attr]) {
@@ -138,6 +159,7 @@ class CompressedColumnStoreValueAccessorHelper {
   const CatalogRelationSchema &relation_;
 
   const tuple_id num_tuples_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   const CompressedBlockInfo &compression_info_;
   const std::vector<bool> &dictionary_coded_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/storage/CompressedPackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreValueAccessor.hpp b/storage/CompressedPackedRowStoreValueAccessor.hpp
index 024b0ec..7058aec 100644
--- a/storage/CompressedPackedRowStoreValueAccessor.hpp
+++ b/storage/CompressedPackedRowStoreValueAccessor.hpp
@@ -58,6 +58,7 @@ class CompressedPackedRowStoreValueAccessorHelper {
         num_tuples_(num_tuples),
         tuple_length_bytes_(tuple_length_bytes),
         attribute_offsets_(attribute_offsets),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
         compression_info_(compression_info),
         dictionary_coded_attributes_(dictionary_coded_attributes),
         truncated_attributes_(truncated_attributes),
@@ -92,6 +93,26 @@ class CompressedPackedRowStoreValueAccessorHelper {
     }
   }
 
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                            const attribute_id attr) const {
+    if (dictionary_coded_attributes_[attr]) {
+      return dictionaries_.atUnchecked(attr).getUntypedValueAndByteLengthForCode<check_null>(
+          getCode(tuple, attr));
+    } else if (truncated_attributes_[attr]) {
+      if (truncated_attribute_is_int_[attr]) {
+        int_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&int_buffer_, sizeof(int_buffer_));
+      } else {
+        long_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&long_buffer_, sizeof(long_buffer_));
+      }
+    } else {
+      return std::make_pair(getAttributePtr<check_null>(tuple, attr),
+                            attr_max_lengths_[attr]);
+    }
+  }
+
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
                                            const attribute_id attr) const {
     if (dictionary_coded_attributes_[attr]) {
@@ -150,6 +171,7 @@ class CompressedPackedRowStoreValueAccessorHelper {
   const tuple_id num_tuples_;
   const std::size_t tuple_length_bytes_;
   const std::vector<std::size_t> &attribute_offsets_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   const CompressedBlockInfo &compression_info_;
   const std::vector<bool> &dictionary_coded_attributes_;



[10/10] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
Initial commit


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

Branch: refs/heads/LIP-for-tpch
Commit: 97d8dca8550e79a123d74545798a84321913ef41
Parents: aaecc76
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sat Jun 11 23:14:00 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Sat Jul 30 01:34:59 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt                                  |   2 +
 catalog/CMakeLists.txt                          |   9 +
 catalog/Catalog.proto                           |   5 +
 catalog/CatalogRelation.cpp                     |  16 +-
 catalog/CatalogRelationConstraints.cpp          |  55 ++
 catalog/CatalogRelationConstraints.hpp          | 110 ++++
 catalog/CatalogRelationSchema.cpp               |  15 +
 catalog/CatalogRelationSchema.hpp               |  16 +-
 cli/CommandExecutor.cpp                         |  25 +-
 cli/QuickstepCli.cpp                            |  69 +++
 compression/CompressionDictionaryLite.hpp       |  42 ++
 query_execution/CMakeLists.txt                  |   1 +
 query_execution/QueryContext.cpp                |  11 +-
 query_execution/Worker.cpp                      |   5 +
 .../tests/QueryManagerSingleNode_unittest.cpp   |   4 +
 query_optimizer/CMakeLists.txt                  |   2 +
 query_optimizer/ExecutionGenerator.cpp          |  87 ++--
 query_optimizer/ExecutionGenerator.hpp          |   2 +-
 query_optimizer/ExecutionHeuristics.cpp         | 171 ++++---
 query_optimizer/ExecutionHeuristics.hpp         |  79 ++-
 query_optimizer/PhysicalGenerator.cpp           |   5 +-
 query_optimizer/cost_model/SimpleCostModel.cpp  |   4 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  42 +-
 query_optimizer/expressions/ExpressionUtil.hpp  |   8 +-
 query_optimizer/physical/Aggregate.cpp          |   5 +
 query_optimizer/physical/Aggregate.hpp          |  23 +-
 query_optimizer/physical/HashJoin.cpp           |  27 +
 query_optimizer/physical/HashJoin.hpp           |  23 +-
 query_optimizer/physical/Physical.hpp           |  55 ++
 query_optimizer/physical/Selection.cpp          |   6 +
 query_optimizer/physical/Selection.hpp          |   3 +
 query_optimizer/physical/TableReference.cpp     |  18 +
 query_optimizer/physical/TableReference.hpp     |   3 +
 query_optimizer/rules/AttachBloomFilters.cpp    | 308 ++++++++++++
 query_optimizer/rules/AttachBloomFilters.hpp    | 118 +++++
 query_optimizer/rules/CMakeLists.txt            |  17 +
 .../StarSchemaHashJoinOrderOptimization.cpp     | 277 ++++++----
 .../StarSchemaHashJoinOrderOptimization.hpp     | 100 ++--
 .../tests/ExecutionHeuristics_unittest.cpp      |   3 +-
 relational_operators/AggregationOperator.hpp    |   4 +
 relational_operators/BuildHashOperator.hpp      |   4 +
 relational_operators/CreateIndexOperator.hpp    |   4 +
 relational_operators/CreateTableOperator.hpp    |   4 +
 relational_operators/DeleteOperator.hpp         |   4 +
 relational_operators/DestroyHashOperator.hpp    |   4 +
 relational_operators/DropTableOperator.hpp      |   4 +
 .../FinalizeAggregationOperator.hpp             |   4 +
 relational_operators/HashJoinOperator.cpp       |  10 +
 relational_operators/HashJoinOperator.hpp       |  40 +-
 relational_operators/InsertOperator.hpp         |   4 +
 .../NestedLoopsJoinOperator.hpp                 |   4 +
 relational_operators/RelationalOperator.hpp     |  16 +
 relational_operators/SampleOperator.hpp         |   4 +
 relational_operators/SaveBlocksOperator.hpp     |   4 +
 relational_operators/SelectOperator.hpp         |   4 +
 relational_operators/SortMergeRunOperator.hpp   |   4 +
 .../SortRunGenerationOperator.hpp               |   4 +
 relational_operators/TableGeneratorOperator.hpp |   4 +
 relational_operators/TextScanOperator.hpp       |   4 +
 relational_operators/UpdateOperator.hpp         |   4 +
 .../WindowAggregationOperator.hpp               |   4 +
 relational_operators/WorkOrder.hpp              |  11 +-
 storage/AggregationOperationState.cpp           |  98 +++-
 storage/AggregationOperationState.hpp           |  10 +-
 storage/AggregationOperationState.proto         |   6 +
 storage/BasicColumnStoreValueAccessor.hpp       |  26 +-
 storage/BloomFilterIndexSubBlock.cpp            |   4 +-
 storage/BloomFilterIndexSubBlock.hpp            |   6 -
 storage/CMakeLists.txt                          |   2 +
 storage/CompressedColumnStoreValueAccessor.hpp  |  22 +
 .../CompressedPackedRowStoreValueAccessor.hpp   |  22 +
 storage/HashTable.hpp                           | 185 ++++---
 storage/HashTable.proto                         |  10 +-
 storage/HashTableFactory.hpp                    |  23 +-
 storage/PackedRowStoreValueAccessor.hpp         |  25 +-
 storage/SplitRowStoreValueAccessor.hpp          |  45 ++
 storage/StorageBlock.cpp                        |  28 +-
 storage/StorageBlock.hpp                        |   7 +-
 storage/ValueAccessor.hpp                       |  36 ++
 types/containers/ColumnVector.hpp               |  35 ++
 types/containers/ColumnVectorsValueAccessor.hpp |  17 +
 utility/BloomFilter.hpp                         | 502 ++++++++++++++-----
 utility/BloomFilter.proto                       |   6 +-
 utility/BloomFilterAdapter.hpp                  | 142 ++++++
 utility/CMakeLists.txt                          |  20 +
 utility/DAGVisualizer.cpp                       | 167 ++++++
 utility/DAGVisualizer.hpp                       |  85 ++++
 utility/DisjointTreeForest.hpp                  | 116 +++++
 utility/EventProfiler.cpp                       |  29 ++
 utility/EventProfiler.hpp                       | 188 +++++++
 utility/PlanVisualizer.cpp                      |  42 +-
 91 files changed, 3240 insertions(+), 588 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 0bbde61..6a0c8b8 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -770,6 +770,8 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_queryoptimizer_QueryProcessor
                       quickstep_storage_PreloaderThread
                       quickstep_threading_ThreadIDBasedMap
+                      quickstep_utility_DAGVisualizer
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector
                       quickstep_utility_SqlError

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index 64b4f16..0f50706 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -35,6 +35,9 @@ add_library(quickstep_catalog_CatalogDatabaseCache CatalogDatabaseCache.cpp Cata
 add_library(quickstep_catalog_CatalogDatabaseLite ../empty_src.cpp CatalogDatabaseLite.hpp)
 add_library(quickstep_catalog_CatalogErrors ../empty_src.cpp CatalogErrors.hpp)
 add_library(quickstep_catalog_CatalogRelation CatalogRelation.cpp CatalogRelation.hpp)
+add_library(quickstep_catalog_CatalogRelationConstraints
+            CatalogRelationConstraints.cpp
+            CatalogRelationConstraints.hpp)
 add_library(quickstep_catalog_CatalogRelationSchema
             CatalogRelationSchema.cpp
             CatalogRelationSchema.hpp)
@@ -117,6 +120,10 @@ target_link_libraries(quickstep_catalog_CatalogRelation
                       quickstep_threading_SpinSharedMutex
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector)
+target_link_libraries(quickstep_catalog_CatalogRelationConstraints
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_Catalog_proto
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
@@ -136,6 +143,7 @@ target_link_libraries(quickstep_catalog_CatalogRelationSchema
                       glog
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogErrors
+                      quickstep_catalog_CatalogRelationConstraints
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
                       quickstep_types_Type
@@ -182,6 +190,7 @@ target_link_libraries(quickstep_catalog
                       quickstep_catalog_CatalogDatabaseLite
                       quickstep_catalog_CatalogErrors
                       quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogRelationConstraints
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index ce4bc2e..a51172f 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -80,6 +80,10 @@ message IndexScheme {
   repeated IndexEntry index_entries = 1;
 }
 
+message CatalogRelationConstraints {
+  repeated int32 primary_key = 1;
+}
+
 message CatalogRelationStatistics {
   optional fixed64 num_tuples = 1;
   
@@ -96,6 +100,7 @@ message CatalogRelationSchema {
   required bool temporary = 3;
 
   repeated CatalogAttribute attributes = 4;
+  optional CatalogRelationConstraints constraints = 5;
 
   extensions 16 to max;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/CatalogRelation.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.cpp b/catalog/CatalogRelation.cpp
index 01aebb5..682b6be 100644
--- a/catalog/CatalogRelation.cpp
+++ b/catalog/CatalogRelation.cpp
@@ -143,21 +143,7 @@ CatalogRelation::CatalogRelation(const serialization::CatalogRelationSchema &pro
 }
 
 serialization::CatalogRelationSchema CatalogRelation::getProto() const {
-  serialization::CatalogRelationSchema proto;
-
-  proto.set_relation_id(id_);
-  proto.set_name(name_);
-  proto.set_temporary(temporary_);
-
-  for (PtrVector<CatalogAttribute, true>::const_iterator it = attr_vec_.begin();
-       it != attr_vec_.end();
-       ++it) {
-    if (it.isNull()) {
-      proto.add_attributes();
-    } else {
-      proto.add_attributes()->MergeFrom(it->getProto());
-    }
-  }
+  serialization::CatalogRelationSchema proto = CatalogRelationSchema::getProto();
 
   proto.MutableExtension(serialization::CatalogRelation::default_layout)
       ->MergeFrom(getDefaultStorageBlockLayout().getDescription());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/CatalogRelationConstraints.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.cpp b/catalog/CatalogRelationConstraints.cpp
new file mode 100644
index 0000000..4525a98
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.cpp
@@ -0,0 +1,55 @@
+/**
+ *   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 "catalog/CatalogRelationConstraints.hpp"
+
+#include "catalog/Catalog.pb.h"
+
+namespace quickstep {
+
+CatalogRelationConstraints::CatalogRelationConstraints(
+    const serialization::CatalogRelationConstraints &proto) {
+  if (proto.primary_key_size() > 0) {
+    primary_key_.reset(new std::set<attribute_id>());
+    for (std::size_t i = 0; i < proto.primary_key_size(); ++i) {
+      primary_key_->emplace(proto.primary_key(i));
+    }
+  }
+}
+
+serialization::CatalogRelationConstraints CatalogRelationConstraints::getProto() const {
+  serialization::CatalogRelationConstraints proto;
+  if (primary_key_ != nullptr) {
+    for (const auto attr_id : *primary_key_) {
+      proto.add_primary_key(attr_id);
+    }
+  }
+  return proto;
+}
+
+bool CatalogRelationConstraints::ProtoIsValid(
+    const serialization::CatalogRelationConstraints &proto,
+    const std::size_t num_attributes) {
+  for (std::size_t i = 0; i < proto.primary_key_size(); ++i) {
+    if (proto.primary_key(i) >= num_attributes) {
+      return false;
+    }
+  }
+  return true;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/CatalogRelationConstraints.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.hpp b/catalog/CatalogRelationConstraints.hpp
new file mode 100644
index 0000000..896c072
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.hpp
@@ -0,0 +1,110 @@
+/**
+ *   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_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
+#define QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <memory>
+#include <set>
+#include <utility>
+
+#include "catalog/Catalog.pb.h"
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Catalog
+ *  @{
+ */
+
+/**
+ * @brief Constraints on a catalog relation.
+ **/
+class CatalogRelationConstraints {
+ public:
+  /**
+   * @brief Constructor.
+   **/
+  CatalogRelationConstraints() {}
+
+  /**
+   * @brief Reconstruct a CatalogRelationConstraints object from its serialized
+   *        Protocol Buffer form.
+   *
+   * @param proto The Protocol Buffer serialization of a CatalogRelationConstraints
+   *        object, previously produced by getProto().
+   **/
+  explicit CatalogRelationConstraints(const serialization::CatalogRelationConstraints &proto);
+
+  /**
+   * @brief Serialize the CatalogRelationConstraints object as Protocol Buffer.
+   *
+   * @return The Protocol Buffer representation of the CatalogRelationConstraints
+   *         object.
+   **/
+  serialization::CatalogRelationConstraints getProto() const;
+
+  static bool ProtoIsValid(const serialization::CatalogRelationConstraints &proto,
+                           const std::size_t num_attributes);
+
+  bool hasPrimaryKey() const {
+    return (primary_key_ != nullptr);
+  }
+
+  const std::set<attribute_id>* getPrimaryKey() const {
+    return primary_key_.get();
+  }
+
+  template <typename IterableT>
+  void setPrimaryKey(IterableT &&primary_key) {
+    CHECK(!primary_key.empty());
+    primary_key_.reset(
+        new std::set<attribute_id>(primary_key.begin(), primary_key.end()));
+  }
+
+  void removePrimaryKey() {
+    primary_key_.reset();
+  }
+
+  bool impliesUniqueAttributes(const std::set<attribute_id> &attributes) const {
+    if (primary_key_ == nullptr) {
+      return false;
+    }
+
+    std::vector<attribute_id> attr_intersection;
+    std::set_intersection(primary_key_->begin(), primary_key_->end(),
+                          attributes.begin(), attributes.end(),
+                          std::back_inserter(attr_intersection));
+    return (attr_intersection.size() == primary_key_->size());
+  }
+
+ private:
+  std::unique_ptr<std::set<attribute_id>> primary_key_;
+
+  DISALLOW_COPY_AND_ASSIGN(CatalogRelationConstraints);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/CatalogRelationSchema.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.cpp b/catalog/CatalogRelationSchema.cpp
index 97c834f..bf8217d 100644
--- a/catalog/CatalogRelationSchema.cpp
+++ b/catalog/CatalogRelationSchema.cpp
@@ -27,6 +27,7 @@
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogErrors.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "types/Type.hpp"
 #include "utility/PtrVector.hpp"
@@ -70,6 +71,12 @@ CatalogRelationSchema::CatalogRelationSchema(const serialization::CatalogRelatio
       attr_vec_.push_back(nullptr);
     }
   }
+
+  if (proto.has_constraints()) {
+    constraints_.reset(new CatalogRelationConstraints(proto.constraints()));
+  } else {
+    constraints_.reset(new CatalogRelationConstraints());
+  }
 }
 
 bool CatalogRelationSchema::ProtoIsValid(const serialization::CatalogRelationSchema &proto) {
@@ -84,6 +91,12 @@ bool CatalogRelationSchema::ProtoIsValid(const serialization::CatalogRelationSch
     }
   }
 
+  if (proto.has_constraints()
+      && !CatalogRelationConstraints::ProtoIsValid(proto.constraints(),
+                                                   proto.attributes_size())) {
+    return false;
+  }
+
   return true;
 }
 
@@ -104,6 +117,8 @@ serialization::CatalogRelationSchema CatalogRelationSchema::getProto() const {
     }
   }
 
+  proto.mutable_constraints()->CopyFrom(constraints_->getProto());
+
   return proto;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/catalog/CatalogRelationSchema.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.hpp b/catalog/CatalogRelationSchema.hpp
index d773bc7..0c6c207 100644
--- a/catalog/CatalogRelationSchema.hpp
+++ b/catalog/CatalogRelationSchema.hpp
@@ -21,12 +21,14 @@
 #define QUICKSTEP_CATALOG_CATALOG_RELATION_SCHEMA_HPP_
 
 #include <cstddef>
+#include <memory>
 #include <string>
 #include <unordered_map>
 #include <vector>
 
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogAttribute.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
@@ -427,6 +429,14 @@ class CatalogRelationSchema {
     return max_byte_lengths_;
   }
 
+  const CatalogRelationConstraints& getConstraints() const {
+    return *constraints_;
+  }
+
+  CatalogRelationConstraints* getConstraintsMutable() {
+    return constraints_.get();
+  }
+
  protected:
   /**
    * @brief Create a new relation.
@@ -456,7 +466,8 @@ class CatalogRelationSchema {
         min_variable_byte_length_excluding_nullable_(0),
         estimated_variable_byte_length_(0),
         current_nullable_attribute_index_(-1),
-        current_variable_length_attribute_index_(-1) {
+        current_variable_length_attribute_index_(-1),
+        constraints_(new CatalogRelationConstraints()) {
   }
 
   /**
@@ -532,6 +543,9 @@ class CatalogRelationSchema {
   std::vector<int> variable_length_attribute_indices_;
   int current_variable_length_attribute_index_;
 
+  // Primary key, foreign keys, etc.
+  std::unique_ptr<CatalogRelationConstraints> constraints_;
+
  private:
   friend class CatalogDatabase;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 8acfae8..5b302c0 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -251,7 +251,8 @@ inline TypedValue executeQueryForSingleResult(
   return value;
 }
 
-void executeAnalyze(const tmb::client_id main_thread_client_id,
+void executeAnalyze(const PtrVector<ParseString> *arguments,
+                    const tmb::client_id main_thread_client_id,
                     const tmb::client_id foreman_client_id,
                     MessageBus *bus,
                     QueryProcessor *query_processor,
@@ -260,8 +261,19 @@ void executeAnalyze(const tmb::client_id main_thread_client_id,
   StorageManager *storage_manager = query_processor->getStorageManager();
 
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
-  std::vector<std::reference_wrapper<const CatalogRelation>> relations(
-      database.begin(), database.end());
+  std::vector<std::reference_wrapper<const CatalogRelation>> relations;
+  if (arguments->size() == 0) {
+    relations.insert(relations.begin(), database.begin(), database.end());
+  } else {
+    for (const auto &rel_name : *arguments) {
+      const CatalogRelation *rel = database.getRelationByName(rel_name.value());
+      if (rel == nullptr) {
+        THROW_SQL_ERROR_AT(&rel_name) << "Table does not exist";
+      } else {
+        relations.emplace_back(*rel);
+      }
+    }
+  }
 
   // Analyze each relation in the database.
   for (const CatalogRelation &relation : relations) {
@@ -341,8 +353,11 @@ void executeCommand(const ParseStatement &statement,
       executeDescribeTable(arguments, catalog_database, out);
     }
   } else if (command_str == C::kAnalyzeCommand) {
-    executeAnalyze(
-        main_thread_client_id, foreman_client_id, bus, query_processor, out);
+    executeAnalyze(arguments,
+                   main_thread_client_id,
+                   foreman_client_id,
+                   bus,
+                   query_processor, out);
   } else {
     THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 68a3599..8031dd3 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -52,6 +52,9 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include <gperftools/profiler.h>
 #endif
 
+#include "catalog/CatalogDatabase.hpp"
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/InputParserUtil.hpp"
 #include "cli/PrintToScreen.hpp"
@@ -75,6 +78,8 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 
 #include "storage/PreloaderThread.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
+#include "utility/DAGVisualizer.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 #include "utility/SqlError.hpp"
@@ -89,6 +94,8 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "tmb/message_bus.h"
 #include "tmb/message_style.h"
 
+#include "google/protobuf/text_format.h"
+
 namespace quickstep {
 class CatalogRelation;
 }
@@ -185,9 +192,50 @@ DEFINE_string(profile_file_name, "",
               // To put things in perspective, the first run is, in my experiments, about 5-10
               // times more expensive than the average run. That means the query needs to be
               // run at least a hundred times to make the impact of the first run small (< 5 %).
+DEFINE_string(profile_output, "",
+              "Output file name for writing the profiled events.");
+DEFINE_bool(visualize_dag, false,
+            "If true, visualize the execution plan DAG into a graph in DOT format.");
 
 }  // namespace quickstep
 
+void addPrimaryKeyInfoForTPCHTables(quickstep::CatalogDatabase *database) {
+  const std::vector<std::pair<std::string, std::vector<std::string>>> rel_pkeys = {
+      { "region", { "r_regionkey" } },
+      { "nation", { "n_nationkey" } },
+      { "supplier", { "s_suppkey" } },
+      { "customer", { "c_custkey" } },
+      { "part", { "p_partkey" } },
+      { "partsupp", { "ps_partkey", "ps_suppkey" } },
+      { "orders", { "o_orderkey" } }
+  };
+  for (const auto &rel_pair : rel_pkeys) {
+    CatalogRelation *rel = database->getRelationByNameMutable(rel_pair.first);
+    std::vector<quickstep::attribute_id> attrs;
+    for (const auto &pkey : rel_pair.second) {
+      attrs.emplace_back(rel->getAttributeByName(pkey)->getID());
+    }
+    rel->getConstraintsMutable()->setPrimaryKey(attrs);
+  }
+}
+
+void addPrimaryKeyInfoForSSBTables(quickstep::CatalogDatabase *database) {
+  const std::vector<std::pair<std::string, std::vector<std::string>>> rel_pkeys = {
+      { "supplier", { "s_suppkey" } },
+      { "customer", { "c_custkey" } },
+      { "part", { "p_partkey" } },
+      { "ddate", { "d_datekey" } }
+  };
+  for (const auto &rel_pair : rel_pkeys) {
+    CatalogRelation *rel = database->getRelationByNameMutable(rel_pair.first);
+    std::vector<quickstep::attribute_id> attrs;
+    for (const auto &pkey : rel_pair.second) {
+      attrs.emplace_back(rel->getAttributeByName(pkey)->getID());
+    }
+    rel->getConstraintsMutable()->setPrimaryKey(attrs);
+  }
+}
+
 int main(int argc, char* argv[]) {
   google::InitGoogleLogging(argv[0]);
   gflags::ParseCommandLineFlags(&argc, &argv, true);
@@ -295,6 +343,15 @@ int main(int argc, char* argv[]) {
     LOG(FATAL) << "NON-STANDARD EXCEPTION DURING STARTUP";
   }
 
+//  addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
+//  addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
+//  std::string proto_str;
+//  google::protobuf::TextFormat::PrintToString(
+//      query_processor->getDefaultDatabase()->getProto(), &proto_str);
+//  std::cerr << proto_str << "\n";
+//  query_processor->markCatalogAltered();
+//  query_processor->saveCatalog();
+
   // Parse the CPU affinities for workers and the preloader thread, if enabled
   // to warm up the buffer pool.
   const vector<int> worker_cpu_affinities =
@@ -434,6 +491,8 @@ int main(int argc, char* argv[]) {
         }
 
         DCHECK(query_handle->getQueryPlanMutable() != nullptr);
+        quickstep::simple_profiler.clear();
+        quickstep::relop_profiler.clear();
         start = std::chrono::steady_clock::now();
         QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
             main_thread_client_id,
@@ -446,6 +505,11 @@ int main(int argc, char* argv[]) {
               main_thread_client_id, &bus);
           end = std::chrono::steady_clock::now();
 
+          if (quickstep::FLAGS_visualize_dag) {
+            quickstep::DAGVisualizer visualizer(*query_handle->getQueryPlanMutable());
+            std::cerr << "\n" << visualizer.toDOT() << "\n";
+          }
+
           const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
           if (query_result_relation) {
             PrintToScreen::PrintRelation(*query_result_relation,
@@ -471,6 +535,11 @@ int main(int argc, char* argv[]) {
             foreman.printWorkOrderProfilingResults(query_handle->query_id(),
                                                    stdout);
           }
+          if (!quickstep::FLAGS_profile_output.empty()) {
+            std::ofstream ofs(quickstep::FLAGS_profile_output, std::ios::out);
+            quickstep::simple_profiler.writeToStream(ofs);
+            ofs.close();
+          }
         } catch (const std::exception &e) {
           fprintf(stderr, "QUERY EXECUTION ERROR: %s\n", e.what());
           break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/compression/CompressionDictionaryLite.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryLite.hpp b/compression/CompressionDictionaryLite.hpp
index 45019c0..8c7741f 100644
--- a/compression/CompressionDictionaryLite.hpp
+++ b/compression/CompressionDictionaryLite.hpp
@@ -174,6 +174,15 @@ class CompressionDictionaryLite {
     }
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthForCode(const std::uint32_t code) const {
+    if (type_is_variable_length_) {
+      return variableLengthGetUntypedValueAndByteLengthHelper<std::uint32_t, check_null>(code);
+    } else {
+      return fixedLengthGetUntypedValueAndByteLengthHelper<std::uint32_t, check_null>(code);
+    }
+  }
+
   /**
    * @brief Get the value represented by the specified code as a TypedValue.
    * @note This version is for codes of 8 bits or less. Also see
@@ -255,6 +264,39 @@ class CompressionDictionaryLite {
     return retval;
   }
 
+  template <typename CodeType, bool check_null = true>
+  inline std::pair<const void*, std::size_t> fixedLengthGetUntypedValueAndByteLengthHelper(
+      const CodeType code) const {
+    if (check_null && (code == getNullCode())) {
+      return std::make_pair(nullptr, 0);
+    }
+    DCHECK_LT(code, numberOfCodes());
+    return std::make_pair(static_cast<const char*>(dictionary_memory_)
+                              + 2 * sizeof(std::uint32_t)        // Header.
+                              + code * type_fixed_byte_length_,  // Index into value array.
+                          type_fixed_byte_length_);
+  }
+
+  template <typename CodeType, bool check_null = true>
+  inline std::pair<const void*, std::size_t> variableLengthGetUntypedValueAndByteLengthHelper(
+      const CodeType code) const {
+    if (check_null && (code == getNullCode())) {
+      return std::make_pair(nullptr, 0);
+    }
+    DCHECK_LT(code, numberOfCodes());
+
+    const std::uint32_t value_offset = static_cast<const std::uint32_t*>(dictionary_memory_)[code + 2];
+    const void *data_ptr = variable_length_data_region_ + value_offset;
+    DCHECK_LT(data_ptr, static_cast<const char*>(dictionary_memory_) + dictionary_memory_size_);
+
+    std::size_t data_size = (code == *static_cast<const std::uint32_t*>(dictionary_memory_) - 1) ?
+        (static_cast<const char*>(dictionary_memory_)
+            + dictionary_memory_size_
+            - static_cast<const char*>(data_ptr))
+        : (static_cast<const std::uint32_t*>(dictionary_memory_)[code + 3] - value_offset);
+    return std::make_pair(data_ptr, data_size);
+  }
+
   template <typename CodeType>
   inline TypedValue fixedLengthGetTypedValueHelper(const CodeType code) const {
     if (code == getNullCode()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 8bf1ab1..6b872c0 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -260,6 +260,7 @@ target_link_libraries(quickstep_queryexecution_Worker
                       quickstep_threading_Thread
                       quickstep_threading_ThreadIDBasedMap
                       quickstep_threading_ThreadUtil
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_queryexecution_WorkerDirectory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 7019b6a..fd0ed08 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -61,15 +61,16 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
       << "Attempted to create QueryContext from an invalid proto description:\n"
       << proto.DebugString();
 
+  for (int i = 0; i < proto.bloom_filters_size(); ++i) {
+    bloom_filters_.emplace_back(new BloomFilter(proto.bloom_filters(i)));
+  }
+
   for (int i = 0; i < proto.aggregation_states_size(); ++i) {
     aggregation_states_.emplace_back(
         AggregationOperationState::ReconstructFromProto(proto.aggregation_states(i),
                                                         database,
-                                                        storage_manager));
-  }
-
-  for (int i = 0; i < proto.bloom_filters_size(); ++i) {
-    bloom_filters_.emplace_back(new BloomFilter(proto.bloom_filters(i)));
+                                                        storage_manager,
+                                                        bloom_filters_));
   }
 
   for (int i = 0; i < proto.generator_functions_size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 6ba27f1..f94089f 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -29,6 +29,7 @@
 #include "relational_operators/WorkOrder.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
 #include "threading/ThreadUtil.hpp"
+#include "utility/EventProfiler.hpp"
 
 #include "glog/logging.h"
 
@@ -116,8 +117,12 @@ void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
   const size_t query_id_for_workorder = worker_message.getWorkOrder()->getQueryID();
 
   // Start measuring the execution time.
+  auto *container = relop_profiler.getContainer();
+  auto *line = container->getEventLine(worker_message.getRelationalOpIndex());
   start = std::chrono::steady_clock::now();
+  line->emplace_back();
   worker_message.getWorkOrder()->execute();
+  line->back().endEvent();
   end = std::chrono::steady_clock::now();
   delete worker_message.getWorkOrder();
   const uint64_t execution_time_microseconds =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 39ca58c..7c96e7f 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -104,6 +104,10 @@ class MockOperator: public RelationalOperator {
         num_calls_donefeedingblocks_(0) {
   }
 
+  std::string getName() const override {
+    return "MockOperator";
+  }
+
 #define MOCK_OP_LOG(x) VLOG(x) << "Op[" << op_index_ << "]: " << __func__ << ": "
 
   // The methods below are used to check whether QueryManager calls the Relational

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index a56b714..e20ca7d 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -79,6 +79,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_QueryPlan
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_costmodel_SimpleCostModel
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       quickstep_queryoptimizer_expressions_AggregateFunction
                       quickstep_queryoptimizer_expressions_Alias
                       quickstep_queryoptimizer_expressions_AttributeReference
@@ -197,6 +198,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_LogicalToPhysicalMapper
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_rules_AttachBloomFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index ce21ade..b8b4c58 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -57,6 +57,7 @@
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/cost_model/SimpleCostModel.hpp"
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AggregateFunction.hpp"
 #include "query_optimizer/expressions/Alias.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
@@ -165,6 +166,8 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
 
   cost_model_.reset(
       new cost::SimpleCostModel(top_level_physical_plan_->shared_subplans()));
+  star_schema_cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(top_level_physical_plan_->shared_subplans()));
 
   const CatalogRelation *result_relation = nullptr;
 
@@ -598,8 +601,10 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   std::vector<attribute_id> probe_attribute_ids;
   std::vector<attribute_id> build_attribute_ids;
 
-  std::vector<attribute_id> probe_original_attribute_ids;
-  std::vector<attribute_id> build_original_attribute_ids;
+  const P::BloomFilterConfig &bloom_filter_config =
+      physical_plan->bloom_filter_config();
+  std::vector<attribute_id> probe_side_bloom_filter_attribute_ids;
+  std::vector<attribute_id> build_side_bloom_filter_attribute_ids;
 
   const CatalogRelation *referenced_stored_probe_relation = nullptr;
   const CatalogRelation *referenced_stored_build_relation = nullptr;
@@ -612,18 +617,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const std::vector<E::AttributeReferencePtr> &left_join_attributes =
       physical_plan->left_join_attributes();
   for (const E::AttributeReferencePtr &left_join_attribute : left_join_attributes) {
-    // Try to determine the original stored relation referenced in the Hash Join.
-    referenced_stored_probe_relation =
-        optimizer_context_->catalog_database()->getRelationByName(left_join_attribute->relation_name());
-    if (referenced_stored_probe_relation == nullptr) {
-      // Hash Join optimizations are not possible, if the referenced relation cannot be determined.
-      skip_hash_join_optimization = true;
-    } else {
-      const attribute_id probe_operator_attribute_id =
-          referenced_stored_probe_relation->getAttributeByName(left_join_attribute->attribute_name())->getID();
-      probe_original_attribute_ids.emplace_back(probe_operator_attribute_id);
-    }
-
     const CatalogAttribute *probe_catalog_attribute
         = attribute_substitution_map_[left_join_attribute->id()];
     probe_attribute_ids.emplace_back(probe_catalog_attribute->getID());
@@ -636,18 +629,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const std::vector<E::AttributeReferencePtr> &right_join_attributes =
       physical_plan->right_join_attributes();
   for (const E::AttributeReferencePtr &right_join_attribute : right_join_attributes) {
-    // Try to determine the original stored relation referenced in the Hash Join.
-    referenced_stored_build_relation =
-        optimizer_context_->catalog_database()->getRelationByName(right_join_attribute->relation_name());
-    if (referenced_stored_build_relation == nullptr) {
-      // Hash Join optimizations are not possible, if the referenced relation cannot be determined.
-      skip_hash_join_optimization = true;
-    } else {
-      const attribute_id build_operator_attribute_id =
-          referenced_stored_build_relation->getAttributeByName(right_join_attribute->attribute_name())->getID();
-      build_original_attribute_ids.emplace_back(build_operator_attribute_id);
-    }
-
     const CatalogAttribute *build_catalog_attribute
         = attribute_substitution_map_[right_join_attribute->id()];
     build_attribute_ids.emplace_back(build_catalog_attribute->getID());
@@ -657,6 +638,20 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     }
   }
 
+  for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+    const CatalogAttribute *probe_bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    probe_side_bloom_filter_attribute_ids.emplace_back(
+        probe_bf_catalog_attribute->getID());
+  }
+
+  for (const auto &bf : bloom_filter_config.build_side_bloom_filters) {
+    const CatalogAttribute *build_bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    build_side_bloom_filter_attribute_ids.emplace_back(
+        build_bf_catalog_attribute->getID());
+  }
+
   // Remember key types for call to SimplifyHashTableImplTypeProto() below.
   std::vector<const Type*> key_types;
   for (std::vector<E::AttributeReferencePtr>::size_type attr_idx = 0;
@@ -671,22 +666,7 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     key_types.push_back(&left_attribute_type);
   }
 
-  std::size_t probe_cardinality = cost_model_->estimateCardinality(probe_physical);
   std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
-  // For inner join, we may swap the probe table and the build table.
-  if (physical_plan->join_type() == P::HashJoin::JoinType::kInnerJoin)  {
-    // Choose the smaller table as the inner build table,
-    // and the other one as the outer probe table.
-    if (probe_cardinality < build_cardinality) {
-      // Switch the probe and build physical nodes.
-      std::swap(probe_physical, build_physical);
-      std::swap(probe_cardinality, build_cardinality);
-      std::swap(probe_attribute_ids, build_attribute_ids);
-      std::swap(any_probe_attributes_nullable, any_build_attributes_nullable);
-      std::swap(probe_original_attribute_ids, build_original_attribute_ids);
-      std::swap(referenced_stored_probe_relation, referenced_stored_build_relation);
-    }
-  }
 
   // Convert the residual predicate proto.
   QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
@@ -848,9 +828,11 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
                                            join_operator_index,
                                            referenced_stored_build_relation,
                                            referenced_stored_probe_relation,
-                                           std::move(build_original_attribute_ids),
-                                           std::move(probe_original_attribute_ids),
-                                           join_hash_table_index);
+                                           bloom_filter_config,
+                                           std::move(build_side_bloom_filter_attribute_ids),
+                                           std::move(probe_side_bloom_filter_attribute_ids),
+                                           join_hash_table_index,
+                                           star_schema_cost_model_->estimateCardinality(build_physical));
   }
 }
 
@@ -1364,6 +1346,16 @@ void ExecutionGenerator::convertAggregate(
       findRelationInfoOutputByPhysical(physical_plan->input());
   aggr_state_proto->set_relation_id(input_relation_info->relation->getID());
 
+  const P::BloomFilterConfig &bloom_filter_config =
+      physical_plan->bloom_filter_config();
+  std::vector<attribute_id> bloom_filter_attribute_ids;
+
+  for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+    const CatalogAttribute *bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    bloom_filter_attribute_ids.emplace_back(bf_catalog_attribute->getID());
+  }
+
   std::vector<const Type*> group_by_types;
   for (const E::NamedExpressionPtr &grouping_expression : physical_plan->grouping_expressions()) {
     unique_ptr<const Scalar> execution_group_by_expression;
@@ -1478,6 +1470,13 @@ void ExecutionGenerator::convertAggregate(
       std::forward_as_tuple(finalize_aggregation_operator_index, output_relation));
   temporary_relation_info_vec_.emplace_back(finalize_aggregation_operator_index,
                                             output_relation);
+
+  if (FLAGS_optimize_joins) {
+    execution_heuristics_->addAggregateInfo(aggregation_operator_index,
+                                            bloom_filter_config,
+                                            std::move(bloom_filter_attribute_ids),
+                                            aggr_state_index);
+  }
 }
 
 void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 9186707..14939ff 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -37,7 +37,6 @@
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
-#include "query_optimizer/cost_model/SimpleCostModel.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/NamedExpression.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
@@ -423,6 +422,7 @@ class ExecutionGenerator {
    * @brief The cost model to use for creating the execution plan.
    */
   std::unique_ptr<cost::CostModel> cost_model_;
+  std::unique_ptr<cost::CostModel> star_schema_cost_model_;
 
   physical::TopLevelPlanPtr top_level_physical_plan_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index fc31c53..7d12745 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -25,6 +25,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -32,95 +34,106 @@
 namespace quickstep {
 namespace optimizer {
 
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+static const std::size_t kNumBitsPerByte = 8;
+DEFINE_double(bloom_num_bits_per_tuple, kNumBitsPerByte,
+    "Number of bits per tuple used to size the Bloom filter.");
+
+DEFINE_int32(bloom_num_hash_fns, 1,
+    "Number of hash functions used in the Bloom filter.");
+
 void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
                                                 serialization::QueryContext *query_context_proto) {
-  // Currently this only optimizes left deep joins using bloom filters.
-  // It uses a simple algorithm to discover the left deep joins.
-  // It starts with the first hash join in the plan and keeps on iterating
-  // over the next hash joins, till a probe on a different relation id is found.
-  // The set of hash joins found in this way forms a chain and can be recognized
-  // as a left deep join. It becomes a candidate for optimization.
-
-  // The optimization is done by modifying each of the build operators in the chain
-  // to generate a bloom filter on the build key during their hash table creation.
-  // The leaf-level probe operator is then modified to query all the bloom
-  // filters generated from all the build operators in the chain. These
-  // bloom filters are queried to test the membership of the probe key
-  // just prior to probing the hash table.
-
-  QueryPlan::DAGNodeIndex origin_node = 0;
-  while (origin_node < hash_joins_.size() - 1) {
-    std::vector<std::size_t> chained_nodes;
-    chained_nodes.push_back(origin_node);
-    for (std::size_t i = origin_node + 1; i < hash_joins_.size(); ++i) {
-      const relation_id checked_relation_id = hash_joins_[origin_node].referenced_stored_probe_relation_->getID();
-      const relation_id expected_relation_id = hash_joins_[i].referenced_stored_probe_relation_->getID();
-      if (checked_relation_id == expected_relation_id) {
-        chained_nodes.push_back(i);
-      } else {
-        break;
-      }
+  std::map<std::pair<E::ExprId, P::PhysicalPtr>,
+           std::pair<QueryContext::bloom_filter_id, QueryPlan::DAGNodeIndex>> bloom_filter_map;
+  for (const auto &info : hash_joins_) {
+    auto *hash_table_proto =
+        query_context_proto->mutable_join_hash_tables(info.join_hash_table_id_);
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.build_side_bloom_filter_ids_.size(); ++i) {
+      const QueryContext::bloom_filter_id bloom_filter_id = query_context_proto->bloom_filters_size();
+      serialization::BloomFilter *bloom_filter_proto = query_context_proto->add_bloom_filters();
+      setBloomFilterProperties(bloom_filter_proto, info.estimated_build_relation_cardinality_);
+
+      const auto &build_side_bf =
+          bloom_filter_config.build_side_bloom_filters[i];
+      bloom_filter_map.emplace(
+          std::make_pair(build_side_bf.attribute->id(),
+                         bloom_filter_config.builder),
+          std::make_pair(bloom_filter_id, info.build_operator_index_));
+
+      auto *build_side_bloom_filter = hash_table_proto->add_build_side_bloom_filters();
+      build_side_bloom_filter->set_bloom_filter_id(bloom_filter_id);
+      build_side_bloom_filter->set_attr_id(info.build_side_bloom_filter_ids_[i]);
+      std::cerr << "Build " << build_side_bf.attribute->toString()
+                << " @" << bloom_filter_config.builder << "\n";
     }
+  }
 
-    // Only chains of length greater than one are suitable candidates for semi-join optimization.
-    if (chained_nodes.size() > 1) {
-      std::unordered_map<QueryContext::bloom_filter_id, std::vector<attribute_id>> probe_bloom_filter_info;
-      for (const std::size_t node : chained_nodes) {
-        // Provision for a new bloom filter to be used by the build operator.
-        const QueryContext::bloom_filter_id bloom_filter_id =  query_context_proto->bloom_filters_size();
-        serialization::BloomFilter *bloom_filter_proto = query_context_proto->add_bloom_filters();
-
-        // Modify the bloom filter properties based on the statistics of the relation.
-        setBloomFilterProperties(bloom_filter_proto, hash_joins_[node].referenced_stored_build_relation_);
-
-        // Add build-side bloom filter information to the corresponding hash table proto.
-        query_context_proto->mutable_join_hash_tables(hash_joins_[node].join_hash_table_id_)
-            ->add_build_side_bloom_filter_id(bloom_filter_id);
-
-        probe_bloom_filter_info.insert(std::make_pair(bloom_filter_id, hash_joins_[node].probe_attributes_));
-      }
-
-      // Add probe-side bloom filter information to the corresponding hash table proto for each build-side bloom filter.
-      for (const std::pair<QueryContext::bloom_filter_id, std::vector<attribute_id>>
-               &bloom_filter_info : probe_bloom_filter_info) {
-        auto *probe_side_bloom_filter =
-            query_context_proto->mutable_join_hash_tables(hash_joins_[origin_node].join_hash_table_id_)
-                                  ->add_probe_side_bloom_filters();
-        probe_side_bloom_filter->set_probe_side_bloom_filter_id(bloom_filter_info.first);
-        for (const attribute_id &probe_attribute_id : bloom_filter_info.second) {
-          probe_side_bloom_filter->add_probe_side_attr_ids(probe_attribute_id);
-        }
-      }
-
-      // Add node dependencies from chained build nodes to origin node probe.
-      for (std::size_t i = 1; i < chained_nodes.size(); ++i) {  // Note: It starts from index 1.
-        query_plan->addDirectDependency(hash_joins_[origin_node].join_operator_index_,
-                                        hash_joins_[origin_node + i].build_operator_index_,
-                                        true /* is_pipeline_breaker */);
-      }
+  for (const auto &info : hash_joins_) {
+    auto *hash_table_proto =
+        query_context_proto->mutable_join_hash_tables(info.join_hash_table_id_);
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.probe_side_bloom_filter_ids_.size(); ++i) {
+      auto *probe_side_bloom_filter = hash_table_proto->add_probe_side_bloom_filters();
+      const auto &probe_side_bf =
+          bloom_filter_config.probe_side_bloom_filters[i];
+      std::cerr << "HashJoin probe " << probe_side_bf.attribute->toString()
+                << " @" << probe_side_bf.builder << "\n";
+
+      const auto &build_side_info =
+           bloom_filter_map.at(
+               std::make_pair(probe_side_bf.source_attribute->id(),
+                              probe_side_bf.builder));
+      probe_side_bloom_filter->set_bloom_filter_id(build_side_info.first);
+      probe_side_bloom_filter->set_attr_id(info.probe_side_bloom_filter_ids_[i]);
+//      std::cerr << "HashJoin probe attr_id = " << info.probe_side_bloom_filter_ids_[i] << "\n";
+
+      query_plan->addDirectDependency(info.join_operator_index_,
+                                      build_side_info.second,
+                                      true /* is_pipeline_breaker */);
     }
+  }
 
-    // Update the origin node.
-    origin_node = chained_nodes.back() + 1;
+  for (const auto &info : aggregates_) {
+    auto *aggregate_proto =
+        query_context_proto->mutable_aggregation_states(info.aggregate_state_id_);
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.bloom_filter_ids_.size(); ++i) {
+      auto *bloom_filter = aggregate_proto->add_bloom_filters();
+      const auto &bf =
+          bloom_filter_config.probe_side_bloom_filters[i];
+      std::cerr << "Aggregate probe " << bf.attribute->toString()
+                << " @" << bf.builder << "\n";
+
+      const auto &build_side_info =
+           bloom_filter_map.at(
+               std::make_pair(bf.source_attribute->id(),
+                              bf.builder));
+      bloom_filter->set_bloom_filter_id(build_side_info.first);
+      bloom_filter->set_attr_id(info.bloom_filter_ids_[i]);
+//      std::cerr << "Aggregate probe attr_id = "
+//                << info.bloom_filter_ids_[i] << "\n";
+
+      query_plan->addDirectDependency(info.aggregate_operator_index_,
+                                      build_side_info.second,
+                                      true /* is_pipeline_breaker */);
+    }
   }
 }
 
 void ExecutionHeuristics::setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
-                                                   const CatalogRelation *relation) {
-  const std::size_t cardinality = relation->estimateTupleCardinality();
-  if (cardinality < kOneThousand) {
-    bloom_filter_proto->set_bloom_filter_size(kOneThousand / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kVeryLowSparsityHash);
-  } else if (cardinality < kTenThousand) {
-    bloom_filter_proto->set_bloom_filter_size(kTenThousand / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kLowSparsityHash);
-  } else if (cardinality < kHundredThousand) {
-    bloom_filter_proto->set_bloom_filter_size(kHundredThousand / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kMediumSparsityHash);
-  } else {
-    bloom_filter_proto->set_bloom_filter_size(kMillion / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kHighSparsityHash);
-  }
+                                                   const std::size_t cardinality) {
+  bloom_filter_proto->set_bloom_filter_size(
+      BloomFilter::getNearestAllowedSize(
+          (FLAGS_bloom_num_bits_per_tuple * cardinality) / kNumBitsPerByte));
+//  std::cerr << "bf size = " << bloom_filter_proto->bloom_filter_size() << "\n";
+  bloom_filter_proto->set_number_of_hashes(FLAGS_bloom_num_hash_fns);
 }
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 92a7fe8..0755124 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -25,6 +25,7 @@
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -49,7 +50,7 @@ class ExecutionHeuristics {
   static const std::size_t kHundredThousand = 100000;
   static const std::size_t kMillion = 1000000;
 
-  static const std::size_t kCompressionFactor = 10;
+  static const std::size_t kCompressionFactor = 1;
 
   static const std::size_t kVeryLowSparsityHash = 1;
   static const std::size_t kLowSparsityHash = 2;
@@ -65,25 +66,48 @@ class ExecutionHeuristics {
                  const QueryPlan::DAGNodeIndex join_operator_index,
                  const CatalogRelation *referenced_stored_build_relation,
                  const CatalogRelation *referenced_stored_probe_relation,
-                 std::vector<attribute_id> &&build_attributes,
-                 std::vector<attribute_id> &&probe_attributes,
-                 const QueryContext::join_hash_table_id join_hash_table_id)
+                 const physical::BloomFilterConfig &bloom_filter_config,
+                 std::vector<attribute_id> &&build_side_bloom_filter_ids,
+                 std::vector<attribute_id> &&probe_side_bloom_filter_ids,
+                 const QueryContext::join_hash_table_id join_hash_table_id,
+                 const std::size_t estimated_build_relation_cardinality)
         : build_operator_index_(build_operator_index),
           join_operator_index_(join_operator_index),
           referenced_stored_build_relation_(referenced_stored_build_relation),
           referenced_stored_probe_relation_(referenced_stored_probe_relation),
-          build_attributes_(std::move(build_attributes)),
-          probe_attributes_(std::move(probe_attributes)),
-          join_hash_table_id_(join_hash_table_id) {
+          bloom_filter_config_(bloom_filter_config),
+          build_side_bloom_filter_ids_(std::move(build_side_bloom_filter_ids)),
+          probe_side_bloom_filter_ids_(std::move(probe_side_bloom_filter_ids)),
+          join_hash_table_id_(join_hash_table_id),
+          estimated_build_relation_cardinality_(estimated_build_relation_cardinality) {
     }
 
     const QueryPlan::DAGNodeIndex build_operator_index_;
     const QueryPlan::DAGNodeIndex join_operator_index_;
     const CatalogRelation *referenced_stored_build_relation_;
     const CatalogRelation *referenced_stored_probe_relation_;
-    const std::vector<attribute_id> build_attributes_;
-    const std::vector<attribute_id> probe_attributes_;
+    const physical::BloomFilterConfig &bloom_filter_config_;
+    const std::vector<attribute_id> build_side_bloom_filter_ids_;
+    const std::vector<attribute_id> probe_side_bloom_filter_ids_;
     const QueryContext::join_hash_table_id join_hash_table_id_;
+    const std::size_t estimated_build_relation_cardinality_;
+  };
+
+  struct AggregateInfo {
+    AggregateInfo(const QueryPlan::DAGNodeIndex aggregate_operator_index,
+                  const physical::BloomFilterConfig &bloom_filter_config,
+                  std::vector<attribute_id> &&bloom_filter_ids,
+                  const QueryContext::aggregation_state_id aggregate_state_id)
+        : aggregate_operator_index_(aggregate_operator_index),
+          bloom_filter_config_(bloom_filter_config),
+          bloom_filter_ids_(bloom_filter_ids),
+          aggregate_state_id_(aggregate_state_id) {
+    }
+
+    const QueryPlan::DAGNodeIndex aggregate_operator_index_;
+    const physical::BloomFilterConfig &bloom_filter_config_;
+    const std::vector<attribute_id> bloom_filter_ids_;
+    const QueryContext::aggregation_state_id aggregate_state_id_;
   };
 
 
@@ -109,16 +133,30 @@ class ExecutionHeuristics {
                               const QueryPlan::DAGNodeIndex join_operator_index,
                               const CatalogRelation *referenced_stored_build_relation,
                               const CatalogRelation *referenced_stored_probe_relation,
-                              std::vector<attribute_id> &&build_attributes,
-                              std::vector<attribute_id> &&probe_attributes,
-                              const QueryContext::join_hash_table_id join_hash_table_id) {
-    hash_joins_.push_back(HashJoinInfo(build_operator_index,
-                                       join_operator_index,
-                                       referenced_stored_build_relation,
-                                       referenced_stored_probe_relation,
-                                       std::move(build_attributes),
-                                       std::move(probe_attributes),
-                                       join_hash_table_id));
+                              const physical::BloomFilterConfig &bloom_filter_config,
+                              std::vector<attribute_id> &&build_side_bloom_filter_ids,
+                              std::vector<attribute_id> &&probe_side_bloom_filter_ids,
+                              const QueryContext::join_hash_table_id join_hash_table_id,
+                              const std::size_t estimated_build_relation_cardinality) {
+    hash_joins_.emplace_back(build_operator_index,
+                             join_operator_index,
+                             referenced_stored_build_relation,
+                             referenced_stored_probe_relation,
+                             bloom_filter_config,
+                             std::move(build_side_bloom_filter_ids),
+                             std::move(probe_side_bloom_filter_ids),
+                             join_hash_table_id,
+                             estimated_build_relation_cardinality);
+  }
+
+  inline void addAggregateInfo(const QueryPlan::DAGNodeIndex aggregate_operator_index,
+                               const physical::BloomFilterConfig &bloom_filter_config,
+                               std::vector<attribute_id> &&bloom_filter_ids,
+                               const QueryContext::aggregation_state_id aggregate_state_id) {
+    aggregates_.emplace_back(aggregate_operator_index,
+                             bloom_filter_config,
+                             std::move(bloom_filter_ids),
+                             aggregate_state_id);
   }
 
   /**
@@ -139,10 +177,11 @@ class ExecutionHeuristics {
    * @param relation The catalog relation on which bloom filter is being built.
    **/
   void setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
-                                const CatalogRelation *relation);
+                                const std::size_t cardinality);
 
  private:
   std::vector<HashJoinInfo> hash_joins_;
+  std::vector<AggregateInfo> aggregates_;
 
   DISALLOW_COPY_AND_ASSIGN(ExecutionHeuristics);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 75a7bc9..f73a546 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -26,6 +26,7 @@
 #include "query_optimizer/Validator.hpp"
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/AttachBloomFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
@@ -95,9 +96,11 @@ P::PhysicalPtr PhysicalGenerator::generateInitialPlan(
 P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
   if (FLAGS_reorder_hash_joins) {
+    rules.emplace_back(new PruneColumns());
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
   }
   rules.emplace_back(new PruneColumns());
+  rules.emplace_back(new AttachBloomFilters());
 
   for (std::unique_ptr<Rule<P::Physical>> &rule : rules) {
     physical_plan_ = rule->apply(physical_plan_);
@@ -108,7 +111,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   DVLOG(4) << "Optimized physical plan:\n" << physical_plan_->toString();
 
   if (FLAGS_visualize_plan) {
-  quickstep::PlanVisualizer plan_visualizer;
+    quickstep::PlanVisualizer plan_visualizer;
     std::cerr << "\n" << plan_visualizer.visualize(physical_plan_) << "\n";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index e5222ff..6794f21 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -88,7 +88,7 @@ std::size_t SimpleCostModel::estimateCardinalityForTopLevelPlan(
 
 std::size_t SimpleCostModel::estimateCardinalityForTableReference(
     const P::TableReferencePtr &physical_plan) {
-  return physical_plan->relation()->estimateTupleCardinality();
+  return physical_plan->relation()->getStatistics().getNumTuples();
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForSelection(
@@ -119,7 +119,7 @@ std::size_t SimpleCostModel::estimateCardinalityForAggregate(
     return 1;
   }
   return std::max(static_cast<std::size_t>(1),
-                  estimateCardinality(physical_plan->input()) / 10);
+                  estimateCardinality(physical_plan->input()));
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForWindowAggregate(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index badfeb1..ea21a2e 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -121,12 +121,26 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableGenerator(
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForHashJoin(
     const P::HashJoinPtr &physical_plan) {
-  std::size_t left_cardinality = estimateCardinality(physical_plan->left());
-  std::size_t right_cardinality = estimateCardinality(physical_plan->right());
-  double left_selectivity = estimateSelectivity(physical_plan->left());
-  double right_selectivity = estimateSelectivity(physical_plan->right());
-  return std::max(static_cast<std::size_t>(left_cardinality * right_selectivity) + 1,
-                  static_cast<std::size_t>(right_cardinality * left_selectivity) + 1);
+  const P::PhysicalPtr &left_child = physical_plan->left();
+  const P::PhysicalPtr &right_child = physical_plan->right();
+
+  std::size_t left_cardinality = estimateCardinality(left_child);
+  std::size_t right_cardinality = estimateCardinality(right_child);
+
+  std::size_t estimated_cardinality = std::max(left_cardinality, right_cardinality);
+  if (left_child->impliesUniqueAttributes(physical_plan->left_join_attributes())) {
+    double left_selectivity = estimateSelectivity(left_child);
+    estimated_cardinality =
+        std::min(estimated_cardinality,
+                 static_cast<std::size_t>(right_cardinality * left_selectivity));
+  }
+  if (right_child->impliesUniqueAttributes(physical_plan->right_join_attributes())) {
+    double right_selectivity = estimateSelectivity(right_child);
+    estimated_cardinality =
+        std::min(estimated_cardinality,
+                 static_cast<std::size_t>(left_cardinality * right_selectivity));
+  }
+  return estimated_cardinality;
 }
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForNestedLoopsJoin(
@@ -141,7 +155,7 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForAggregate(
     return 1;
   }
   return std::max(static_cast<std::size_t>(1),
-                  estimateCardinality(physical_plan->input()) / 10);
+                  estimateCardinality(physical_plan->input()) / 100);
 }
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForWindowAggregate(
@@ -159,8 +173,14 @@ double StarSchemaSimpleCostModel::estimateSelectivity(
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
-      return std::min(estimateSelectivity(hash_join->left()),
-                      estimateSelectivity(hash_join->right()));
+      double left_selectivity = estimateSelectivity(hash_join->left());
+      double right_selectivity = estimateSelectivity(hash_join->right());
+      double min_sel = std::min(left_selectivity, right_selectivity);
+      double max_sel = std::max(left_selectivity, right_selectivity);
+      if (max_sel < 1) {
+        min_sel *= std::max(max_sel, 0.9);
+      }
+      return min_sel;
     }
     case P::PhysicalType::kNestedLoopsJoin: {
       const P::NestedLoopsJoinPtr &nested_loop_join =
@@ -213,7 +233,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
     case E::ExpressionType::kComparisonExpression: {
       // Case 1 - Number of distinct values statistics available
       //   Case 1.1 - Equality comparison: 1.0 / num_distinct_values
-      //   Case 1.2 - Otherwise: 5.0 / num_distinct_values
+      //   Case 1.2 - Otherwise: 0.5
       // Case 2 - Number of distinct values statistics not available
       //   Case 2.1 - Equality comparison: 0.1
       //   Case 2.2 - Otherwise: 0.5
@@ -229,7 +249,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
           double unit_selectivity = 1.0 / it->second;
           return comparison_expression->isEqualityComparisonPredicate()
                      ? unit_selectivity
-                     : std::min(0.5, unit_selectivity * 5.0);
+                     : 0.5;
         }
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index 4c35719..5e9d29d 100644
--- a/query_optimizer/expressions/ExpressionUtil.hpp
+++ b/query_optimizer/expressions/ExpressionUtil.hpp
@@ -103,12 +103,12 @@ bool ContainsExpression(
  *              contain the other operand).
  * @return True if \p left is a subset of \p right.
  */
-template <class NamedExpressionType>
+template <class NamedExpressionType1, class NamedExpressionType2>
 bool SubsetOfExpressions(
-    const std::vector<std::shared_ptr<const NamedExpressionType>> &left,
-    const std::vector<std::shared_ptr<const NamedExpressionType>> &right) {
+    const std::vector<std::shared_ptr<const NamedExpressionType1>> &left,
+    const std::vector<std::shared_ptr<const NamedExpressionType2>> &right) {
   UnorderedNamedExpressionSet supset(right.begin(), right.end());
-  for (const std::shared_ptr<const NamedExpressionType> &expr : left) {
+  for (const std::shared_ptr<const NamedExpressionType1> &expr : left) {
     if (supset.find(expr) == supset.end()) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.cpp b/query_optimizer/physical/Aggregate.cpp
index c582bba..969daa7 100644
--- a/query_optimizer/physical/Aggregate.cpp
+++ b/query_optimizer/physical/Aggregate.cpp
@@ -87,6 +87,11 @@ std::vector<E::AttributeReferencePtr> Aggregate::getReferencedAttributes()
   return referenced_attributes;
 }
 
+bool Aggregate::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  return E::SubsetOfExpressions(grouping_expressions_, attributes);
+}
+
 void Aggregate::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.hpp b/query_optimizer/physical/Aggregate.hpp
index 2c2aee7..b40997c 100644
--- a/query_optimizer/physical/Aggregate.hpp
+++ b/query_optimizer/physical/Aggregate.hpp
@@ -98,6 +98,13 @@ class Aggregate : public Physical {
     return false;
   }
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
+  const BloomFilterConfig &bloom_filter_config() const {
+    return bloom_filter_config_;
+  }
+
   /**
    * @brief Creates an Aggregate physical node.
    *
@@ -111,9 +118,14 @@ class Aggregate : public Physical {
       PhysicalPtr input,
       const std::vector<expressions::NamedExpressionPtr> &grouping_expressions,
       const std::vector<expressions::AliasPtr> &aggregate_expressions,
-      const expressions::PredicatePtr &filter_predicate) {
+      const expressions::PredicatePtr &filter_predicate,
+      const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
     return AggregatePtr(
-        new Aggregate(input, grouping_expressions, aggregate_expressions, filter_predicate));
+        new Aggregate(input,
+                      grouping_expressions,
+                      aggregate_expressions,
+                      filter_predicate,
+                      bloom_filter_config));
   }
 
  protected:
@@ -130,11 +142,13 @@ class Aggregate : public Physical {
       PhysicalPtr input,
       const std::vector<expressions::NamedExpressionPtr> &grouping_expressions,
       const std::vector<expressions::AliasPtr> &aggregate_expressions,
-      const expressions::PredicatePtr &filter_predicate)
+      const expressions::PredicatePtr &filter_predicate,
+      const BloomFilterConfig &bloom_filter_config)
       : input_(input),
         grouping_expressions_(grouping_expressions),
         aggregate_expressions_(aggregate_expressions),
-        filter_predicate_(filter_predicate) {
+        filter_predicate_(filter_predicate),
+        bloom_filter_config_(bloom_filter_config) {
     addChild(input_);
   }
 
@@ -142,6 +156,7 @@ class Aggregate : public Physical {
   std::vector<expressions::NamedExpressionPtr> grouping_expressions_;
   std::vector<expressions::AliasPtr> aggregate_expressions_;
   expressions::PredicatePtr filter_predicate_;
+  BloomFilterConfig bloom_filter_config_;
 
   DISALLOW_COPY_AND_ASSIGN(Aggregate);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/HashJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.cpp b/query_optimizer/physical/HashJoin.cpp
index 71c3692..f0e72e8 100644
--- a/query_optimizer/physical/HashJoin.cpp
+++ b/query_optimizer/physical/HashJoin.cpp
@@ -85,6 +85,15 @@ bool HashJoin::maybeCopyWithPrunedExpressions(
   return false;
 }
 
+bool HashJoin::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  return (left()->impliesUniqueAttributes(left_join_attributes_)
+              && right()->impliesUniqueAttributes(attributes))
+         || (right()->impliesUniqueAttributes(right_join_attributes_)
+                 && left()->impliesUniqueAttributes(attributes));
+
+}
+
 void HashJoin::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,
@@ -106,6 +115,24 @@ void HashJoin::getFieldStringItems(
   container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(left_join_attributes_));
   container_child_field_names->push_back("right_join_attributes");
   container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(right_join_attributes_));
+
+  if (!bloom_filter_config_.build_side_bloom_filters.empty()) {
+    container_child_field_names->push_back("build_side_bloom_filters");
+    container_child_fields->emplace_back();
+    auto &container = container_child_fields->back();
+    for (const auto& bf : bloom_filter_config_.build_side_bloom_filters) {
+      container.emplace_back(bf.attribute);
+    }
+  }
+
+  if (!bloom_filter_config_.probe_side_bloom_filters.empty()) {
+    container_child_field_names->push_back("probe_side_bloom_filters");
+    container_child_fields->emplace_back();
+    auto &container = container_child_fields->back();
+    for (const auto& bf : bloom_filter_config_.probe_side_bloom_filters) {
+      container.emplace_back(bf.attribute);
+    }
+  }
 }
 
 }  // namespace physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index b904b5f..104cb52 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -115,7 +115,8 @@ class HashJoin : public BinaryJoin {
                   right_join_attributes_,
                   residual_predicate_,
                   project_expressions(),
-                  join_type_);
+                  join_type_,
+                  bloom_filter_config_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
@@ -124,6 +125,13 @@ class HashJoin : public BinaryJoin {
       const expressions::UnorderedNamedExpressionSet &referenced_expressions,
       PhysicalPtr *output) const override;
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
+  const BloomFilterConfig &bloom_filter_config() const {
+    return bloom_filter_config_;
+  }
+
   /**
    * @brief Creates a physical HashJoin. The left/right operand does not correspond to
    *        probe/build operand.
@@ -144,7 +152,8 @@ class HashJoin : public BinaryJoin {
       const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
       const expressions::PredicatePtr &residual_predicate,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions,
-      const JoinType join_type) {
+      const JoinType join_type,
+      const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
     return HashJoinPtr(
         new HashJoin(left,
                      right,
@@ -152,7 +161,8 @@ class HashJoin : public BinaryJoin {
                      right_join_attributes,
                      residual_predicate,
                      project_expressions,
-                     join_type));
+                     join_type,
+                     bloom_filter_config));
   }
 
  protected:
@@ -172,18 +182,21 @@ class HashJoin : public BinaryJoin {
       const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
       const expressions::PredicatePtr &residual_predicate,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions,
-      const JoinType join_type)
+      const JoinType join_type,
+      const BloomFilterConfig &bloom_filter_config)
       : BinaryJoin(left, right, project_expressions),
         left_join_attributes_(left_join_attributes),
         right_join_attributes_(right_join_attributes),
         residual_predicate_(residual_predicate),
-        join_type_(join_type) {
+        join_type_(join_type),
+        bloom_filter_config_(bloom_filter_config) {
   }
 
   std::vector<expressions::AttributeReferencePtr> left_join_attributes_;
   std::vector<expressions::AttributeReferencePtr> right_join_attributes_;
   expressions::PredicatePtr residual_predicate_;
   JoinType join_type_;
+  BloomFilterConfig bloom_filter_config_;
 
   DISALLOW_COPY_AND_ASSIGN(HashJoin);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/Physical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Physical.hpp b/query_optimizer/physical/Physical.hpp
index 9fdbeb5..389cd05 100644
--- a/query_optimizer/physical/Physical.hpp
+++ b/query_optimizer/physical/Physical.hpp
@@ -39,6 +39,56 @@ namespace physical {
 class Physical;
 typedef std::shared_ptr<const Physical> PhysicalPtr;
 
+struct BloomFilterConfig {
+  struct BuildSide {
+    BuildSide(const expressions::AttributeReferencePtr &attribute_in)
+        : attribute(attribute_in) {
+    }
+    expressions::AttributeReferencePtr attribute;
+  };
+  struct ProbeSide {
+    ProbeSide(const expressions::AttributeReferencePtr &attribute_in,
+              const expressions::AttributeReferencePtr &source_attribute_in,
+              const physical::PhysicalPtr &builder_in)
+        : attribute(attribute_in),
+          source_attribute(source_attribute_in),
+          builder(builder_in) {
+    }
+    expressions::AttributeReferencePtr attribute;
+    expressions::AttributeReferencePtr source_attribute;
+    PhysicalPtr builder;
+  };
+  BloomFilterConfig() {}
+  BloomFilterConfig(const PhysicalPtr &builder_in)
+      : builder(builder_in) {
+  }
+  BloomFilterConfig(const PhysicalPtr &builder_in,
+                    const std::vector<BuildSide> &build_side_bloom_filters_in,
+                    const std::vector<ProbeSide> &probe_side_bloom_filters_in)
+      : builder(builder_in),
+        build_side_bloom_filters(build_side_bloom_filters_in),
+        probe_side_bloom_filters(probe_side_bloom_filters_in) {
+  }
+  void addBuildSideBloomFilter(const expressions::AttributeReferencePtr &attribute_in) {
+    for (const auto &build_bf : build_side_bloom_filters) {
+      if (attribute_in == build_bf.attribute) {
+        return;
+      }
+    }
+    build_side_bloom_filters.emplace_back(attribute_in);
+  }
+  void addProbeSideBloomFilter(const expressions::AttributeReferencePtr &attribute_in,
+                               const expressions::AttributeReferencePtr &source_attribute_in,
+                               const physical::PhysicalPtr &builder_in) {
+    probe_side_bloom_filters.emplace_back(attribute_in,
+                                          source_attribute_in,
+                                          builder_in);
+  }
+  PhysicalPtr builder;
+  std::vector<BuildSide> build_side_bloom_filters;
+  std::vector<ProbeSide> probe_side_bloom_filters;
+};
+
 /**
  * @brief Base class for physical plan nodes.
  */
@@ -84,6 +134,11 @@ class Physical : public OptimizerTree<Physical> {
       const expressions::UnorderedNamedExpressionSet &referenced_expressions,
       PhysicalPtr *output) const = 0;
 
+  virtual bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+    return false;
+  }
+
  protected:
   /**
    * @brief Constructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/Selection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.cpp b/query_optimizer/physical/Selection.cpp
index 5e1a03f..f4cdd1a 100644
--- a/query_optimizer/physical/Selection.cpp
+++ b/query_optimizer/physical/Selection.cpp
@@ -80,6 +80,12 @@ bool Selection::maybeCopyWithPrunedExpressions(
   return false;
 }
 
+bool Selection::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  return input()->impliesUniqueAttributes(attributes);
+}
+
+
 void Selection::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index d8c1319..68cae65 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -84,6 +84,9 @@ class Selection : public Physical {
       const expressions::UnorderedNamedExpressionSet &referenced_attributes,
       PhysicalPtr *output) const override;
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates a Selection.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/TableReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.cpp b/query_optimizer/physical/TableReference.cpp
index 4a66ddf..bc73046 100644
--- a/query_optimizer/physical/TableReference.cpp
+++ b/query_optimizer/physical/TableReference.cpp
@@ -18,6 +18,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 
 #include <string>
+#include <set>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -30,6 +31,23 @@ namespace physical {
 
 namespace E = ::quickstep::optimizer::expressions;
 
+bool TableReference::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  std::set<E::ExprId> attr_ids;
+  for (const auto &attr : attributes) {
+    attr_ids.emplace(attr->id());
+  }
+
+  std::set<attribute_id> rel_attr_ids;
+  for (std::size_t i = 0; i < attribute_list_.size(); ++i) {
+    if (attr_ids.find(attribute_list_[i]->id()) != attr_ids.end()) {
+      rel_attr_ids.emplace(i);
+    }
+  }
+
+  return relation_->getConstraints().impliesUniqueAttributes(rel_attr_ids);
+}
+
 void TableReference::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/97d8dca8/query_optimizer/physical/TableReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.hpp b/query_optimizer/physical/TableReference.hpp
index bde9b97..bc07043 100644
--- a/query_optimizer/physical/TableReference.hpp
+++ b/query_optimizer/physical/TableReference.hpp
@@ -88,6 +88,9 @@ class TableReference : public Physical {
     return false;
   }
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates a TableReference.
    *