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 2017/02/21 03:37:44 UTC

[01/50] [abbrv] incubator-quickstep git commit: Add unit test for CatalogRelationStatistics [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/LIP-time-decomposition 12f0a8ec1 -> 0ba3eb695 (forced update)


Add unit test for CatalogRelationStatistics


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

Branch: refs/heads/LIP-time-decomposition
Commit: 0f4938caa29096f18bb699c8f746a733f2262698
Parents: 0780b84
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 23 20:54:51 2017 -0600
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Sun Jan 29 23:13:45 2017 -0800

----------------------------------------------------------------------
 catalog/CMakeLists.txt                          |  27 +++
 .../CatalogRelationStatistics_unittest.cpp      | 219 +++++++++++++++++++
 2 files changed, 246 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f4938ca/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index 7de9a67..3c64e97 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -225,6 +225,31 @@ target_link_libraries(Catalog_unittest
                       quickstep_utility_PtrVector)
 add_test(Catalog_unittest Catalog_unittest)
 
+add_executable(CatalogRelationStatistics_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/CatalogRelationStatistics_unittest.cpp")
+target_link_libraries(CatalogRelationStatistics_unittest
+                      gtest
+                      gtest_main
+                      quickstep_catalog_Catalog
+                      quickstep_catalog_CatalogDatabase
+                      quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogRelationStatistics
+                      quickstep_cli_CommandExecutor
+                      quickstep_cli_DropRelation
+                      quickstep_parser_ParseStatement
+                      quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_ForemanSingleNode
+                      quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_queryexecution_Worker
+                      quickstep_queryexecution_WorkerDirectory
+                      quickstep_queryoptimizer_QueryHandle
+                      quickstep_queryoptimizer_QueryProcessor
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      tmb)
+add_test(CatalogRelationStatistics_unittest CatalogRelationStatistics_unittest)
+
 if(QUICKSTEP_HAVE_LIBNUMA)
 add_executable(NUMAPlacementScheme_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/NUMAPlacementScheme_unittest.cpp")
@@ -253,3 +278,5 @@ target_link_libraries(PartitionScheme_unittest
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_EqualComparison)
 add_test(PartitionScheme_unittest PartitionScheme_unittest)
+
+file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/catalog_relation_statistics_test_data)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0f4938ca/catalog/tests/CatalogRelationStatistics_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/CatalogRelationStatistics_unittest.cpp b/catalog/tests/CatalogRelationStatistics_unittest.cpp
new file mode 100644
index 0000000..294a6c7
--- /dev/null
+++ b/catalog/tests/CatalogRelationStatistics_unittest.cpp
@@ -0,0 +1,219 @@
+/**
+ * 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 <cstdio>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "catalog/Catalog.hpp"
+#include "catalog/CatalogDatabase.hpp"
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationStatistics.hpp"
+#include "cli/CommandExecutor.hpp"
+#include "cli/DropRelation.hpp"
+#include "parser/ParseStatement.hpp"
+#include "parser/SqlParserWrapper.hpp"
+#include "query_execution/ForemanSingleNode.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/Worker.hpp"
+#include "query_execution/WorkerDirectory.hpp"
+#include "query_optimizer/QueryHandle.hpp"
+#include "query_optimizer/QueryProcessor.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+
+#include "glog/logging.h"
+#include "gtest/gtest.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+namespace {
+
+constexpr char kStoragePath[] = "./catalog_relation_statistics_test_data/";
+
+constexpr attribute_id kFirstAttributeId = 0;
+constexpr attribute_id kSecondAttributeId = 1;
+
+}  // namespace
+
+class CatalogRelationStatisticsTest : public ::testing::Test {
+ protected:
+  virtual void SetUp() {
+    // Set up the environment for running end-to-end queries.
+    quickstep::ClientIDMap::Instance();
+
+    bus_.Initialize();
+
+    main_thread_client_id_ = bus_.Connect();
+    bus_.RegisterClientAsSender(main_thread_client_id_, kAdmitRequestMessage);
+    bus_.RegisterClientAsSender(main_thread_client_id_, kPoisonMessage);
+    bus_.RegisterClientAsReceiver(main_thread_client_id_, kWorkloadCompletionMessage);
+
+    std::string catalog_path(kStoragePath);
+    catalog_path.append(kCatalogFilename);
+
+    std::ofstream catalog_file(catalog_path.c_str());
+    Catalog catalog;
+    catalog.addDatabase(new CatalogDatabase(nullptr, "default"));
+    catalog.getProto().SerializeToOstream(&catalog_file);
+    catalog_file.close();
+
+    storage_manager_.reset(new StorageManager(kStoragePath));
+    query_processor_.reset(new QueryProcessor(std::move(catalog_path)));
+
+    worker_.reset(new Worker(0, &bus_));
+    worker_directory_.reset(
+        new WorkerDirectory(1, {worker_->getBusClientID()}, {-1}));
+
+    foreman_.reset(
+        new ForemanSingleNode(main_thread_client_id_,
+                              worker_directory_.get(),
+                              &bus_,
+                              query_processor_->getDefaultDatabase(),
+                              storage_manager_.get()));
+
+    worker_->start();
+    foreman_->start();
+  }
+
+  virtual void TearDown() {
+    for (const auto &relation : *query_processor_->getDefaultDatabase()) {
+      DropRelation::Drop(relation,
+                         query_processor_->getDefaultDatabase(),
+                         storage_manager_.get());
+    }
+
+    QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id_, &bus_);
+    worker_->join();
+    foreman_->join();
+  }
+
+  void executeQuery(const std::string &query_string) {
+    SqlParserWrapper parser_wrapper;
+    parser_wrapper.feedNextBuffer(new std::string(query_string));
+
+    ParseResult result = parser_wrapper.getNextStatement();
+    DCHECK(result.condition == ParseResult::kSuccess);
+
+    const ParseStatement &statement = *result.parsed_statement;
+    std::unique_ptr<QueryHandle> query_handle =
+        std::make_unique<QueryHandle>(query_processor_->query_id(),
+                                      main_thread_client_id_,
+                                      statement.getPriority());
+    query_processor_->generateQueryHandle(statement, query_handle.get());
+
+    QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+        main_thread_client_id_,
+        foreman_->getBusClientID(),
+        query_handle.release(),
+        &bus_);
+
+    QueryExecutionUtil::ReceiveQueryCompletionMessage(main_thread_client_id_, &bus_);
+  }
+
+  void executeAnalyze(const std::string &rel_name) {
+    SqlParserWrapper parser_wrapper;
+    parser_wrapper.feedNextBuffer(new std::string("\\analyze " + rel_name));
+
+    ParseResult result = parser_wrapper.getNextStatement();
+    DCHECK(result.condition == ParseResult::kSuccess);
+
+    const ParseStatement &statement = *result.parsed_statement;
+    DCHECK(statement.getStatementType() == ParseStatement::kCommand);
+    quickstep::cli::executeCommand(statement,
+                                   *(query_processor_->getDefaultDatabase()),
+                                   main_thread_client_id_,
+                                   foreman_->getBusClientID(),
+                                   &bus_,
+                                   storage_manager_.get(),
+                                   query_processor_.get(),
+                                   stdout);
+  }
+
+  const CatalogRelation *getRelationByName(const std::string &rel_name) const {
+    const CatalogRelation *relation =
+        query_processor_->getDefaultDatabase()->getRelationByName(rel_name);
+    DCHECK(relation != nullptr);
+    return relation;
+  }
+
+ private:
+  MessageBusImpl bus_;
+  tmb::client_id main_thread_client_id_;
+
+  std::unique_ptr<StorageManager> storage_manager_;
+  std::unique_ptr<QueryProcessor> query_processor_;
+
+  std::unique_ptr<Worker> worker_;
+  std::unique_ptr<WorkerDirectory> worker_directory_;
+  std::unique_ptr<ForemanSingleNode> foreman_;
+};
+
+TEST_F(CatalogRelationStatisticsTest, AnalyzeTest) {
+  executeQuery("CREATE TABLE analyzetest(x INT, y DOUBLE);");
+  executeQuery("INSERT INTO analyzetest VALUES(0, -0.5);");
+  executeQuery("INSERT INTO analyzetest VALUES(1, 0);");
+  executeQuery("INSERT INTO analyzetest VALUES(0, 0.5);");
+  executeAnalyze("analyzetest");
+
+  const CatalogRelation *relation = getRelationByName("analyzetest");
+  const CatalogRelationStatistics &stat = relation->getStatistics();
+
+  EXPECT_EQ(3u, stat.getNumTuples());
+
+  EXPECT_EQ(2u, stat.getNumDistinctValues(kFirstAttributeId));
+  EXPECT_EQ(0, stat.getMinValue(kFirstAttributeId).getLiteral<int>());
+  EXPECT_EQ(1, stat.getMaxValue(kFirstAttributeId).getLiteral<int>());
+
+  EXPECT_EQ(3u, stat.getNumDistinctValues(kSecondAttributeId));
+  EXPECT_EQ(-0.5, stat.getMinValue(kSecondAttributeId).getLiteral<double>());
+  EXPECT_EQ(0.5, stat.getMaxValue(kSecondAttributeId).getLiteral<double>());
+}
+
+TEST_F(CatalogRelationStatisticsTest, ExactnessTest) {
+  executeQuery("CREATE TABLE exactnesstest(x INT);");
+
+  const CatalogRelationStatistics &stat =
+      getRelationByName("exactnesstest")->getStatistics();
+
+  EXPECT_FALSE(stat.isExact());
+
+  const std::vector<std::string> queries = {
+      "INSERT INTO exactnesstest VALUES(1);",
+      "INSERT INTO exactnesstest SELECT i FROM generate_series(2, 10) AS gs(i);",
+      "DELETE FROM exactnesstest WHERE x = 5;",
+      "UPDATE exactnesstest SET x = 100 WHERE x = 10;"
+  };
+
+  for (const std::string &query : queries) {
+    executeQuery(query);
+    EXPECT_FALSE(stat.isExact());
+
+    executeAnalyze("exactnesstest");
+    EXPECT_TRUE(stat.isExact());
+  }
+}
+
+}  // namespace quickstep


[37/50] [abbrv] incubator-quickstep git commit: Minor refactored distributed query execution.

Posted by ji...@apache.org.
Minor refactored distributed query execution.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 3011ddf61ec92efcb833ef0a1168255ff97fb9f9
Parents: 5773027
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 17:36:45 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 17:42:42 2017 -0800

----------------------------------------------------------------------
 query_execution/ForemanDistributed.cpp        |  1 -
 query_execution/PolicyEnforcerBase.cpp        |  2 -
 query_execution/PolicyEnforcerBase.hpp        | 14 -----
 query_execution/PolicyEnforcerDistributed.cpp | 59 ++++++++++------------
 4 files changed, 27 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3011ddf6/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index 4d95f16..8c20e65 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -175,7 +175,6 @@ void ForemanDistributed::run() {
       case kQueryInitiateResponseMessage: {
         S::QueryInitiateResponseMessage proto;
         CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-        CHECK(policy_enforcer_->existQuery(proto.query_id()));
         break;
       }
       case kCatalogRelationNewBlockMessage:  // Fall through

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3011ddf6/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index a26b84e..082f6e9 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -156,8 +156,6 @@ void PolicyEnforcerBase::removeQuery(const std::size_t query_id) {
                  << " that hasn't finished its execution";
   }
   admitted_queries_.erase(query_id);
-
-  removed_query_ids_.insert(query_id);
 }
 
 bool PolicyEnforcerBase::admitQueries(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3011ddf6/query_execution/PolicyEnforcerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.hpp b/query_execution/PolicyEnforcerBase.hpp
index baf9c68..4107817 100644
--- a/query_execution/PolicyEnforcerBase.hpp
+++ b/query_execution/PolicyEnforcerBase.hpp
@@ -103,16 +103,6 @@ class PolicyEnforcerBase {
   void processMessage(const TaggedMessage &tagged_message);
 
   /**
-   * @brief Check if the given query id ever exists.
-   *
-   * @return True if the query ever exists, otherwise false.
-   **/
-  inline bool existQuery(const std::size_t query_id) const {
-    return admitted_queries_.find(query_id) != admitted_queries_.end() ||
-           removed_query_ids_.find(query_id) != removed_query_ids_.end();
-  }
-
-  /**
    * @brief Check if there are any queries to be executed.
    *
    * @return True if there is at least one active or waiting query, false if
@@ -179,10 +169,6 @@ class PolicyEnforcerBase {
   // Key = query ID, value = QueryManagerBase* for the key query.
   std::unordered_map<std::size_t, std::unique_ptr<QueryManagerBase>> admitted_queries_;
 
-  // TODO(quickstep-team): Delete a 'query_id' after receiving all
-  // 'QueryInitiateResponseMessage's for the 'query_id'.
-  std::unordered_set<std::size_t> removed_query_ids_;
-
   // The queries which haven't been admitted yet.
   std::queue<QueryHandle*> waiting_queries_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3011ddf6/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 49a1d9a..ef5abb0 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -68,8 +68,15 @@ void PolicyEnforcerDistributed::getWorkOrderProtoMessages(
   // TODO(harshad) - Make this function generic enough so that it
   // works well when multiple queries are getting executed.
   if (admitted_queries_.empty()) {
-    LOG(WARNING) << "Requesting WorkOrderProtoMessages when no query is running";
-    return;
+    if (waiting_queries_.empty()) {
+      LOG(WARNING) << "Requesting WorkOrderProtoMessages when no query is running";
+      return;
+    } else {
+      // Admit the earliest waiting query.
+      QueryHandle *new_query = waiting_queries_.front();
+      waiting_queries_.pop();
+      admitQuery(new_query);
+    }
   }
 
   const std::size_t per_query_share =
@@ -106,28 +113,28 @@ void PolicyEnforcerDistributed::getWorkOrderProtoMessages(
 }
 
 bool PolicyEnforcerDistributed::admitQuery(QueryHandle *query_handle) {
-  if (admitted_queries_.size() < PolicyEnforcerBase::kMaxConcurrentQueries) {
-    // Ok to admit the query.
-    const std::size_t query_id = query_handle->query_id();
-    if (admitted_queries_.find(query_id) == admitted_queries_.end()) {
-      // NOTE(zuyu): Should call before constructing a 'QueryManager'.
-      // Otherwise, an InitiateRebuildMessage may be sent before 'QueryContext'
-      // initializes.
-      initiateQueryInShiftboss(query_handle);
-
-      // Query with the same ID not present, ok to admit.
-      admitted_queries_[query_id].reset(
-          new QueryManagerDistributed(query_handle, shiftboss_directory_, foreman_client_id_, bus_));
-      return true;
-    } else {
-      LOG(ERROR) << "Query with the same ID " << query_id << " exists";
-      return false;
-    }
-  } else {
+  if (admitted_queries_.size() >= PolicyEnforcerBase::kMaxConcurrentQueries) {
     // This query will have to wait.
     waiting_queries_.push(query_handle);
     return false;
   }
+
+  const std::size_t query_id = query_handle->query_id();
+  if (admitted_queries_.find(query_id) != admitted_queries_.end()) {
+    LOG(ERROR) << "Query with the same ID " << query_id << " exists";
+    return false;
+  }
+
+  // Ok to admit the query.
+  // NOTE(zuyu): Should call before constructing a 'QueryManager'.
+  // Otherwise, an InitiateRebuildMessage may be sent before 'QueryContext'
+  // initializes.
+  initiateQueryInShiftboss(query_handle);
+
+  // Query with the same ID not present, ok to admit.
+  admitted_queries_[query_id].reset(
+      new QueryManagerDistributed(query_handle, shiftboss_directory_, foreman_client_id_, bus_));
+  return true;
 }
 
 void PolicyEnforcerDistributed::processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message) {
@@ -144,18 +151,6 @@ void PolicyEnforcerDistributed::processInitiateRebuildResponseMessage(const tmb:
   query_manager->processInitiateRebuildResponseMessage(
       proto.operator_index(), num_rebuild_work_orders, shiftboss_index);
   shiftboss_directory_->addNumQueuedWorkOrders(shiftboss_index, num_rebuild_work_orders);
-
-  if (query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
-    onQueryCompletion(query_manager);
-
-    removeQuery(query_id);
-    if (!waiting_queries_.empty()) {
-      // Admit the earliest waiting query.
-      QueryHandle *new_query = waiting_queries_.front();
-      waiting_queries_.pop();
-      admitQuery(new_query);
-    }
-  }
 }
 
 void PolicyEnforcerDistributed::getShiftbossIndexForAggregation(


[38/50] [abbrv] incubator-quickstep git commit: Fixed the dangling reference bug in CreateIndexOperator.

Posted by ji...@apache.org.
Fixed the dangling reference bug in CreateIndexOperator.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 266b9b9a96d94a4461eb451cb214ea545f3e9415
Parents: 3011ddf
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 17:44:00 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 17:44:00 2017 -0800

----------------------------------------------------------------------
 relational_operators/CreateIndexOperator.hpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/266b9b9a/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index fa992c9..0286e1c 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -94,7 +94,7 @@ class CreateIndexOperator : public RelationalOperator {
 
  private:
   CatalogRelation *relation_;
-  const std::string &index_name_;
+  const std::string index_name_;
   IndexSubBlockDescription index_description_;
 
   DISALLOW_COPY_AND_ASSIGN(CreateIndexOperator);


[18/50] [abbrv] incubator-quickstep git commit: A workaround to remove query result relation in the distributed version.

Posted by ji...@apache.org.
A workaround to remove query result relation in 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/aef1c358
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/aef1c358
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/aef1c358

Branch: refs/heads/LIP-time-decomposition
Commit: aef1c3586580cfa72eb031fafe08700f6d5d9a86
Parents: 27a8055
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Feb 7 00:41:45 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Feb 7 00:41:45 2017 -0800

----------------------------------------------------------------------
 query_execution/ForemanDistributed.cpp | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aef1c358/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index 4d95f16..e6f22ec 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -201,8 +201,12 @@ void ForemanDistributed::run() {
 
         // TODO(quickstep-team): Dynamically scale-up/down Shiftbosses.
         if (query_result_saved_shiftbosses_[query_id].size() == shiftboss_directory_.size()) {
-          processSaveQueryResultResponseMessage(proto.cli_id(), proto.relation_id());
+          const relation_id result_relation_id = proto.relation_id();
+          processSaveQueryResultResponseMessage(proto.cli_id(), result_relation_id);
           query_result_saved_shiftbosses_.erase(query_id);
+
+          // TODO(zuyu): Refactor to clean-up blocks in Shiftbosses.
+          catalog_database_->dropRelationById(result_relation_id);
         }
         break;
       }


[03/50] [abbrv] incubator-quickstep git commit: Minor refactor for HashJoinInnerJoin.

Posted by ji...@apache.org.
Minor refactor for HashJoinInnerJoin.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 23e14b8e078f42a8d3e5f6c0c4885dee271d99aa
Parents: f2e7726
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Jan 30 15:28:49 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jan 30 20:21:23 2017 -0800

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt       |  1 +
 relational_operators/HashJoinOperator.cpp | 42 ++++++++++++++------------
 2 files changed, 23 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/23e14b8e/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c1caaa3..b2e08cf 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -199,6 +199,7 @@ target_link_libraries(quickstep_relationaloperators_FinalizeAggregationOperator
 target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       ${GFLAGS_LIB_NAME}
                       glog
+                      quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/23e14b8e/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index fd3841f..7394554 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -25,6 +25,7 @@
 #include <utility>
 #include <vector>
 
+#include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
@@ -165,6 +166,12 @@ class OuterJoinTupleCollector {
   TupleIdSequence *filter_;
 };
 
+// For InnerJoin.
+constexpr std::size_t kNumValueAccessors = 3u;
+constexpr std::size_t kBuildValueAccessorIndex = 0,
+                      kProbeValueAccessorIndex = 1u,
+                      kTempResultValueAccessorIndex = 2u;
+
 }  // namespace
 
 bool HashJoinOperator::getAllWorkOrders(
@@ -565,31 +572,27 @@ void HashInnerJoinWorkOrder::execute() {
         });
     }
 
-
     // We also need a temp value accessor to store results of any scalar expressions.
     ColumnVectorsValueAccessor temp_result;
 
     // Create a map of ValueAccessors and what attributes we want to pick from them
-    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map;
-    const std::vector<ValueAccessor *> accessors{
-        ordered_build_accessor.get(), ordered_probe_accessor.get(), &temp_result};
-    const unsigned int build_index = 0, probe_index = 1, temp_index = 2;
-    for (auto &accessor : accessors) {
-      accessor_attribute_map.push_back(std::make_pair(
-          accessor,
-          std::vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
-    }
+    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map(
+        kNumValueAccessors, std::make_pair(nullptr,  // A late binding ValueAccessor.
+                                           vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
 
-    attribute_id dest_attr = 0;
-    std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
+    accessor_attribute_map[kBuildValueAccessorIndex].first = ordered_build_accessor.get();
+    accessor_attribute_map[kProbeValueAccessorIndex].first = ordered_probe_accessor.get();
+    accessor_attribute_map[kTempResultValueAccessorIndex].first = &temp_result;
 
+    attribute_id dest_attr = 0;
     for (auto &selection_cit : selection_) {
       // If the Scalar (column) is not an attribute in build/probe blocks, then
       // insert it into a ColumnVectorsValueAccessor.
       if (selection_cit->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
         // Current destination attribute maps to the column we'll create now.
-        accessor_attribute_map[temp_index].second[dest_attr] = temp_result.getNumColumns();
+        accessor_attribute_map[kTempResultValueAccessorIndex].second[dest_attr] = temp_result.getNumColumns();
 
+        std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
         if (temp_result.getNumColumns() == 0) {
           // The getAllValuesForJoin function below needs joined tuple IDs as
           // a vector of pair of (build-tuple-ID, probe-tuple-ID), and we have
@@ -599,9 +602,8 @@ void HashInnerJoinWorkOrder::execute() {
           // they don't have scalar expressions with attributes from both
           // build and probe relations (other expressions would have been
           // pushed down to before the join).
-          zipped_joined_tuple_ids.reserve(build_tids.size());
           for (std::size_t i = 0; i < build_tids.size(); ++i) {
-            zipped_joined_tuple_ids.push_back(std::make_pair(build_tids[i], probe_tids[i]));
+            zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
           }
         }
         temp_result.addColumn(
@@ -610,12 +612,12 @@ void HashInnerJoinWorkOrder::execute() {
                                       probe_relation_id, probe_accessor.get(),
                                       zipped_joined_tuple_ids));
       } else {
-        auto scalar_attr = static_cast<const ScalarAttribute *>(selection_cit.get());
-        const attribute_id attr_id = scalar_attr->getAttribute().getID();
-        if (scalar_attr->getAttribute().getParent().getID() == build_relation_id) {
-          accessor_attribute_map[build_index].second[dest_attr] = attr_id;
+        const CatalogAttribute &attr = static_cast<const ScalarAttribute *>(selection_cit.get())->getAttribute();
+        const attribute_id attr_id = attr.getID();
+        if (attr.getParent().getID() == build_relation_id) {
+          accessor_attribute_map[kBuildValueAccessorIndex].second[dest_attr] = attr_id;
         } else {
-          accessor_attribute_map[probe_index].second[dest_attr] = attr_id;
+          accessor_attribute_map[kProbeValueAccessorIndex].second[dest_attr] = attr_id;
         }
       }
       ++dest_attr;


[49/50] [abbrv] incubator-quickstep git commit: Adds marcs ssh key to KEYS

Posted by ji...@apache.org.
Adds marcs ssh key to KEYS


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

Branch: refs/heads/LIP-time-decomposition
Commit: f6480fb59a2f3be6d37274f2a2376e733ba98d22
Parents: ab46d78
Author: cramja <ma...@gmail.com>
Authored: Tue Feb 14 17:02:20 2017 -0600
Committer: cramja <ma...@gmail.com>
Committed: Tue Feb 14 17:02:20 2017 -0600

----------------------------------------------------------------------
 KEYS | 52 ++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 52 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f6480fb5/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
index ce3964f..a10f790 100644
--- a/KEYS
+++ b/KEYS
@@ -49,3 +49,55 @@ iYXZdNoVw3RZC2XRQB+as9wYnz/Ziqbrrw58/E5FdmC8U8+Fa/0lTUh6VsPjpu5u
 E7agqOm2ReVbNLPyHa2oGftKu0Cwyghbys5xNxqbNPQnFR9N9Soi+0n4IGCZ/tj5
 =qv6a
 -----END PGP PUBLIC KEY BLOCK-----
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v2
+
+mQINBFijfvUBEACtbFV+BN1MRgP5s1/W1cKHbXsjBz8C+xlblRSjewrdlrHJEiu7
++qP3DkaJse8odnUbuveYuVyQQ+gQq8Qn+w8CzRBqnFn4V2xGKouQhlZ9Q+QhNzI/
+Y2+2XfDSY8GAxV3e5BnLCAnRhQJ4UX3aIU1ZmdVJ8Hag3Pfv5BKf/DpgEfPJ+G0z
+d4oy9gN96Zjlui/QwMaF4wCCE8P5tXm04QXyK+bs4pV9WsjMf5LW/gBZAgof2Kfp
+rEBM2aAjSmsfysNLXB/aakyjdEMtUV7wmtHe+mhbo8JmTNrhXsIhNbzXIeq2lRLL
+dNzUn5WQ03F69sx1tBXG5Ro8xA0TEjzigsy/4RlIgplws8rR3gP3H9RSlXNvifoC
+ZCHNxFdXHJ908q6SFfCCZR8eUd5mbG3mB04HEXccOl3E0456U0Aee2DK4uGp/RVN
+b4pvZgLRkGcymy4hg+QR56ixq+ovRi75m9zcGe3wvYZNoAL5IscqHvQpn2iCTBuJ
+ZbTOENKTfzncZZhApgzLZhdnAkEsyyhY8qOJnS+0TERSXXyoR1nHpVgnLwwCS7Oi
+rXhQR+YqA3gwcuyukwMSy25+VJLAtls0sH7wi6EWtWE/4mytScQV/AFTLqpA7wYY
+X5ceEdIqp0YwBi8HRrd1sDyRWYHTKLLUS9jgzZzgI54UexggXxn/h46DAwARAQAB
+tCpNYXJjIFNwZWhsbWFubiAoYXBhY2hlKSA8c3BlaGxAYXBhY2hlLm9yZz6JAjcE
+EwEKACEFAlijfvUCGwMFCwkIBwMFFQoJCAsFFgIDAQACHgECF4AACgkQ+r4yMlop
+iZqLDw//Q3MQX6xj1kSAxEHWbKziW49oMQylBwlLd9G0Zvo4VUd4Kgkl/sTF1nRn
+a3gp0SFI68z4I53DjBy5gzMQLhUArj6ygoTIFEqmaqnqXdwbYitUuexFmU7QuffU
+eJGrfbnCR+412SuSbs/yUIiqtzZQMMyX9eHRDKa03bRmXiKOvH4yWoQHmhNSI9jx
+cm0NJA7UirmtZ7Od3U+yEJCAje9RockzIyzY9Wgmmanxw14ig6RvIebBiEUeiyM1
+S5YsQ4r3SjZQyK2IlStDQbIrw/npg7SMpi/1QAgNBtyNuIJN4ehUukVwBzH75kK9
+d8R2tfn4ECNdjyR7zxGTu+C6Xy7i5nMyVDlU5KG+3tTwFh4RmxcuJrR6rNjnBtj+
+GUJHselHEcaDRW3x4w1kIIBqdiCo8JP8F1p786lZiCSYWeHHrXNQlwKnmtoyzn7x
+RuHjY6N6bY/HuJO0DVKW1L7ModtqpANe4e1sJbBmkJcrs9Cybiyfo/uGHG+tirKp
+9+dvoe6PFynA2TpwIluJHni0YNA9ipZHATZciFYo7+/XNXHei1YFYWObgdYTyGKk
+txTpcSsnJiDsNc1QEQ5DqxgeM4BV1/3wn64M83jI+6KkL+FWTKPemAg+OvL/Cmf7
+B/NBXi/QNvhwfSfD09naAnosjeG23Ggq1uOmOawoL7D/Y38+/LG5Ag0EWKN+9QEQ
+AJ01B8e5B6A+h+qS/ttGqZr1KKE0/yE6HNvxaxPo7GIJftqFdLnnZ786GUEecBV0
+XD+bejYv4bMEnlzv9UpQRJqY2npsnjf1UqlK2ODYyn0bD3LZ1nooRRGfUyiacPLy
+7KXwgweIuuwgS/tdTh2c+v9wYx4fmOlb4ZV7cmlyx4s3IJW3b0EhejsUBsIoFRRS
+59x+QlLQMlyPhLEvAUo+OsWaa4Wotjbxfv3VrK+0ZTEGHteL/CoQ+xZI1friLbKY
+OZZJiQTBQQaW6ojp6vtxYHNBrWIVuM41skKSwvHx7tKLJu03EY3/xdgCoKZ5+KTX
+NSPlfzYmWAAqVDGiH76WEHUQtp3E3Er/iloZM4dmBsi7So1yahk2dbCnfpjBHPMi
+a0t81RbSdAURzC+6tCNianUy/COed4+FBPDTHaekf1qtrBJNTV6iTsECf9uvp6qD
+QGDpah4/swKLFFqxClEAW+VaeWB+9nP6yZUGh7gp5SAMoAa5vSC8g8FzuTIsg/MI
+a0GhkHVopuILqzDSlhlJeh1cGvx1Hm2j0MtywMctJ0OXAO+oJcH0cx+JtJz3IFXS
+SwipTc9ySSk9bMOVhuQyouziU1UWW/sPSUvtKX4vV4pAaTL0ERzNH8gRsCf8/9Cu
+XGqJMlrVwZ+yFUaImiseAkzpBXlGYjgQBtd6e2/A8qsVABEBAAGJAh4EGAEKAAkF
+AlijfvUCGwwACgkQ+r4yMlopiZoW+g/1F8nTwnf+l3VcIUMQRwNe8X7/RZ5BrQnN
+oWZ9Nw4Fc40L90l/KZSp/k2pYo6KqRYlcHsYyMdjAuv+hSkPckwnS11wsrwG8CxQ
+1JuW5P9TQUpyN+367/AEOwc4qwJMo1RtLpjdqy3ssl1Rv25BJovzbd1ag7LF+RlM
+2o8o05t0gPkMa8bYu05XBTISKCT4qtAn8RrrUjvDNP7hsGfRUsN2FGIcvRu5Rt7t
+Uwt2JBjZ0+xmOt6f7ytzw6vNuItALDtzP1geXAHM+BddOXut9AnjVuEK+v92duPW
+LHepFB9zyNN5AoENGQvzl413lF9EuXOoiEXmtg+vRyKeEmJTtwzL3FTahvLg0pYm
+7h1InRxXtVilPPn3tcjNIvfpcHNUXwj3YstT380zBYAKxovG9rmHR8sKwgSNiK4N
+gbnVVBcIYQ/ogsdxqr6yjxqrWpeuph9BCeroxxDLMaFq260KyMIPsB34F5g3L1v5
+NM0GzLhrr5xa3B7hWvDmqxMIMsRtlf+37NqV0ylc6n4ZHJa1U60jSJx2BN0jnaMe
+LriD7AMhIAI4Ew70dYSHeVtzDO2MD3GTFqdHPuVCRi3322NSSvNjjZleCNE0q3Ca
+N5xnL7HQ8wj2da/uvr6A0oX46nYMBPwDdP5nkvHmNbMPLF5b2IOeoijL48x+EVQS
+U/KHHqP5JA==
+=mW4v
+-----END PGP PUBLIC KEY BLOCK-----


[09/50] [abbrv] incubator-quickstep git commit: Add BitVectorExactFilter as a LIP filter and supports Join-to-Semijoin transformation.

Posted by ji...@apache.org.
Add BitVectorExactFilter as a LIP filter and supports Join-to-Semijoin transformation.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 4ba819c5b82af1d9284525bd7a16784e0254be3f
Parents: 5ffdaaf
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Oct 27 14:16:32 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Tue Jan 31 16:57:09 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   4 +
 query_optimizer/ExecutionGenerator.cpp          |  62 +++
 query_optimizer/ExecutionGenerator.hpp          |   8 +
 query_optimizer/LIPFilterGenerator.cpp          | 109 +++--
 query_optimizer/LIPFilterGenerator.hpp          |  49 ++-
 query_optimizer/PhysicalGenerator.cpp           |  19 +-
 query_optimizer/cost_model/CMakeLists.txt       |   5 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |   9 +
 query_optimizer/cost_model/SimpleCostModel.hpp  |   5 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    | 163 ++++++-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |  83 ++++
 query_optimizer/expressions/ExpressionUtil.hpp  |   8 +-
 query_optimizer/physical/CMakeLists.txt         |  14 +
 query_optimizer/physical/FilterJoin.cpp         | 115 +++++
 query_optimizer/physical/FilterJoin.hpp         | 187 ++++++++
 .../physical/LIPFilterConfiguration.hpp         | 265 ++++++++---
 query_optimizer/physical/PatternMatcher.hpp     |   2 +
 query_optimizer/physical/PhysicalType.hpp       |   1 +
 query_optimizer/physical/TopLevelPlan.hpp       |   3 +-
 query_optimizer/rules/AttachLIPFilters.cpp      |  28 +-
 query_optimizer/rules/CMakeLists.txt            |  22 +
 query_optimizer/rules/InjectJoinFilters.cpp     | 438 +++++++++++++++++++
 query_optimizer/rules/InjectJoinFilters.hpp     | 116 +++++
 query_optimizer/tests/OptimizerTextTest.cpp     |   2 +
 relational_operators/BuildLIPFilterOperator.cpp | 154 +++++++
 relational_operators/BuildLIPFilterOperator.hpp | 200 +++++++++
 relational_operators/CMakeLists.txt             |  24 +
 relational_operators/WorkOrder.proto            |  49 ++-
 relational_operators/WorkOrderFactory.cpp       |  45 ++
 utility/CMakeLists.txt                          |   1 +
 utility/PlanVisualizer.cpp                      |  24 +-
 utility/lip_filter/BitVectorExactFilter.hpp     | 202 +++++++++
 utility/lip_filter/CMakeLists.txt               |  11 +
 utility/lip_filter/LIPFilter.hpp                |   2 +-
 utility/lip_filter/LIPFilter.proto              |  25 +-
 utility/lip_filter/LIPFilterDeployment.cpp      |  72 +--
 utility/lip_filter/LIPFilterDeployment.hpp      |  28 +-
 utility/lip_filter/LIPFilterFactory.cpp         |  50 +++
 38 files changed, 2394 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 0ca971d..7f90e11 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -96,6 +96,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_CreateTable
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple
@@ -115,6 +116,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_relationaloperators_AggregationOperator
                       quickstep_relationaloperators_BuildHashOperator
+                      quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_CreateIndexOperator
                       quickstep_relationaloperators_CreateTableOperator
                       quickstep_relationaloperators_DeleteOperator
@@ -161,6 +163,7 @@ target_link_libraries(quickstep_queryoptimizer_LIPFilterGenerator
                       quickstep_queryoptimizer_QueryPlan
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_LIPFilterConfiguration
                       quickstep_queryoptimizer_physical_Physical
@@ -206,6 +209,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
+                      quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_ReorderColumns

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index e25b8ad..ce1452e 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -76,6 +76,7 @@
 #include "query_optimizer/physical/CreateTable.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
@@ -95,6 +96,7 @@
 #include "query_optimizer/physical/WindowAggregate.hpp"
 #include "relational_operators/AggregationOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
+#include "relational_operators/BuildLIPFilterOperator.hpp"
 #include "relational_operators/CreateIndexOperator.hpp"
 #include "relational_operators/CreateTableOperator.hpp"
 #include "relational_operators/DeleteOperator.hpp"
@@ -271,6 +273,9 @@ void ExecutionGenerator::generatePlanInternal(
     case P::PhysicalType::kDropTable:
       return convertDropTable(
           std::static_pointer_cast<const P::DropTable>(physical_plan));
+    case P::PhysicalType::kFilterJoin:
+      return convertFilterJoin(
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan));
     case P::PhysicalType::kHashJoin:
       return convertHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
@@ -608,6 +613,63 @@ void ExecutionGenerator::convertSharedSubplanReference(const physical::SharedSub
   }
 }
 
+void ExecutionGenerator::convertFilterJoin(const P::FilterJoinPtr &physical_plan) {
+  P::PhysicalPtr probe_physical = physical_plan->left();
+  P::PhysicalPtr build_physical = physical_plan->right();
+
+  // Let B denote the build side child. If B is also a FilterJoin, then the
+  // actual "concrete" input relation is B's probe side child, and B's build
+  // side becomes a LIPFilter that is attached to the BuildLIPFilterOperator
+  // created below.
+  P::FilterJoinPtr filter_join;
+  if (P::SomeFilterJoin::MatchesWithConditionalCast(build_physical, &filter_join)) {
+    build_physical = filter_join->left();
+    DCHECK(build_physical->getPhysicalType() != P::PhysicalType::kFilterJoin);
+  }
+
+  // Convert the predicate proto.
+  QueryContext::predicate_id build_side_predicate_index = QueryContext::kInvalidPredicateId;
+  if (physical_plan->build_side_filter_predicate()) {
+    build_side_predicate_index = query_context_proto_->predicates_size();
+
+    std::unique_ptr<const Predicate> build_side_predicate(
+        convertPredicate(physical_plan->build_side_filter_predicate()));
+    query_context_proto_->add_predicates()->CopyFrom(build_side_predicate->getProto());
+  }
+
+  const CatalogRelationInfo *probe_relation_info =
+      findRelationInfoOutputByPhysical(probe_physical);
+  const CatalogRelationInfo *build_relation_info =
+      findRelationInfoOutputByPhysical(build_physical);
+
+  // Create a BuildLIPFilterOperator for the FilterJoin. This operator builds
+  // LIP filters that are applied properly in downstream operators to achieve
+  // the filter-join semantics.
+  const QueryPlan::DAGNodeIndex build_filter_operator_index =
+      execution_plan_->addRelationalOperator(
+          new BuildLIPFilterOperator(
+              query_handle_->query_id(),
+              *build_relation_info->relation,
+              build_side_predicate_index,
+              build_relation_info->isStoredRelation()));
+
+  if (!build_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(build_filter_operator_index,
+                                         build_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+
+  physical_to_output_relation_map_.emplace(
+      std::piecewise_construct,
+      std::forward_as_tuple(physical_plan),
+      std::forward_as_tuple(probe_relation_info->producer_operator_index,
+                            probe_relation_info->relation));
+
+  DCHECK(lip_filter_generator_ != nullptr);
+  lip_filter_generator_->addFilterJoinInfo(physical_plan,
+                                           build_filter_operator_index);
+}
+
 void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   // HashJoin is converted to three operators:
   //     BuildHash, HashJoin, DestroyHash. The second is the primary operator.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 55197c9..eba6eee 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -46,6 +46,7 @@
 #include "query_optimizer/physical/CreateTable.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
@@ -248,6 +249,13 @@ class ExecutionGenerator {
   void convertSharedSubplanReference(const physical::SharedSubplanReferencePtr &physical_plan);
 
   /**
+   * @brief Converts a FilterJoin to a BuildLIPFilter operator.
+   *
+   * @param physical_plan The FilterJoin to be converted.
+   */
+  void convertFilterJoin(const physical::FilterJoinPtr &physical_plan);
+
+  /**
    * @brief Converts a HashJoin to BuildHash, HashJoin and
    *        DestroyHash operators.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/LIPFilterGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LIPFilterGenerator.cpp b/query_optimizer/LIPFilterGenerator.cpp
index 404037e..2ce2ea8 100644
--- a/query_optimizer/LIPFilterGenerator.cpp
+++ b/query_optimizer/LIPFilterGenerator.cpp
@@ -20,11 +20,13 @@
 #include "query_optimizer/LIPFilterGenerator.hpp"
 
 #include <map>
+#include <memory>
 #include <utility>
 #include <vector>
 
 #include "catalog/CatalogAttribute.hpp"
 #include "query_execution/QueryContext.pb.h"
+#include "query_optimizer/physical/LIPFilterConfiguration.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "types/Type.hpp"
 #include "utility/lip_filter/LIPFilter.hpp"
@@ -47,7 +49,7 @@ void LIPFilterGenerator::registerAttributeMap(
   if (build_it != build_info_map.end()) {
     auto &map_entry = attribute_map_[node];
     for (const auto &info : build_it->second) {
-      E::ExprId attr_id = info.build_attribute->id();
+      E::ExprId attr_id = info->build_attribute()->id();
       map_entry.emplace(attr_id, attribute_substitution_map.at(attr_id));
     }
   }
@@ -57,15 +59,16 @@ void LIPFilterGenerator::registerAttributeMap(
   if (probe_it != probe_info_map.end()) {
     auto &map_entry = attribute_map_[node];
     for (const auto &info : probe_it->second) {
-      E::ExprId attr_id = info.probe_attribute->id();
+      E::ExprId attr_id = info->probe_attribute()->id();
       map_entry.emplace(attr_id, attribute_substitution_map.at(attr_id));
     }
   }
 }
 
 void LIPFilterGenerator::deployLIPFilters(QueryPlan *execution_plan,
-                                          serialization::QueryContext *query_context_proto) const {
-  LIPFilterBuilderMap lip_filter_builder_map;
+                                          serialization::QueryContext *query_context_proto) {
+  lip_filter_builder_map_.clear();
+  lip_filter_deployment_protos_.clear();
 
   // Deploy builders
   const auto &build_info_map = lip_filter_configuration_->getBuildInfoMap();
@@ -76,8 +79,7 @@ void LIPFilterGenerator::deployLIPFilters(QueryPlan *execution_plan,
                             query_context_proto,
                             info.builder_node,
                             info.builder_operator_index,
-                            build_it->second,
-                            &lip_filter_builder_map);
+                            build_it->second);
     }
   }
 
@@ -90,10 +92,16 @@ void LIPFilterGenerator::deployLIPFilters(QueryPlan *execution_plan,
                           query_context_proto,
                           info.prober_node,
                           info.prober_operator_index,
-                          probe_it->second,
-                          lip_filter_builder_map);
+                          probe_it->second);
     }
   }
+
+  // Attach LIPFilterDeployment information to the RelationalOperators.
+  for (const auto &entry : lip_filter_deployment_protos_) {
+    RelationalOperator *relop =
+        execution_plan->getQueryPlanDAGMutable()->getNodePayloadMutable(entry.first);
+    relop->deployLIPFilters(entry.second.first);
+  }
 }
 
 void LIPFilterGenerator::deployBuilderInternal(
@@ -101,30 +109,46 @@ void LIPFilterGenerator::deployBuilderInternal(
     serialization::QueryContext *query_context_proto,
     const physical::PhysicalPtr &builder_node,
     const QueryPlan::DAGNodeIndex builder_operator_index,
-    const std::vector<physical::LIPFilterBuildInfo> &build_info_vec,
-    LIPFilterBuilderMap *lip_filter_builder_map) const {
-  const auto lip_deployment_index = query_context_proto->lip_filter_deployments_size();
+    const std::vector<physical::LIPFilterBuildInfoPtr> &build_info_vec) {
   auto *lip_filter_deployment_info_proto =
-      query_context_proto->add_lip_filter_deployments();
-  lip_filter_deployment_info_proto->set_action_type(serialization::LIPFilterActionType::BUILD);
+      getLIPFilterDeploymentProto(builder_operator_index, query_context_proto);
 
   const auto &builder_attribute_map = attribute_map_.at(builder_node);
   for (const auto &info : build_info_vec) {
     // Add the LIPFilter information into query context.
     const QueryContext::lip_filter_id lip_filter_id = query_context_proto->lip_filters_size();
     serialization::LIPFilter *lip_filter_proto = query_context_proto->add_lip_filters();
-    const CatalogAttribute *target_attr = builder_attribute_map.at(info.build_attribute->id());
+    const CatalogAttribute *target_attr =
+        builder_attribute_map.at(info->build_attribute()->id());
     const Type &attr_type = target_attr->getType();
 
-    switch (info.filter_type) {
+    switch (info->filter_type()) {
       case LIPFilterType::kSingleIdentityHashFilter: {
         DCHECK(!attr_type.isVariableLength());
+        const P::SingleIdentityHashFilterBuildInfo &sihf_info =
+            *std::static_pointer_cast<const P::SingleIdentityHashFilterBuildInfo>(info);
         lip_filter_proto->set_lip_filter_type(
             serialization::LIPFilterType::SINGLE_IDENTITY_HASH_FILTER);
-        lip_filter_proto->SetExtension(
-            serialization::SingleIdentityHashFilter::filter_cardinality, info.filter_cardinality);
-        lip_filter_proto->SetExtension(
-            serialization::SingleIdentityHashFilter::attribute_size, attr_type.minimumByteLength());
+        lip_filter_proto->SetExtension(serialization::SingleIdentityHashFilter::filter_cardinality,
+                                       sihf_info.filter_cardinality());
+        lip_filter_proto->SetExtension(serialization::SingleIdentityHashFilter::attribute_size,
+                                       attr_type.minimumByteLength());
+        break;
+      }
+      case LIPFilterType::kBitVectorExactFilter: {
+        DCHECK(!attr_type.isVariableLength());
+        const P::BitVectorExactFilterBuildInfo &bvef_info =
+            *std::static_pointer_cast<const P::BitVectorExactFilterBuildInfo>(info);
+        lip_filter_proto->set_lip_filter_type(
+            serialization::LIPFilterType::BIT_VECTOR_EXACT_FILTER);
+        lip_filter_proto->SetExtension(serialization::BitVectorExactFilter::min_value,
+                                       bvef_info.min_value());
+        lip_filter_proto->SetExtension(serialization::BitVectorExactFilter::max_value,
+                                       bvef_info.max_value());
+        lip_filter_proto->SetExtension(serialization::BitVectorExactFilter::attribute_size,
+                                       attr_type.minimumByteLength());
+        lip_filter_proto->SetExtension(serialization::BitVectorExactFilter::is_anti_filter,
+                                       bvef_info.is_anti_filter());
         break;
       }
       default:
@@ -133,21 +157,16 @@ void LIPFilterGenerator::deployBuilderInternal(
     }
 
     // Register the builder information which is needed later by the probers.
-    lip_filter_builder_map->emplace(
-        std::make_pair(info.build_attribute->id(), builder_node),
+    lip_filter_builder_map_.emplace(
+        std::make_pair(info->build_attribute()->id(), builder_node),
         std::make_pair(lip_filter_id, builder_operator_index));
 
     // Add the builder deployment information into query context.
-    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_entries();
+    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_build_entries();
     lip_filter_entry_proto->set_lip_filter_id(lip_filter_id);
     lip_filter_entry_proto->set_attribute_id(target_attr->getID());
     lip_filter_entry_proto->mutable_attribute_type()->CopyFrom(attr_type.getProto());
   }
-
-  // Attach the LIPFilterDeployment information to the RelationalOperator.
-  RelationalOperator *relop =
-      execution_plan->getQueryPlanDAGMutable()->getNodePayloadMutable(builder_operator_index);
-  relop->deployLIPFilters(lip_deployment_index);
 }
 
 void LIPFilterGenerator::deployProberInteral(
@@ -155,23 +174,21 @@ void LIPFilterGenerator::deployProberInteral(
     serialization::QueryContext *query_context_proto,
     const physical::PhysicalPtr &prober_node,
     const QueryPlan::DAGNodeIndex prober_operator_index,
-    const std::vector<physical::LIPFilterProbeInfo> &probe_info_vec,
-    const LIPFilterBuilderMap &lip_filter_builder_map) const {
-  const auto lip_deployment_index = query_context_proto->lip_filter_deployments_size();
+    const std::vector<physical::LIPFilterProbeInfoPtr> &probe_info_vec) {
   auto *lip_filter_deployment_info_proto =
-      query_context_proto->add_lip_filter_deployments();
-  lip_filter_deployment_info_proto->set_action_type(serialization::LIPFilterActionType::PROBE);
+      getLIPFilterDeploymentProto(prober_operator_index, query_context_proto);
 
   const auto &prober_attribute_map = attribute_map_.at(prober_node);
   for (const auto &info : probe_info_vec) {
     // Find the corresponding builder for the to-be-probed LIPFilter.
     const auto &builder_info =
-        lip_filter_builder_map.at(
-            std::make_pair(info.build_attribute->id(), info.builder));
-    const CatalogAttribute *target_attr = prober_attribute_map.at(info.probe_attribute->id());
+        lip_filter_builder_map_.at(
+            std::make_pair(info->build_attribute()->id(), info->builder()));
+    const CatalogAttribute *target_attr =
+        prober_attribute_map.at(info->probe_attribute()->id());
 
     // Add the prober deployment information into query context.
-    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_entries();
+    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_probe_entries();
     lip_filter_entry_proto->set_lip_filter_id(builder_info.first);
     lip_filter_entry_proto->set_attribute_id(target_attr->getID());
     lip_filter_entry_proto->mutable_attribute_type()->CopyFrom(
@@ -183,11 +200,23 @@ void LIPFilterGenerator::deployProberInteral(
                                                  builder_info.second,
                                                  true /* is_pipeline_breaker */);
   }
+}
 
-  // Attach the LIPFilterDeployment information to the RelationalOperator.
-  RelationalOperator *relop =
-      execution_plan->getQueryPlanDAGMutable()->getNodePayloadMutable(prober_operator_index);
-  relop->deployLIPFilters(lip_deployment_index);
+serialization::LIPFilterDeployment* LIPFilterGenerator::getLIPFilterDeploymentProto(
+    const QueryPlan::DAGNodeIndex op_index,
+    serialization::QueryContext *query_context_proto) {
+  const auto proto_it = lip_filter_deployment_protos_.find(op_index);
+  if (proto_it == lip_filter_deployment_protos_.end()) {
+    const int lip_deployment_index =
+        query_context_proto->lip_filter_deployments_size();
+    auto *lip_filter_deployment_proto =
+        query_context_proto->add_lip_filter_deployments();
+    lip_filter_deployment_protos_.emplace(
+        op_index, std::make_pair(lip_deployment_index, lip_filter_deployment_proto));
+    return lip_filter_deployment_proto;
+  } else {
+    return proto_it->second.second;
+  }
 }
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/LIPFilterGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LIPFilterGenerator.hpp b/query_optimizer/LIPFilterGenerator.hpp
index 9d191a1..750499d 100644
--- a/query_optimizer/LIPFilterGenerator.hpp
+++ b/query_optimizer/LIPFilterGenerator.hpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/physical/LIPFilterConfiguration.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/Selection.hpp"
@@ -39,7 +40,12 @@
 
 namespace quickstep {
 
-namespace serialization { class QueryContext; }
+namespace serialization {
+
+class QueryContext;
+class LIPFilterDeployment;
+
+}
 
 class CatalogAttribute;
 
@@ -93,6 +99,20 @@ class LIPFilterGenerator {
 
   /**
    * @brief Add physical-to-execution mapping information for deploying LIPFilters
+   *        to a FilterJoin node.
+   *
+   * @param filter_join A physical FilterJoin node.
+   * @param build_filter_operator_index The index of the BuildLIPFilterOperator
+   *        that corresponds to \p filter_join in the execution plan.
+   */
+  void addFilterJoinInfo(const physical::FilterJoinPtr &filter_join,
+                         const QueryPlan::DAGNodeIndex build_filter_operator_index) {
+    builder_infos_.emplace_back(filter_join, build_filter_operator_index);
+    prober_infos_.emplace_back(filter_join, build_filter_operator_index);
+  }
+
+  /**
+   * @brief Add physical-to-execution mapping information for deploying LIPFilters
    *        to a hash-join.
    *
    * @param hash_join A physical HashJoin node.
@@ -128,7 +148,7 @@ class LIPFilterGenerator {
    * @param query_context_proto QueryContext protobuf for the execution plan.
    */
   void deployLIPFilters(QueryPlan *execution_plan,
-                        serialization::QueryContext *query_context_proto) const;
+                        serialization::QueryContext *query_context_proto);
 
  private:
   /**
@@ -157,24 +177,21 @@ class LIPFilterGenerator {
     const QueryPlan::DAGNodeIndex prober_operator_index;
   };
 
-  // Maps each LIPFilter's building attribute to the LIPFilter's id in QueryContext
-  // as well as the LIPFilter's building relational operator's index.
-  typedef std::map<std::pair<expressions::ExprId, physical::PhysicalPtr>,
-                   std::pair<QueryContext::lip_filter_id, QueryPlan::DAGNodeIndex>> LIPFilterBuilderMap;
-
   void deployBuilderInternal(QueryPlan *execution_plan,
                              serialization::QueryContext *query_context_proto,
                              const physical::PhysicalPtr &builder_node,
                              const QueryPlan::DAGNodeIndex builder_operator_index,
-                             const std::vector<physical::LIPFilterBuildInfo> &build_info_vec,
-                             LIPFilterBuilderMap *lip_filter_builder_map) const;
+                             const std::vector<physical::LIPFilterBuildInfoPtr> &build_info_vec);
 
   void deployProberInteral(QueryPlan *execution_plan,
                            serialization::QueryContext *query_context_proto,
                            const physical::PhysicalPtr &prober_node,
                            const QueryPlan::DAGNodeIndex prober_operator_index,
-                           const std::vector<physical::LIPFilterProbeInfo> &probe_info_vec,
-                           const LIPFilterBuilderMap &lip_filter_builder_map) const;
+                           const std::vector<physical::LIPFilterProbeInfoPtr> &probe_info_vec);
+
+  serialization::LIPFilterDeployment* getLIPFilterDeploymentProto(
+      const QueryPlan::DAGNodeIndex op_index,
+      serialization::QueryContext *query_context_proto);
 
   const physical::LIPFilterConfigurationPtr lip_filter_configuration_;
 
@@ -183,6 +200,16 @@ class LIPFilterGenerator {
 
   std::map<physical::PhysicalPtr, std::map<expressions::ExprId, const CatalogAttribute *>> attribute_map_;
 
+  // Maps each LIPFilter's building attribute to the LIPFilter's id in QueryContext
+  // as well as the LIPFilter's building relational operator's index.
+  std::map<std::pair<expressions::ExprId, physical::PhysicalPtr>,
+           std::pair<QueryContext::lip_filter_id, QueryPlan::DAGNodeIndex>> lip_filter_builder_map_;
+
+  // Maps each relational operator's index to the attached LIPFilterDeployment's
+  // index and proto.
+  std::map<QueryPlan::DAGNodeIndex,
+           std::pair<int, serialization::LIPFilterDeployment *>> lip_filter_deployment_protos_;
+
   DISALLOW_COPY_AND_ASSIGN(LIPFilterGenerator);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index bd05267..5dc0ffb 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -27,6 +27,7 @@
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
+#include "query_optimizer/rules/InjectJoinFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
 #include "query_optimizer/rules/ReorderColumns.hpp"
@@ -56,6 +57,14 @@ DEFINE_bool(reorder_hash_joins, true,
             "cardinality and selective tables to be joined first, which is suitable "
             "for queries on star-schema tables.");
 
+DEFINE_bool(use_filter_joins, true,
+            "If true, apply an optimization that strength-reduces HashJoins to "
+            "FilterJoins (implemented as LIPFilters attached to some anchoring "
+            "operators. Briefly speaking, in the case that the join attribute has "
+            "consecutive integer values bounded in a reasonably small range, we "
+            "build a BitVector on the build-side attribute and use the BitVector "
+            "to filter the probe side table.");
+
 DEFINE_bool(use_lip_filters, true,
             "If true, use LIP (Lookahead Information Passing) filters to accelerate "
             "query processing. LIP filters are effective for queries on star schema "
@@ -133,9 +142,13 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     rules.emplace_back(new ReorderColumns());
   }
 
-  // NOTE(jianqiao): Adding rules after AttachLIPFilters requires extra handling
-  // of LIPFilterConfiguration for transformed nodes. So currently it is suggested
-  // that all the new rules be placed before this point.
+  // NOTE(jianqiao): Adding rules after InjectJoinFilters (or AttachLIPFilters) requires
+  // extra handling of LIPFilterConfiguration for transformed nodes. So currently it is
+  // suggested that all the new rules be placed before this point.
+  if (FLAGS_use_filter_joins) {
+    rules.emplace_back(new InjectJoinFilters());
+  }
+
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 90133e7..5f28bb3 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -33,6 +33,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
                       quickstep_catalog_CatalogRelationStatistics
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
                       quickstep_queryoptimizer_physical_Physical
@@ -49,6 +50,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationStatistics
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression
@@ -60,6 +62,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Predicate
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
                       quickstep_queryoptimizer_physical_PatternMatcher
@@ -72,6 +75,8 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_WindowAggregate
+                      quickstep_types_NullType
+                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 
 # Module all-in-one library:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index 7808898..e9d2e3a 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -27,6 +27,7 @@
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -61,6 +62,9 @@ std::size_t SimpleCostModel::estimateCardinality(
     case P::PhysicalType::kTableGenerator:
       return estimateCardinalityForTableGenerator(
           std::static_pointer_cast<const P::TableGenerator>(physical_plan));
+    case P::PhysicalType::kFilterJoin:
+      return estimateCardinalityForFilterJoin(
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan));
     case P::PhysicalType::kHashJoin:
       return estimateCardinalityForHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
@@ -119,6 +123,11 @@ std::size_t SimpleCostModel::estimateCardinalityForTableGenerator(
   return physical_plan->generator_function_handle()->getEstimatedCardinality();
 }
 
+std::size_t SimpleCostModel::estimateCardinalityForFilterJoin(
+    const P::FilterJoinPtr &physical_plan) {
+  return estimateCardinality(physical_plan->left());
+}
+
 std::size_t SimpleCostModel::estimateCardinalityForHashJoin(
     const P::HashJoinPtr &physical_plan) {
   return std::max(estimateCardinality(physical_plan->left()),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/cost_model/SimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.hpp b/query_optimizer/cost_model/SimpleCostModel.hpp
index 16366cd..4edc2fe 100644
--- a/query_optimizer/cost_model/SimpleCostModel.hpp
+++ b/query_optimizer/cost_model/SimpleCostModel.hpp
@@ -26,6 +26,7 @@
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/Selection.hpp"
@@ -80,6 +81,10 @@ class SimpleCostModel : public CostModel {
   std::size_t estimateCardinalityForSort(
       const physical::SortPtr &physical_plan);
 
+  // Returns the left child's cardinality
+  std::size_t estimateCardinalityForFilterJoin(
+      const physical::FilterJoinPtr &physical_plan);
+
   // Returns the larger value of the estimated cardinalities of two
   // input plans.
   std::size_t estimateCardinalityForHashJoin(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 75b1b2b..7afa1c3 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -21,11 +21,11 @@
 
 #include <algorithm>
 #include <memory>
-#include <unordered_map>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
@@ -38,6 +38,7 @@
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/PatternMatcher.hpp"
 #include "query_optimizer/physical/Physical.hpp"
@@ -48,6 +49,8 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "types/TypedValue.hpp"
+#include "types/NullType.hpp"
 
 #include "glog/logging.h"
 
@@ -73,6 +76,9 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinality(
     case P::PhysicalType::kTableGenerator:
       return estimateCardinalityForTableGenerator(
           std::static_pointer_cast<const P::TableGenerator>(physical_plan));
+    case P::PhysicalType::kFilterJoin:
+      return estimateCardinalityForFilterJoin(
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan));
     case P::PhysicalType::kHashJoin:
       return estimateCardinalityForHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
@@ -134,6 +140,17 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableGenerator(
   return physical_plan->generator_function_handle()->getEstimatedCardinality();
 }
 
+std::size_t StarSchemaSimpleCostModel::estimateCardinalityForFilterJoin(
+    const P::FilterJoinPtr &physical_plan) {
+  double build_side_filter_selectivity =
+      estimateSelectivityForPredicate(physical_plan->build_side_filter_predicate(),
+                                      physical_plan->right());
+  std::size_t left_cardinality = estimateCardinality(physical_plan->left());
+  double right_selectivity = estimateSelectivity(physical_plan->right());
+  return static_cast<std::size_t>(
+      left_cardinality * build_side_filter_selectivity * right_selectivity + 0.5);
+}
+
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForHashJoin(
     const P::HashJoinPtr &physical_plan) {
   std::size_t left_cardinality = estimateCardinality(physical_plan->left());
@@ -216,6 +233,18 @@ std::size_t StarSchemaSimpleCostModel::estimateNumDistinctValues(
       }
       break;
     }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan);
+      if (E::ContainsExprId(filter_join->left()->getOutputAttributes(), attribute_id)) {
+        std::size_t left_child_num_distinct_values =
+            estimateNumDistinctValues(attribute_id, filter_join->left());
+        double right_child_selectivity =
+            estimateSelectivity(filter_join->right());
+        return static_cast<std::size_t>(
+            left_child_num_distinct_values * right_child_selectivity + 0.5);
+      }
+    }
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
@@ -254,6 +283,16 @@ double StarSchemaSimpleCostModel::estimateSelectivity(
       double child_selectivity = estimateSelectivity(selection->input());
       return filter_selectivity * child_selectivity;
     }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan);
+      double left_selectivity = estimateSelectivity(filter_join->left());
+      double right_selectivity = estimateSelectivity(filter_join->right());
+      double build_side_filter_selectivity =
+          estimateSelectivityForPredicate(filter_join->build_side_filter_predicate(),
+                                          filter_join->right());
+      return left_selectivity * right_selectivity * build_side_filter_selectivity;
+    }
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
@@ -383,18 +422,124 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
 std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
     const E::ExprId attribute_id,
     const P::TableReferencePtr &table_reference) {
-  const CatalogRelation &relation = *table_reference->relation();
-  const std::vector<E::AttributeReferencePtr> &attributes = table_reference->attribute_list();
-  for (std::size_t i = 0; i < attributes.size(); ++i) {
-    if (attributes[i]->id() == attribute_id) {
-      const CatalogRelationStatistics &stat = relation.getStatistics();
-      if (stat.hasNumDistinctValues(i)) {
-        return stat.getNumDistinctValues(i);
+  const auto rel_attr_id =
+      findCatalogRelationAttributeId(table_reference, attribute_id);
+  if (rel_attr_id != kInvalidAttributeID) {
+    const CatalogRelationStatistics &stat =
+        table_reference->relation()->getStatistics();
+    if (stat.hasNumDistinctValues(rel_attr_id)) {
+      return stat.getNumDistinctValues(rel_attr_id);
+    }
+  }
+  return estimateCardinalityForTableReference(table_reference);
+}
+
+bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
+    const P::PhysicalPtr &physical_plan,
+    const std::vector<E::AttributeReferencePtr> &attributes) {
+  switch (physical_plan->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(physical_plan);
+      return E::SubsetOfExpressions(aggregate->grouping_expressions(), attributes);
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr &hash_join =
+          std::static_pointer_cast<const P::HashJoin>(physical_plan);
+      bool unique_from_left =
+          impliesUniqueAttributes(hash_join->right(), hash_join->right_join_attributes())
+              && impliesUniqueAttributes(hash_join->left(), attributes);
+      bool unique_from_right =
+          impliesUniqueAttributes(hash_join->left(), hash_join->left_join_attributes())
+              && impliesUniqueAttributes(hash_join->right(), attributes);
+      return unique_from_left || unique_from_right;
+    }
+    case P::PhysicalType::kTableReference: {
+      const P::TableReferencePtr &table_reference =
+          std::static_pointer_cast<const P::TableReference>(physical_plan);
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+      if (stat.hasNumTuples()) {
+        const std::size_t num_tuples = stat.getNumTuples();
+        for (const auto &attr : attributes) {
+          const attribute_id rel_attr_id =
+              findCatalogRelationAttributeId(table_reference, attr->id());
+          if (rel_attr_id != kInvalidAttributeID &&
+              stat.hasNumDistinctValues(rel_attr_id) &&
+              stat.getNumDistinctValues(rel_attr_id) == num_tuples) {
+            return true;
+          }
+        }
       }
+      return false;
+    }
+    case P::PhysicalType::kSample:  // Fall through
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kSort: {
+      DCHECK_EQ(physical_plan->getNumChildren(), 1u);
+      return impliesUniqueAttributes(physical_plan->children()[0], attributes);
+    }
+    default:
       break;
+  }
+  return false;
+}
+
+TypedValue StarSchemaSimpleCostModel::findCatalogRelationStat(
+    const P::PhysicalPtr &physical_plan,
+    const E::ExprId attr_id,
+    const StatType stat_type,
+    bool *is_exact_stat) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(physical_plan, &table_reference)) {
+    const attribute_id rel_attr_id =
+        findCatalogRelationAttributeId(table_reference, attr_id);
+    if (rel_attr_id != kInvalidAttributeID) {
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+
+      if (is_exact_stat != nullptr) {
+        *is_exact_stat = stat.isExact();
+      }
+
+      switch (stat_type) {
+        case StatType::kMin: {
+          if (stat.hasMinValue(rel_attr_id)) {
+            return stat.getMinValue(rel_attr_id);
+          }
+          break;
+        }
+        case StatType::kMax: {
+          if (stat.hasMaxValue(rel_attr_id)) {
+            return stat.getMaxValue(rel_attr_id);
+          }
+          break;
+        }
+        default:
+          break;
+      }
+      return NullType::InstanceNullable().makeNullValue();
     }
   }
-  return estimateCardinalityForTableReference(table_reference);
+
+  for (const auto &child : physical_plan->children()) {
+    if (E::ContainsExprId(child->getOutputAttributes(), attr_id)) {
+      return findCatalogRelationStat(child, attr_id, stat_type, is_exact_stat);
+    }
+  }
+  return NullType::InstanceNullable().makeNullValue();
+}
+
+attribute_id StarSchemaSimpleCostModel::findCatalogRelationAttributeId(
+    const physical::TableReferencePtr &table_reference,
+    const expressions::ExprId expr_id) {
+  const auto &attribute_list = table_reference->attribute_list();
+  for (std::size_t i = 0; i < attribute_list.size(); ++i) {
+    if (attribute_list[i]->id() == expr_id) {
+      return i;
+    }
+  }
+  return kInvalidAttributeID;
 }
 
 }  // namespace cost

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 6f6aa29..cbe18f4 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -23,11 +23,14 @@
 #include <cstddef>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/Selection.hpp"
@@ -36,6 +39,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -105,10 +109,70 @@ class StarSchemaSimpleCostModel : public CostModel {
   double estimateSelectivityForFilterPredicate(
       const physical::PhysicalPtr &physical_plan);
 
+  /**
+   * @brief Check whether a set of attributes are unique (i.e. have distinct
+   *        values) for a relation.
+   *
+   * @param physical_plan The physical plan that corresponds to a relation.
+   * @param attributes The set of attributes to be checked. Note that each
+   *        attribute in this set must be an output attribute of the physical
+   *        plan.
+   * @return True if it is guaranteed that the attributes are unique; false
+   *         otherwise.
+   */
+  bool impliesUniqueAttributes(
+      const physical::PhysicalPtr &physical_plan,
+      const std::vector<expressions::AttributeReferencePtr> &attributes);
+
+  /**
+   * @brief For a physical plan attribute, find its correponding catalog attribute's
+   *        MIN statistic. Returns Null value if there is no corresponding catalog
+   *        attribute for the physical plan attribute.
+   *
+   * @param physical_plan The physical plan.
+   * @param attribute The attribute. Must be an output attribute of the given
+   *        physical plan.
+   * @param is_exact_stat If this pointer is not null, its pointed content will
+   *        be modified by this method to indicate whether the returned statistic
+   *        is EXACT for the stored relation (i.e. not outdated or estimated).
+   * @return The MIN statistic for the attribute.
+   */
+  TypedValue findMinValueStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      bool *is_exact_stat = nullptr) {
+    return findCatalogRelationStat(
+        physical_plan, attribute->id(), StatType::kMin, is_exact_stat);
+  }
+
+  /**
+   * @brief For a physical plan attribute, find its correponding catalog attribute's
+   *        MAX statistic. Returns Null value if there is no corresponding catalog
+   *        attribute for the physical plan attribute.
+   *
+   * @param physical_plan The physical plan.
+   * @param attribute The attribute. Must be an output attribute of the given
+   *        physical plan.
+   * @param is_exact_stat If this pointer is not null, its pointed content will
+   *        be modified by this method to indicate whether the returned statistic
+   *        is EXACT for the stored relation (i.e. not not outdated or estimated).
+   * @return The MAX statistic for the attribute.
+   */
+  TypedValue findMaxValueStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      bool *is_exact_stat = nullptr) {
+    return findCatalogRelationStat(
+        physical_plan, attribute->id(), StatType::kMax, is_exact_stat);
+  }
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
 
+  std::size_t estimateCardinalityForFilterJoin(
+      const physical::FilterJoinPtr &physical_plan);
+
   std::size_t estimateCardinalityForHashJoin(
       const physical::HashJoinPtr &physical_plan);
 
@@ -144,6 +208,25 @@ class StarSchemaSimpleCostModel : public CostModel {
   std::size_t getNumDistinctValues(const expressions::ExprId attribute_id,
                                    const physical::TableReferencePtr &table_reference);
 
+  enum class StatType {
+    kMax = 0,
+    kMin
+  };
+
+  // For a physical plan attribute, find its correponding catalog attribute's
+  // min/max statistics. Returns Null value if there is no corresponding catalog
+  // attribute for the physical plan attribute (e.g. the attribute is the result
+  // of an expression).
+  TypedValue findCatalogRelationStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::ExprId expr_id,
+      const StatType stat_type,
+      bool *is_exact_stat);
+
+  // For a table reference attribute, find its correponding catalog attribute.
+  attribute_id findCatalogRelationAttributeId(
+      const physical::TableReferencePtr &table_reference,
+      const expressions::ExprId expr_id);
 
   DISALLOW_COPY_AND_ASSIGN(StarSchemaSimpleCostModel);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index 422d5ab..6b8666e 100644
--- a/query_optimizer/expressions/ExpressionUtil.hpp
+++ b/query_optimizer/expressions/ExpressionUtil.hpp
@@ -122,12 +122,12 @@ bool ContainsExprId(
  *              contain the other operand).
  * @return True if \p left is a subset of \p right.
  */
-template <class NamedExpressionType>
+template <class LeftNamedExpressionType, class RightNamedExpressionType>
 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 LeftNamedExpressionType>> &left,
+    const std::vector<std::shared_ptr<const RightNamedExpressionType>> &right) {
   UnorderedNamedExpressionSet supset(right.begin(), right.end());
-  for (const std::shared_ptr<const NamedExpressionType> &expr : left) {
+  for (const std::shared_ptr<const LeftNamedExpressionType> &expr : left) {
     if (supset.find(expr) == supset.end()) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index 7f26943..f68ed39 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -23,6 +23,7 @@ add_library(quickstep_queryoptimizer_physical_CreateIndex CreateIndex.cpp Create
 add_library(quickstep_queryoptimizer_physical_CreateTable CreateTable.cpp CreateTable.hpp)
 add_library(quickstep_queryoptimizer_physical_DeleteTuples DeleteTuples.cpp DeleteTuples.hpp)
 add_library(quickstep_queryoptimizer_physical_DropTable DropTable.cpp DropTable.hpp)
+add_library(quickstep_queryoptimizer_physical_FilterJoin FilterJoin.cpp FilterJoin.hpp)
 add_library(quickstep_queryoptimizer_physical_HashJoin HashJoin.cpp HashJoin.hpp)
 add_library(quickstep_queryoptimizer_physical_InsertSelection InsertSelection.cpp InsertSelection.hpp)
 add_library(quickstep_queryoptimizer_physical_InsertTuple InsertTuple.cpp InsertTuple.hpp)
@@ -115,6 +116,18 @@ target_link_libraries(quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_FilterJoin
+                      glog
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_BinaryJoin
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_physical_HashJoin
                       glog
                       quickstep_queryoptimizer_OptimizerTree
@@ -282,6 +295,7 @@ target_link_libraries(quickstep_queryoptimizer_physical
                       quickstep_queryoptimizer_physical_CreateTable
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/FilterJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/FilterJoin.cpp b/query_optimizer/physical/FilterJoin.cpp
new file mode 100644
index 0000000..1817a1c
--- /dev/null
+++ b/query_optimizer/physical/FilterJoin.cpp
@@ -0,0 +1,115 @@
+/**
+ * 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 "query_optimizer/physical/FilterJoin.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "utility/Cast.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+std::vector<E::AttributeReferencePtr> FilterJoin::getReferencedAttributes() const {
+  std::vector<E::AttributeReferencePtr> referenced_attributes;
+  for (const auto &project_expression : project_expressions()) {
+    const auto referenced_attributes_in_expression =
+        project_expression->getReferencedAttributes();
+    referenced_attributes.insert(referenced_attributes.end(),
+                                 referenced_attributes_in_expression.begin(),
+                                 referenced_attributes_in_expression.end());
+  }
+  referenced_attributes.insert(referenced_attributes.end(),
+                               probe_attributes_.begin(),
+                               probe_attributes_.end());
+  referenced_attributes.insert(referenced_attributes.end(),
+                               build_attributes_.begin(),
+                               build_attributes_.end());
+  if (build_side_filter_predicate_ != nullptr) {
+    const auto referenced_attributes_in_predicate =
+        build_side_filter_predicate_->getReferencedAttributes();
+    referenced_attributes.insert(referenced_attributes.end(),
+                                 referenced_attributes_in_predicate.begin(),
+                                 referenced_attributes_in_predicate.end());
+  }
+  return referenced_attributes;
+}
+
+bool FilterJoin::maybeCopyWithPrunedExpressions(
+    const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+    PhysicalPtr *output) const {
+  std::vector<E::NamedExpressionPtr> new_project_expressions;
+  const auto &current_project_expressions = project_expressions();
+  for (const auto &project_expression : current_project_expressions) {
+    if (referenced_expressions.find(project_expression) != referenced_expressions.end()) {
+      new_project_expressions.emplace_back(project_expression);
+    }
+  }
+  if (new_project_expressions.size() != current_project_expressions.size()) {
+    *output = Create(left(),
+                     right(),
+                     probe_attributes_,
+                     build_attributes_,
+                     new_project_expressions,
+                     build_side_filter_predicate_,
+                     is_anti_join_);
+    return true;
+  }
+  return false;
+}
+
+void FilterJoin::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  BinaryJoin::getFieldStringItems(inline_field_names,
+                                  inline_field_values,
+                                  non_container_child_field_names,
+                                  non_container_child_fields,
+                                  container_child_field_names,
+                                  container_child_fields);
+
+  inline_field_names->push_back("is_anti_join");
+  inline_field_values->push_back(std::to_string(is_anti_join_));
+
+  if (build_side_filter_predicate_ != nullptr) {
+    non_container_child_field_names->emplace_back("build_side_filter_predicate");
+    non_container_child_fields->emplace_back(build_side_filter_predicate_);
+  }
+
+  container_child_field_names->push_back("probe_attributes");
+  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(probe_attributes_));
+  container_child_field_names->push_back("build_attributes");
+  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(build_attributes_));
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/FilterJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/FilterJoin.hpp b/query_optimizer/physical/FilterJoin.hpp
new file mode 100644
index 0000000..ad4e18b
--- /dev/null
+++ b/query_optimizer/physical/FilterJoin.hpp
@@ -0,0 +1,187 @@
+/**
+ * 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_QUERY_OPTIMIZER_PHYSICAL_FILTER_JOIN_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_FILTER_JOIN_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/BinaryJoin.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerPhysical
+ *  @{
+ */
+
+class FilterJoin;
+typedef std::shared_ptr<const FilterJoin> FilterJoinPtr;
+
+/**
+ * @brief Physical filter join node. Semantically, FilterJoin is similar to
+ *        HashJoin where the difference is that FilterJoin builds a bit vector
+ *        instead of a hash table.
+ *
+ * @note FilterJoin's backend execution relies on LIPFilter injection (attach
+ *       the bit vectors as filters into downstream relational operators).
+ */
+class FilterJoin : public BinaryJoin {
+ public:
+  PhysicalType getPhysicalType() const override {
+    return PhysicalType::kFilterJoin;
+  }
+
+  std::string getName() const override {
+    if (is_anti_join_) {
+      return "FilterJoin(Anti)";
+    } else {
+      return "FilterJoin";
+    }
+  }
+
+  /**
+   * @return The probe side attributes.
+   */
+  const std::vector<expressions::AttributeReferencePtr>& probe_attributes() const {
+    return probe_attributes_;
+  }
+
+  /**
+   * @return The build side attributes.
+   */
+  const std::vector<expressions::AttributeReferencePtr>& build_attributes() const {
+    return build_attributes_;
+  }
+
+  /**
+   * @return The build side filter predicate.
+   */
+  const expressions::PredicatePtr& build_side_filter_predicate() const {
+    return build_side_filter_predicate_;
+  }
+
+  /**
+   * @return Whether this is an anti-join.
+   */
+  const bool is_anti_join() const {
+    return is_anti_join_;
+  }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override {
+    DCHECK_EQ(children().size(), new_children.size());
+    return Create(new_children[0],
+                  new_children[1],
+                  probe_attributes_,
+                  build_attributes_,
+                  project_expressions(),
+                  build_side_filter_predicate_,
+                  is_anti_join_);
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  bool maybeCopyWithPrunedExpressions(
+      const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+      PhysicalPtr *output) const override;
+
+  /**
+   * @brief Creates a physical FilterJoin.
+   * @param probe_child The probe side child plan.
+   * @param build_child The build side child plan.
+   * @param probe_attributes The probe side attributes.
+   * @param build_attributes The build side attributes.
+   * @param project_expressions The project expressions.
+   * @param build_side_filter_predicate Optional filtering predicate to be
+   *        applied to the build side child BEFORE join.
+   * @param is_anti_join Whether this is an anti-join.
+   * @return An immutable physical FilterJoin.
+   */
+  static FilterJoinPtr Create(
+      const PhysicalPtr &probe_child,
+      const PhysicalPtr &build_child,
+      const std::vector<expressions::AttributeReferencePtr> &probe_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &build_attributes,
+      const std::vector<expressions::NamedExpressionPtr> &project_expressions,
+      const expressions::PredicatePtr &build_side_filter_predicate,
+      const bool is_anti_join) {
+    return FilterJoinPtr(
+        new FilterJoin(probe_child,
+                       build_child,
+                       probe_attributes,
+                       build_attributes,
+                       project_expressions,
+                       build_side_filter_predicate,
+                       is_anti_join));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  FilterJoin(
+      const PhysicalPtr &probe_child,
+      const PhysicalPtr &build_child,
+      const std::vector<expressions::AttributeReferencePtr> &probe_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &build_attributes,
+      const std::vector<expressions::NamedExpressionPtr> &project_expressions,
+      const expressions::PredicatePtr &build_side_filter_predicate,
+      const bool is_anti_join)
+      : BinaryJoin(probe_child, build_child, project_expressions),
+        probe_attributes_(probe_attributes),
+        build_attributes_(build_attributes),
+        build_side_filter_predicate_(build_side_filter_predicate),
+        is_anti_join_(is_anti_join) {
+  }
+
+  std::vector<expressions::AttributeReferencePtr> probe_attributes_;
+  std::vector<expressions::AttributeReferencePtr> build_attributes_;
+  expressions::PredicatePtr build_side_filter_predicate_;
+  bool is_anti_join_;
+
+  DISALLOW_COPY_AND_ASSIGN(FilterJoin);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_FILTER_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/LIPFilterConfiguration.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/LIPFilterConfiguration.hpp b/query_optimizer/physical/LIPFilterConfiguration.hpp
index 62a6149..90c81fe 100644
--- a/query_optimizer/physical/LIPFilterConfiguration.hpp
+++ b/query_optimizer/physical/LIPFilterConfiguration.hpp
@@ -21,6 +21,7 @@
 #define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_LIP_FILTER_CONFIGURATION_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <map>
 #include <memory>
 #include <vector>
@@ -40,50 +41,211 @@ namespace physical {
 class Physical;
 typedef std::shared_ptr<const Physical> PhysicalPtr;
 
+class LIPFilterBuildInfo;
+typedef std::shared_ptr<const LIPFilterBuildInfo> LIPFilterBuildInfoPtr;
+
+class LIPFilterProbeInfo;
+typedef std::shared_ptr<const LIPFilterProbeInfo> LIPFilterProbeInfoPtr;
+
 /**
  * @brief Optimizer information for a LIP filter builder.
  */
-struct LIPFilterBuildInfo {
+class LIPFilterBuildInfo {
+ public:
+  /**
+   * @return The LIPFilter's type.
+   */
+  LIPFilterType filter_type() const {
+    return filter_type_;
+  }
+
+  /**
+   * @return The LIPFilter's build attribute.
+   */
+  const expressions::AttributeReferencePtr& build_attribute() const {
+    return build_attribute_;
+  }
+
+ protected:
   /**
    * @brief Constructor.
    *
-   * @param build_attribute_in The attribute to build the LIP filter with.
-   * @param filter_cardinality_in The LIP filter's cardinality.
    * @param filter_type_in The LIP filter's type.
+   * @param build_attribute_in The attribute to build the LIP filter with.
+   */
+  LIPFilterBuildInfo(const LIPFilterType &filter_type,
+                     const expressions::AttributeReferencePtr &build_attribute)
+      : filter_type_(filter_type),
+        build_attribute_(build_attribute) {}
+
+ private:
+  const LIPFilterType filter_type_;
+  const expressions::AttributeReferencePtr build_attribute_;
+
+  DISALLOW_COPY_AND_ASSIGN(LIPFilterBuildInfo);
+};
+
+/**
+ * @brief Subclass that contains extra information for SingleIdentityHashFilter
+ *        builder.
+ */
+class SingleIdentityHashFilterBuildInfo : public LIPFilterBuildInfo {
+ public:
+  /**
+   * @return The cardinality of this SingleIdentityHashFilter.
    */
-  LIPFilterBuildInfo(const expressions::AttributeReferencePtr &build_attribute_in,
-                     const std::size_t filter_cardinality_in,
-                     const LIPFilterType &filter_type_in)
-      : build_attribute(build_attribute_in),
-        filter_cardinality(filter_cardinality_in),
-        filter_type(filter_type_in) {
+  std::size_t filter_cardinality() const {
+    return filter_cardinality_;
   }
-  const expressions::AttributeReferencePtr build_attribute;
-  const std::size_t filter_cardinality;
-  const LIPFilterType filter_type;
+
+  /**
+   * @brief Creates a shared SingleIdentityHashFilterBuildInfo.
+   *
+   * @param build_attribute The attribute to build the filter with.
+   * @param filter_cardinality The cardinality of this SingleIdentityHashFilter.
+   */
+  static LIPFilterBuildInfoPtr Create(
+      const expressions::AttributeReferencePtr &build_attribute,
+      const std::size_t filter_cardinality) {
+    return LIPFilterBuildInfoPtr(
+        new SingleIdentityHashFilterBuildInfo(build_attribute,
+                                              filter_cardinality));
+  }
+
+ private:
+  SingleIdentityHashFilterBuildInfo(const expressions::AttributeReferencePtr &build_attribute,
+                                    const std::size_t filter_cardinality)
+      : LIPFilterBuildInfo(LIPFilterType::kSingleIdentityHashFilter,
+                           build_attribute),
+        filter_cardinality_(filter_cardinality) {}
+
+  const std::size_t filter_cardinality_;
+
+  DISALLOW_COPY_AND_ASSIGN(SingleIdentityHashFilterBuildInfo);
 };
 
 /**
+ * @brief Subclass that contains extra information for BitVectorExactFilter
+ *        builder.
+ */
+class BitVectorExactFilterBuildInfo : public LIPFilterBuildInfo {
+ public:
+  /**
+   * @return The minimum possible value for this BitVectorExactFilter.
+   */
+  std::int64_t min_value() const {
+    return min_value_;
+  }
+
+  /**
+   * @return The maximum possible value for this BitVectorExactFilter.
+   */
+  std::int64_t max_value() const {
+    return max_value_;
+  }
+
+  /**
+   * @return Whether this is an anti-filter.
+   */
+  bool is_anti_filter() const {
+    return is_anti_filter_;
+  }
+
+  /**
+   * @brief Creates a shared BitVectorExactFilterBuildInfo.
+   *
+   * @param build_attribute The attribute to build the filter with.
+   * @param min_value The minimum possible value for this BitVectorExactFilter
+   *        to set.
+   * @param max_value The maximum possible value for this BitVectorExactFilter
+   *        to set.
+   * @param is_anti_filter Whether this is an anti-filter.
+   */
+  static LIPFilterBuildInfoPtr Create(
+      const expressions::AttributeReferencePtr &build_attribute,
+      const std::int64_t min_value,
+      const std::int64_t max_value,
+      const bool is_anti_filter) {
+    return LIPFilterBuildInfoPtr(
+        new BitVectorExactFilterBuildInfo(build_attribute,
+                                          min_value,
+                                          max_value,
+                                          is_anti_filter));
+  }
+
+ private:
+  BitVectorExactFilterBuildInfo(const expressions::AttributeReferencePtr &build_attribute,
+                                const std::int64_t min_value,
+                                const std::int64_t max_value,
+                                const bool is_anti_filter)
+      : LIPFilterBuildInfo(LIPFilterType::kBitVectorExactFilter,
+                           build_attribute),
+        min_value_(min_value),
+        max_value_(max_value),
+        is_anti_filter_(is_anti_filter) {}
+
+  const std::int64_t min_value_;
+  const std::int64_t max_value_;
+  const bool is_anti_filter_;
+
+  DISALLOW_COPY_AND_ASSIGN(BitVectorExactFilterBuildInfo);
+};
+
+
+/**
  * @brief Optimizer information for a LIP filter prober.
  */
-struct LIPFilterProbeInfo {
+class LIPFilterProbeInfo {
+ public:
   /**
-   * @brief Constructor.
+   * @return The attribute to probe the LIP Filter with.
+   */
+  const expressions::AttributeReferencePtr& probe_attribute() const {
+    return probe_attribute_;
+  }
+
+  /**
+   * @return The attribute that the LIP filter is built with.
+   */
+  const expressions::AttributeReferencePtr& build_attribute() const {
+    return build_attribute_;
+  }
+
+  /**
+   * @return The physical node that the LIP filter's builder is attached to.
+   */
+  const PhysicalPtr& builder() const {
+    return builder_;
+  }
+
+  /**
+   * @brief Creates a shared LIPFilterProbeInfo.
    *
-   * @param probe_attribute_in The attribute to probe the LIP filter with.
-   * @param build_attribute_in The attribute that the LIP filter is built with.
-   * @param builder_in The physical node that the LIP filter's builder is attached to.
-   */
-  LIPFilterProbeInfo(const expressions::AttributeReferencePtr &probe_attribute_in,
-                     const expressions::AttributeReferencePtr &build_attribute_in,
-                     const PhysicalPtr &builder_in)
-      : probe_attribute(probe_attribute_in),
-        build_attribute(build_attribute_in),
-        builder(builder_in) {
-  }
-  const expressions::AttributeReferencePtr probe_attribute;
-  const expressions::AttributeReferencePtr build_attribute;
-  const PhysicalPtr builder;
+   * @param probe_attribute The attribute to probe the LIP filter with.
+   * @param build_attribute The attribute that the LIP filter is built with.
+   * @param builder The physical node that the LIP filter's builder is attached to.
+   */
+  static LIPFilterProbeInfoPtr Create(
+      const expressions::AttributeReferencePtr &probe_attribute,
+      const expressions::AttributeReferencePtr &build_attribute,
+      const PhysicalPtr &builder) {
+    return LIPFilterProbeInfoPtr(
+        new LIPFilterProbeInfo(probe_attribute, build_attribute, builder));
+  }
+
+ private:
+  LIPFilterProbeInfo(const expressions::AttributeReferencePtr &probe_attribute,
+                     const expressions::AttributeReferencePtr &build_attribute,
+                     const PhysicalPtr &builder)
+      : probe_attribute_(probe_attribute),
+        build_attribute_(build_attribute),
+        builder_(builder) {}
+
+  const expressions::AttributeReferencePtr probe_attribute_;
+  const expressions::AttributeReferencePtr build_attribute_;
+  const PhysicalPtr builder_;
+
+  DISALLOW_COPY_AND_ASSIGN(LIPFilterProbeInfo);
 };
 
 
@@ -104,33 +266,23 @@ class LIPFilterConfiguration {
   /**
    * @brief Add information for a LIP filter builder.
    *
-   * @param build_attribute The attribute to build the LIP filter with.
+   * @param build_info A shared_ptr to LIPFilterBuildInfo.
    * @param builder The physical node to attach the LIP filter builder to.
-   * @param filter_size The LIP filter's cardinality.
-   * @param filter_type The LIP filter's type.
    */
-  void addBuildInfo(const expressions::AttributeReferencePtr &build_attribute,
-                    const PhysicalPtr &builder,
-                    const std::size_t filter_size,
-                    const LIPFilterType &filter_type) {
-    build_info_map_[builder].emplace_back(
-        build_attribute, filter_size, filter_type);
+  void addBuildInfo(const LIPFilterBuildInfoPtr &build_info,
+                    const PhysicalPtr &builder) {
+    build_info_map_[builder].emplace_back(build_info);
   }
 
   /**
    * @brief Add information for a LIP filter prober.
    *
-   * @param probe_attribute The attribute to probe the LIP filter with.
+   * @param probe_info A shared_ptr to LIPFilterProbeInfo.
    * @param prober The physical node to attach the LIP filter prober to.
-   * @param build_attribute The attribute that the LIP filter is built with.
-   * @param builder The physical node that the LIP filter's builder is attached to.
    */
-  void addProbeInfo(const expressions::AttributeReferencePtr &probe_attribute,
-                    const PhysicalPtr &prober,
-                    const expressions::AttributeReferencePtr &build_attribute,
-                    const PhysicalPtr &builder) {
-    probe_info_map_[prober].emplace_back(
-        probe_attribute, build_attribute, builder);
+  void addProbeInfo(const LIPFilterProbeInfoPtr &probe_info,
+                    const PhysicalPtr &prober) {
+    probe_info_map_[prober].emplace_back(probe_info);
   }
 
   /**
@@ -140,7 +292,7 @@ class LIPFilterConfiguration {
    *         a vector of all the LIP filter builders that are attached to the
    *         physical node.
    */
-  const std::map<PhysicalPtr, std::vector<LIPFilterBuildInfo>>& getBuildInfoMap() const {
+  const std::map<PhysicalPtr, std::vector<LIPFilterBuildInfoPtr>>& getBuildInfoMap() const {
     return build_info_map_;
   }
 
@@ -151,13 +303,26 @@ class LIPFilterConfiguration {
    *         a vector of all the LIP filter probers that are attached to the
    *         physical node.
    */
-  const std::map<PhysicalPtr, std::vector<LIPFilterProbeInfo>>& getProbeInfoMap() const {
+  const std::map<PhysicalPtr, std::vector<LIPFilterProbeInfoPtr>>& getProbeInfoMap() const {
     return probe_info_map_;
   }
 
+  /**
+   * @brief Clone a copy of this configuration.
+   *
+   * @return A copy of this confiugration. Caller should take ownership of the
+   *         returned object.
+   */
+  LIPFilterConfiguration* clone() const {
+    LIPFilterConfiguration *new_conf = new LIPFilterConfiguration();
+    new_conf->build_info_map_ = build_info_map_;
+    new_conf->probe_info_map_ = probe_info_map_;
+    return new_conf;
+  }
+
  private:
-  std::map<PhysicalPtr, std::vector<LIPFilterBuildInfo>> build_info_map_;
-  std::map<PhysicalPtr, std::vector<LIPFilterProbeInfo>> probe_info_map_;
+  std::map<PhysicalPtr, std::vector<LIPFilterBuildInfoPtr>> build_info_map_;
+  std::map<PhysicalPtr, std::vector<LIPFilterProbeInfoPtr>> probe_info_map_;
 
   DISALLOW_COPY_AND_ASSIGN(LIPFilterConfiguration);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PatternMatcher.hpp b/query_optimizer/physical/PatternMatcher.hpp
index 5cd6fd3..4336767 100644
--- a/query_optimizer/physical/PatternMatcher.hpp
+++ b/query_optimizer/physical/PatternMatcher.hpp
@@ -35,6 +35,7 @@ class CopyFrom;
 class CreateTable;
 class DeleteTuples;
 class DropTable;
+class FilterJoin;
 class HashJoin;
 class InsertTuple;
 class Join;
@@ -113,6 +114,7 @@ using SomeCopyFrom = SomePhysicalNode<CopyFrom, PhysicalType::kCopyFrom>;
 using SomeCreateTable = SomePhysicalNode<CreateTable, PhysicalType::kCreateTable>;
 using SomeDeleteTuples = SomePhysicalNode<DeleteTuples, PhysicalType::kDeleteTuples>;
 using SomeDropTable = SomePhysicalNode<DropTable, PhysicalType::kDropTable>;
+using SomeFilterJoin = SomePhysicalNode<FilterJoin, PhysicalType::kFilterJoin>;
 using SomeHashJoin = SomePhysicalNode<HashJoin, PhysicalType::kHashJoin>;
 using SomeInsertTuple = SomePhysicalNode<InsertTuple, PhysicalType::kInsertTuple>;
 using SomeJoin = SomePhysicalNode<Join, PhysicalType::kHashJoin, PhysicalType::kNestedLoopsJoin>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index f5f35a1..1da5929 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -38,6 +38,7 @@ enum class PhysicalType {
   kCreateTable,
   kDeleteTuples,
   kDropTable,
+  kFilterJoin,
   kHashJoin,
   kInsertSelection,
   kInsertTuple,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/physical/TopLevelPlan.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TopLevelPlan.hpp b/query_optimizer/physical/TopLevelPlan.hpp
index 7dfc2b6..9e567e1 100644
--- a/query_optimizer/physical/TopLevelPlan.hpp
+++ b/query_optimizer/physical/TopLevelPlan.hpp
@@ -126,7 +126,8 @@ class TopLevelPlan : public Physical {
     }
     return TopLevelPlan::Create(new_children[0],
                                 new_shared_subplans,
-                                uncorrelated_subquery_map_);
+                                uncorrelated_subquery_map_,
+                                lip_filter_configuration_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/rules/AttachLIPFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachLIPFilters.cpp b/query_optimizer/rules/AttachLIPFilters.cpp
index b3c57ab..48b68bc 100644
--- a/query_optimizer/rules/AttachLIPFilters.cpp
+++ b/query_optimizer/rules/AttachLIPFilters.cpp
@@ -55,7 +55,14 @@ P::PhysicalPtr AttachLIPFilters::apply(const P::PhysicalPtr &input) {
   cost_model_.reset(
       new cost::StarSchemaSimpleCostModel(
           top_level_plan->shared_subplans()));
-  lip_filter_configuration_.reset(new P::LIPFilterConfiguration());
+
+  const P::LIPFilterConfigurationPtr &existing_configuration =
+      top_level_plan->lip_filter_configuration();
+  if (existing_configuration != nullptr) {
+    lip_filter_configuration_.reset(existing_configuration->clone());
+  } else {
+    lip_filter_configuration_.reset(new P::LIPFilterConfiguration());
+  }
 
   std::set<E::ExprId> already_filtered_attributes;
   attachLIPFilters(NodeList(input), &already_filtered_attributes);
@@ -101,7 +108,7 @@ void AttachLIPFilters::attachLIPFilters(
   }
 
   if (probe_child != nullptr &&
-      cost_model_->estimateCardinality(probe_child) > 10000000) {
+      cost_model_->estimateCardinality(probe_child) >= 100000) {
     const auto &candidate_lip_filters = getProbeSideInfo(path.cons(probe_child));
     if (!candidate_lip_filters.empty()) {
       std::map<E::AttributeReferencePtr, LIPFilterInfoPtr> selected_filters;
@@ -119,15 +126,16 @@ void AttachLIPFilters::attachLIPFilters(
         if (already_filtered_attributes->find(source_attr_id)
                 == already_filtered_attributes->end()) {
           lip_filter_configuration_->addBuildInfo(
-              pair.second->source_attribute,
-              pair.second->source,
-              pair.second->estimated_cardinality * 8,
-              LIPFilterType::kSingleIdentityHashFilter);
-          lip_filter_configuration_->addProbeInfo(
-              pair.first,
-              node,
-              pair.second->source_attribute,
+              P::SingleIdentityHashFilterBuildInfo::Create(
+                  pair.second->source_attribute,
+                  pair.second->estimated_cardinality * 8),
               pair.second->source);
+          lip_filter_configuration_->addProbeInfo(
+              P::LIPFilterProbeInfo::Create(
+                  pair.first,
+                  pair.second->source_attribute,
+                  pair.second->source),
+              node);
           already_filtered_attributes->emplace(source_attr_id);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 86d1ef7..223c78c 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -22,6 +22,7 @@ add_library(quickstep_queryoptimizer_rules_AttachLIPFilters AttachLIPFilters.cpp
 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)
+add_library(quickstep_queryoptimizer_rules_InjectJoinFilters InjectJoinFilters.cpp InjectJoinFilters.hpp)
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
@@ -196,6 +197,26 @@ target_link_libraries(quickstep_queryoptimizer_rules_SwapProbeBuild
 target_link_libraries(quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_InjectJoinFilters
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_LIPFilterConfiguration
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_utility_Macros
+                      quickstep_utility_lipfilter_LIPFilter)
 target_link_libraries(quickstep_queryoptimizer_rules_UnnestSubqueries
                       quickstep_queryoptimizer_OptimizerContext
                       quickstep_queryoptimizer_expressions_AttributeReference
@@ -246,6 +267,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_BottomUpRule
                       quickstep_queryoptimizer_rules_CollapseProject
                       quickstep_queryoptimizer_rules_GenerateJoins
+                      quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate



[39/50] [abbrv] incubator-quickstep git commit: Fuse Aggregate with LeftOuterJoin to accelerate evaluation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/relational_operators/BuildAggregationExistenceMapOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildAggregationExistenceMapOperator.hpp b/relational_operators/BuildAggregationExistenceMapOperator.hpp
new file mode 100644
index 0000000..e2928a8
--- /dev/null
+++ b/relational_operators/BuildAggregationExistenceMapOperator.hpp
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_RELATIONAL_OPERATORS_BUILD_AGGREGATION_EXISTENCE_MAP_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_BUILD_AGGREGATION_EXISTENCE_MAP_OPERATOR_HPP_
+
+#include <cstddef>
+
+#include <string>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class AggregationOperationState;
+class CatalogRelationSchema;
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+namespace serialization { class WorkOrder; }
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+/**
+ * @brief An operator which builds a bit vector on the input relation's one
+ *        attribute where the bit vector serves as the existence map for an
+ *        AggregationOperationState's CollisionFreeVectorTable.
+ **/
+class BuildAggregationExistenceMapOperator : public RelationalOperator {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this operator belongs.
+   * @param input_relation The relation to build the existence map on.
+   * @param build_attribute The ID of the attribute to build the existence map on.
+   * @param input_relation_is_stored If input_relation is a stored relation and
+   *        is fully available to the operator before it can start generating
+   *        workorders.
+   * @param aggr_state_index The index of the AggregationState in QueryContext.
+   **/
+  BuildAggregationExistenceMapOperator(const std::size_t query_id,
+                                       const CatalogRelation &input_relation,
+                                       const attribute_id build_attribute,
+                                       const bool input_relation_is_stored,
+                                       const QueryContext::aggregation_state_id aggr_state_index)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
+        build_attribute_(build_attribute),
+        input_relation_is_stored_(input_relation_is_stored),
+        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
+                                                           : std::vector<block_id>()),
+        aggr_state_index_(aggr_state_index),
+        num_workorders_generated_(0),
+        started_(false) {}
+
+  ~BuildAggregationExistenceMapOperator() override {}
+
+  std::string getName() const override {
+    return "BuildAggregationExistenceMapOperator";
+  }
+
+  /**
+   * @return The input relation.
+   */
+  const CatalogRelation& input_relation() const {
+    return input_relation_;
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
+    input_relation_block_ids_.push_back(input_block_id);
+  }
+
+ private:
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
+  const CatalogRelation &input_relation_;
+  const attribute_id build_attribute_;
+  const bool input_relation_is_stored_;
+  std::vector<block_id> input_relation_block_ids_;
+  const QueryContext::aggregation_state_id aggr_state_index_;
+
+  std::vector<block_id>::size_type num_workorders_generated_;
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildAggregationExistenceMapOperator);
+};
+
+/**
+ * @brief A WorkOrder produced by BuildAggregationExistenceMapOperator.
+ **/
+class BuildAggregationExistenceMapWorkOrder : public WorkOrder {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param query_id The ID of this query.
+   * @param input_relation The relation to build the existence map on.
+   * @param build_block_id The block id.
+   * @param build_attribute The ID of the attribute to build on.
+   * @param state The AggregationState to use.
+   * @param storage_manager The StorageManager to use.
+   **/
+  BuildAggregationExistenceMapWorkOrder(const std::size_t query_id,
+                                        const CatalogRelationSchema &input_relation,
+                                        const block_id build_block_id,
+                                        const attribute_id build_attribute,
+                                        AggregationOperationState *state,
+                                        StorageManager *storage_manager)
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
+        build_block_id_(build_block_id),
+        build_attribute_(build_attribute),
+        state_(DCHECK_NOTNULL(state)),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
+
+  ~BuildAggregationExistenceMapWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  const CatalogRelationSchema &input_relation_;
+  const block_id build_block_id_;
+  const attribute_id build_attribute_;
+  AggregationOperationState *state_;
+
+  StorageManager *storage_manager_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildAggregationExistenceMapWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_BUILD_AGGREGATION_EXISTENCE_MAP_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index df4114d..457d58a 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -33,6 +33,9 @@ set_gflags_lib_name ()
 
 # Declare micro-libs:
 add_library(quickstep_relationaloperators_AggregationOperator AggregationOperator.cpp AggregationOperator.hpp)
+add_library(quickstep_relationaloperators_BuildAggregationExistenceMapOperator
+            BuildAggregationExistenceMapOperator.cpp
+            BuildAggregationExistenceMapOperator.hpp)
 add_library(quickstep_relationaloperators_BuildHashOperator BuildHashOperator.cpp BuildHashOperator.hpp)
 add_library(quickstep_relationaloperators_BuildLIPFilterOperator BuildLIPFilterOperator.cpp BuildLIPFilterOperator.hpp)
 add_library(quickstep_relationaloperators_CreateIndexOperator CreateIndexOperator.cpp CreateIndexOperator.hpp)
@@ -95,6 +98,31 @@ target_link_libraries(quickstep_relationaloperators_AggregationOperator
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_BuildAggregationExistenceMapOperator
+                      glog
+                      quickstep_catalog_CatalogAttribute
+                      quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogRelationSchema
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_storage_AggregationOperationState
+                      quickstep_storage_CollisionFreeVectorTable
+                      quickstep_storage_StorageBlock
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageManager
+                      quickstep_storage_TupleStorageSubBlock
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -518,6 +546,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
                       quickstep_relationaloperators_AggregationOperator
+                      quickstep_relationaloperators_BuildAggregationExistenceMapOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_DeleteOperator
@@ -552,6 +581,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrder_proto
 add_library(quickstep_relationaloperators ../empty_src.cpp RelationalOperatorsModule.hpp)
 target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_AggregationOperator
+                      quickstep_relationaloperators_BuildAggregationExistenceMapOperator
                       quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_CreateIndexOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 76753d2..d0d0753 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -44,6 +44,7 @@ enum WorkOrderType {
   UPDATE = 20;
   WINDOW_AGGREGATION = 21;
   DESTROY_AGGREGATION_STATE = 22;
+  BUILD_AGGREGATION_EXISTENCE_MAP = 23;
 }
 
 message WorkOrder {
@@ -278,6 +279,15 @@ message WindowAggregationWorkOrder {
 
 message DestroyAggregationStateWorkOrder {
   extend WorkOrder {
-    optional uint32 aggr_state_index = 339;
+    optional uint32 aggr_state_index = 352;
+  }
+}
+
+message BuildAggregationExistenceMapWorkOrder {
+  extend WorkOrder {
+    optional int32 relation_id = 368;
+    optional fixed64 build_block_id = 369;
+    optional int32 build_attribute = 370;
+    optional uint32 aggr_state_index = 371;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index bd2a0f8..d2c8251 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -29,6 +29,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/AggregationOperator.hpp"
+#include "relational_operators/BuildAggregationExistenceMapOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
 #include "relational_operators/BuildLIPFilterOperator.hpp"
 #include "relational_operators/DeleteOperator.hpp"
@@ -91,6 +92,19 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           CreateLIPFilterAdaptiveProberHelper(
               proto.GetExtension(serialization::AggregationWorkOrder::lip_deployment_index), query_context));
     }
+    case serialization::BUILD_AGGREGATION_EXISTENCE_MAP: {
+      LOG(INFO) << "Creating BuildAggregationExistenceMapWorkOrder in Shiftboss " << shiftboss_index;
+
+      return new BuildAggregationExistenceMapWorkOrder(
+          proto.query_id(),
+          catalog_database->getRelationSchemaById(
+              proto.GetExtension(serialization::BuildAggregationExistenceMapWorkOrder::relation_id)),
+          proto.GetExtension(serialization::BuildAggregationExistenceMapWorkOrder::build_block_id),
+          proto.GetExtension(serialization::BuildAggregationExistenceMapWorkOrder::build_attribute),
+          query_context->getAggregationState(
+              proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index)),
+          storage_manager);
+    }
     case serialization::BUILD_LIP_FILTER: {
       LOG(INFO) << "Creating BuildLIPFilterWorkOrder in Shiftboss " << shiftboss_index;
 
@@ -525,6 +539,29 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              query_context.isValidAggregationStateId(
                  proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index));
     }
+    case serialization::BUILD_AGGREGATION_EXISTENCE_MAP: {
+      if (!proto.HasExtension(serialization::BuildAggregationExistenceMapWorkOrder::relation_id)) {
+        return false;
+      }
+
+      const relation_id rel_id =
+          proto.GetExtension(serialization::BuildAggregationExistenceMapWorkOrder::relation_id);
+      if (!catalog_database.hasRelationWithId(rel_id)) {
+        return false;
+      }
+
+      const CatalogRelationSchema &relation = catalog_database.getRelationSchemaById(rel_id);
+      const attribute_id build_attribute =
+          proto.GetExtension(serialization::BuildAggregationExistenceMapWorkOrder::build_attribute);
+      if (!relation.hasAttributeWithId(build_attribute)) {
+        return false;
+      }
+
+      return proto.HasExtension(serialization::BuildAggregationExistenceMapWorkOrder::build_block_id) &&
+             proto.HasExtension(serialization::BuildAggregationExistenceMapWorkOrder::aggr_state_index) &&
+             query_context.isValidAggregationStateId(
+                 proto.GetExtension(serialization::BuildAggregationExistenceMapWorkOrder::aggr_state_index));
+    }
     case serialization::BUILD_HASH: {
       if (!proto.HasExtension(serialization::BuildHashWorkOrder::relation_id)) {
         return false;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 0b34908..0f39b41 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -412,12 +412,18 @@ std::size_t AggregationOperationState::getNumFinalizationPartitions() const {
   }
 }
 
+CollisionFreeVectorTable* AggregationOperationState
+    ::getCollisionFreeVectorTable() const {
+  return static_cast<CollisionFreeVectorTable *>(
+      collision_free_hashtable_.get());
+}
+
 void AggregationOperationState::initialize(const std::size_t partition_id) {
   if (is_aggregate_collision_free_) {
     static_cast<CollisionFreeVectorTable *>(
         collision_free_hashtable_.get())->initialize(partition_id);
   } else {
-    LOG(FATAL) << "AggregationOperationState::initializeState() "
+    LOG(FATAL) << "AggregationOperationState::initialize() "
                << "is not supported by this aggregation";
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 13ee377..c8930ee 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -41,6 +41,7 @@ namespace serialization { class AggregationOperationState; }
 class AggregateFunction;
 class CatalogDatabaseLite;
 class CatalogRelationSchema;
+class CollisionFreeVectorTable;
 class InsertDestination;
 class LIPFilterAdaptiveProber;
 class StorageManager;
@@ -198,6 +199,14 @@ class AggregationOperationState {
   void finalizeAggregate(const std::size_t partition_id,
                          InsertDestination *output_destination);
 
+  /**
+   * @brief Get the collision-free vector table used by this aggregation.
+   *
+   * @return The collision-free vector table used by this aggregation.
+   *         Returns NULL if collision-free vector table is not used.
+   */
+  CollisionFreeVectorTable* getCollisionFreeVectorTable() const;
+
  private:
   // Check whether partitioned aggregation can be applied.
   bool checkAggregatePartitioned(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/storage/CollisionFreeVectorTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.hpp b/storage/CollisionFreeVectorTable.hpp
index 4f3e238..772d47d 100644
--- a/storage/CollisionFreeVectorTable.hpp
+++ b/storage/CollisionFreeVectorTable.hpp
@@ -105,6 +105,15 @@ class CollisionFreeVectorTable : public AggregationStateHashTableBase {
   }
 
   /**
+   * @brief Get the existence map for this vector table.
+   *
+   * @return The existence map for this vector table.
+   */
+  inline BarrieredReadWriteConcurrentBitVector* getExistenceMap() const {
+    return existence_map_.get();
+  }
+
+  /**
    * @brief Initialize the specified partition of this aggregation table.
    *
    * @param partition_id ID of the partition to be initialized.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/utility/lip_filter/BitVectorExactFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/BitVectorExactFilter.hpp b/utility/lip_filter/BitVectorExactFilter.hpp
index 6ad0567..48fd5e1 100644
--- a/utility/lip_filter/BitVectorExactFilter.hpp
+++ b/utility/lip_filter/BitVectorExactFilter.hpp
@@ -20,17 +20,16 @@
 #ifndef QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_
 #define QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_
 
-#include <atomic>
+#include <cstddef>
 #include <cstdint>
-#include <cstring>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageConstants.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "storage/ValueAccessorUtil.hpp"
 #include "types/Type.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
 #include "utility/Macros.hpp"
 #include "utility/lip_filter/LIPFilter.hpp"
 
@@ -64,14 +63,10 @@ class BitVectorExactFilter : public LIPFilter {
       : LIPFilter(LIPFilterType::kBitVectorExactFilter),
         min_value_(static_cast<CppType>(min_value)),
         max_value_(static_cast<CppType>(max_value)),
-        bit_array_(GetByteSize(max_value - min_value + 1)) {
+        bit_vector_(max_value - min_value + 1) {
     DCHECK_EQ(min_value, static_cast<std::int64_t>(min_value_));
     DCHECK_EQ(max_value, static_cast<std::int64_t>(max_value_));
     DCHECK_GE(max_value_, min_value_);
-
-    std::memset(bit_array_.data(),
-                0x0,
-                sizeof(std::atomic<std::uint8_t>) * GetByteSize(max_value - min_value + 1));
   }
 
   void insertValueAccessor(ValueAccessor *accessor,
@@ -109,13 +104,6 @@ class BitVectorExactFilter : public LIPFilter {
 
  private:
   /**
-   * @brief Round up bit_size to multiples of 8.
-   */
-  inline static std::size_t GetByteSize(const std::size_t bit_size) {
-    return (bit_size + 7u) / 8u;
-  }
-
-  /**
    * @brief Iterate through the accessor and hash values into the internal bit
    *        array.
    */
@@ -164,8 +152,7 @@ class BitVectorExactFilter : public LIPFilter {
     DCHECK_GE(value, min_value_);
     DCHECK_LE(value, max_value_);
 
-    const CppType loc = value - min_value_;
-    bit_array_[loc >> 3u].fetch_or(1u << (loc & 7u), std::memory_order_relaxed);
+    bit_vector_.setBit(value - min_value_);
   }
 
   /**
@@ -177,9 +164,7 @@ class BitVectorExactFilter : public LIPFilter {
       return is_anti_filter;
     }
 
-    const CppType loc = value - min_value_;
-    const bool is_bit_set =
-        (bit_array_[loc >> 3u].load(std::memory_order_relaxed) & (1u << (loc & 7u))) != 0;
+    const bool is_bit_set = bit_vector_.getBit(value - min_value_);
 
     if (is_anti_filter) {
       return !is_bit_set;
@@ -190,7 +175,7 @@ class BitVectorExactFilter : public LIPFilter {
 
   const CppType min_value_;
   const CppType max_value_;
-  alignas(kCacheLineBytes) std::vector<std::atomic<std::uint8_t>> bit_array_;
+  BarrieredReadWriteConcurrentBitVector bit_vector_;
 
   DISALLOW_COPY_AND_ASSIGN(BitVectorExactFilter);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/utility/lip_filter/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/lip_filter/CMakeLists.txt b/utility/lip_filter/CMakeLists.txt
index edd0d24..519d3e9 100644
--- a/utility/lip_filter/CMakeLists.txt
+++ b/utility/lip_filter/CMakeLists.txt
@@ -35,12 +35,12 @@ add_library(quickstep_utility_lipfilter_SingleIdentityHashFilter ../../empty_src
 target_link_libraries(quickstep_utility_lipfilter_BitVectorExactFilter
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageConstants
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
                       quickstep_types_Type
-                      quickstep_utility_lipfilter_LIPFilter
-                      quickstep_utility_Macros)
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_Macros
+                      quickstep_utility_lipfilter_LIPFilter)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilter
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
@@ -79,9 +79,9 @@ target_link_libraries(quickstep_utility_lipfilter_LIPFilter_proto
 target_link_libraries(quickstep_utility_lipfilter_SingleIdentityHashFilter
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageConstants
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
                       quickstep_types_Type
-                      quickstep_utility_lipfilter_LIPFilter
-                      quickstep_utility_Macros)
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_Macros
+                      quickstep_utility_lipfilter_LIPFilter)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/utility/lip_filter/SingleIdentityHashFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/SingleIdentityHashFilter.hpp b/utility/lip_filter/SingleIdentityHashFilter.hpp
index 5c0e8a2..d7e3475 100644
--- a/utility/lip_filter/SingleIdentityHashFilter.hpp
+++ b/utility/lip_filter/SingleIdentityHashFilter.hpp
@@ -20,18 +20,15 @@
 #ifndef QUICKSTEP_UTILITY_LIP_FILTER_SINGLE_IDENTITY_HASH_FILTER_HPP_
 #define QUICKSTEP_UTILITY_LIP_FILTER_SINGLE_IDENTITY_HASH_FILTER_HPP_
 
-#include <atomic>
 #include <cstddef>
-#include <cstdint>
-#include <cstring>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageConstants.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "storage/ValueAccessorUtil.hpp"
 #include "types/Type.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
 #include "utility/Macros.hpp"
 #include "utility/lip_filter/LIPFilter.hpp"
 
@@ -65,11 +62,8 @@ class SingleIdentityHashFilter : public LIPFilter {
   explicit SingleIdentityHashFilter(const std::size_t filter_cardinality)
       : LIPFilter(LIPFilterType::kSingleIdentityHashFilter),
         filter_cardinality_(filter_cardinality),
-        bit_array_(GetByteSize(filter_cardinality)) {
+        bit_vector_(filter_cardinality) {
     DCHECK_GE(filter_cardinality, 1u);
-    std::memset(bit_array_.data(),
-                0x0,
-                sizeof(std::atomic<std::uint8_t>) * GetByteSize(filter_cardinality));
   }
 
   void insertValueAccessor(ValueAccessor *accessor,
@@ -158,8 +152,9 @@ class SingleIdentityHashFilter : public LIPFilter {
    * @brief Inserts a given value into the hash filter.
    */
   inline void insert(const void *key_begin) {
-    const CppType hash = *reinterpret_cast<const CppType *>(key_begin) % filter_cardinality_;
-    bit_array_[hash >> 3u].fetch_or(1u << (hash & 7u), std::memory_order_relaxed);
+    const CppType hash =
+        *reinterpret_cast<const CppType *>(key_begin) % filter_cardinality_;
+    bit_vector_.setBit(hash);
   }
 
   /**
@@ -168,12 +163,13 @@ class SingleIdentityHashFilter : public LIPFilter {
    *        If false is returned, a value is certainly not present in the hash filter.
    */
   inline bool contains(const void *key_begin) const {
-    const CppType hash = *reinterpret_cast<const CppType *>(key_begin) % filter_cardinality_;
-    return (bit_array_[hash >> 3u].load(std::memory_order_relaxed) & (1u << (hash & 7u)));
+    const CppType hash =
+        *reinterpret_cast<const CppType *>(key_begin) % filter_cardinality_;
+    return bit_vector_.getBit(hash);
   }
 
   std::size_t filter_cardinality_;
-  alignas(kCacheLineBytes) std::vector<std::atomic<std::uint8_t>> bit_array_;
+  BarrieredReadWriteConcurrentBitVector bit_vector_;
 
   DISALLOW_COPY_AND_ASSIGN(SingleIdentityHashFilter);
 };


[43/50] [abbrv] incubator-quickstep git commit: Fixed a bug in the distributed version to check rebuild finished.

Posted by ji...@apache.org.
Fixed a bug in the distributed version to check rebuild finished.


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

Branch: refs/heads/LIP-time-decomposition
Commit: c40c5534cbf6f8bc5f1dd5adc43ba705021a8c74
Parents: 8229994
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 9 15:41:04 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 15:41:04 2017 -0800

----------------------------------------------------------------------
 query_execution/QueryManagerDistributed.hpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c40c5534/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 759fa70..14401a6 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -152,7 +152,7 @@ class QueryManagerDistributed final : public QueryManagerBase {
 
   bool checkRebuildOver(const dag_node_index index) const override {
     return query_exec_state_->hasRebuildInitiated(index) &&
-           (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
+           query_exec_state_->hasRebuildFinished(index, num_shiftbosses_);
   }
 
   const tmb::client_id foreman_client_id_;


[30/50] [abbrv] incubator-quickstep git commit: Reduce the number of group-by attributes by pulling tables up aggregations

Posted by ji...@apache.org.
Reduce the number of group-by attributes by pulling tables up aggregations


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

Branch: refs/heads/LIP-time-decomposition
Commit: f46ae1512e5b4665b855d9b3201d42773fa899e8
Parents: 2d89e4f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sun Jan 29 18:36:14 2017 -0600
Committer: Jignesh Patel <jm...@hotmail.com>
Committed: Tue Feb 7 17:45:58 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/Optimizer.cpp                   |   3 +-
 query_optimizer/Optimizer.hpp                   |   2 -
 query_optimizer/PhysicalGenerator.cpp           |   3 +
 query_optimizer/PhysicalGenerator.hpp           |  11 +-
 query_optimizer/rules/CMakeLists.txt            |  23 ++
 .../rules/ReduceGroupByAttributes.cpp           | 217 +++++++++++++++++++
 .../rules/ReduceGroupByAttributes.hpp           | 143 ++++++++++++
 query_optimizer/tests/OptimizerTest.cpp         |   2 +-
 .../tests/OptimizerTextTestRunner.cpp           |   7 +-
 .../tests/OptimizerTextTestRunner.hpp           |   3 +-
 11 files changed, 405 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 8b8fa3c..e750a1e 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -216,6 +216,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+                      quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/Optimizer.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.cpp b/query_optimizer/Optimizer.cpp
index b14c938..1b91574 100644
--- a/query_optimizer/Optimizer.cpp
+++ b/query_optimizer/Optimizer.cpp
@@ -30,10 +30,11 @@ void Optimizer::generateQueryHandle(const ParseStatement &parse_statement,
                                     OptimizerContext *optimizer_context,
                                     QueryHandle *query_handle) {
   LogicalGenerator logical_generator(optimizer_context);
+  PhysicalGenerator physical_generator(optimizer_context);
   ExecutionGenerator execution_generator(catalog_database, query_handle);
 
   execution_generator.generatePlan(
-      physical_generator_.generatePlan(
+      physical_generator.generatePlan(
           logical_generator.generatePlan(*catalog_database, parse_statement)));
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/Optimizer.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.hpp b/query_optimizer/Optimizer.hpp
index 36f956a..227dd04 100644
--- a/query_optimizer/Optimizer.hpp
+++ b/query_optimizer/Optimizer.hpp
@@ -70,8 +70,6 @@ class Optimizer {
                            QueryHandle *query_handle);
 
  private:
-  PhysicalGenerator physical_generator_;
-
   DISALLOW_COPY_AND_ASSIGN(Optimizer);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 5dc0ffb..1b68f49 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/rules/InjectJoinFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
+#include "query_optimizer/rules/ReduceGroupByAttributes.hpp"
 #include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
@@ -127,6 +128,8 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   // general FusePhysical optimization) in the future.
   rules.emplace_back(new PushDownLowCostDisjunctivePredicate());
 
+  rules.emplace_back(new ReduceGroupByAttributes(optimizer_context_));
+
   if (FLAGS_reorder_hash_joins) {
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
     rules.emplace_back(new PruneColumns());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/PhysicalGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.hpp b/query_optimizer/PhysicalGenerator.hpp
index 886a173..42fea86 100644
--- a/query_optimizer/PhysicalGenerator.hpp
+++ b/query_optimizer/PhysicalGenerator.hpp
@@ -33,6 +33,8 @@
 namespace quickstep {
 namespace optimizer {
 
+class OptimizerContext;
+
 /** \addtogroup QueryOptimizer
  *  @{
  */
@@ -43,9 +45,12 @@ namespace optimizer {
 class PhysicalGenerator : public LogicalToPhysicalMapper {
  public:
   /**
-   * @brief Constructor
+   * @brief Constructor.
+   *
+   * @param optimizer_context The optimizer context.
    */
-  PhysicalGenerator() {
+  explicit PhysicalGenerator(OptimizerContext *optimizer_context)
+      : optimizer_context_(optimizer_context) {
     createStrategies();
   }
 
@@ -125,6 +130,8 @@ class PhysicalGenerator : public LogicalToPhysicalMapper {
    */
   std::unordered_map<logical::LogicalPtr, physical::PhysicalPtr> logical_to_physical_map_;
 
+  OptimizerContext *optimizer_context_;
+
   /**
    * @brief The complete physical plan.
    */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 223c78c..029d816 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -29,6 +29,9 @@ add_library(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
             PushDownLowCostDisjunctivePredicate.cpp
             PushDownLowCostDisjunctivePredicate.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
+add_library(quickstep_queryoptimizer_rules_ReduceGroupByAttributes
+            ReduceGroupByAttributes.cpp
+            ReduceGroupByAttributes.hpp)
 add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
@@ -143,6 +146,25 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ReduceGroupByAttributes
+                      ${GFLAGS_LIB_NAME}
+                      quickstep_catalog_CatalogRelation
+                      quickstep_queryoptimizer_OptimizerContext
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExprId
@@ -272,6 +294,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
+                      quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/ReduceGroupByAttributes.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReduceGroupByAttributes.cpp b/query_optimizer/rules/ReduceGroupByAttributes.cpp
new file mode 100644
index 0000000..dcdd27a
--- /dev/null
+++ b/query_optimizer/rules/ReduceGroupByAttributes.cpp
@@ -0,0 +1,217 @@
+/**
+ * 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 "query_optimizer/rules/ReduceGroupByAttributes.hpp"
+
+#include <algorithm>
+#include <map>
+#include <vector>
+#include <unordered_set>
+#include <utility>
+
+#include "catalog/CatalogRelation.hpp"
+#include "query_optimizer/OptimizerContext.hpp"
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/Aggregate.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/TableReference.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/rules/PruneColumns.hpp"
+
+#include "gflags/gflags.h"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+DEFINE_uint64(reduce_group_by_attributes_threshold, 3u,
+              "The threshold for a stored relation's number of attributes in a "
+              "group-by clause for the ReduceGroupByAttributes optimization "
+              "rule to pull the stored relation up the aggregation");
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr ReduceGroupByAttributes::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()));
+
+  P::PhysicalPtr output = applyInternal(input);
+  if (output != input) {
+    output = PruneColumns().apply(output);
+  }
+  return output;
+}
+
+P::PhysicalPtr ReduceGroupByAttributes::applyInternal(const P::PhysicalPtr &input) {
+  std::vector<P::PhysicalPtr> new_children;
+  for (const P::PhysicalPtr &child : input->children()) {
+    new_children.push_back(applyInternal(child));
+  }
+
+  if (new_children != input->children()) {
+    return applyToNode(input->copyWithNewChildren(new_children));
+  } else {
+    return applyToNode(input);
+  }
+}
+
+P::PhysicalPtr ReduceGroupByAttributes::applyToNode(const P::PhysicalPtr &input) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(input, &table_reference)) {
+    // Collect the attributes-to-TableReference mapping info.
+    for (const auto &attr : table_reference->attribute_list()) {
+      source_.emplace(attr->id(), std::make_pair(table_reference, attr));
+    }
+    return input;
+  }
+
+  P::AggregatePtr aggregate;
+  if (!P::SomeAggregate::MatchesWithConditionalCast(input, &aggregate) ||
+      aggregate->grouping_expressions().size() <= 1u) {
+    return input;
+  }
+
+  // Divide the group-by attributes into groups based on their source table.
+  std::map<P::TableReferencePtr, std::vector<E::AttributeReferencePtr>> table_attributes;
+  for (const auto &expr : aggregate->grouping_expressions()) {
+    const auto source_it = source_.find(expr->id());
+    if (source_it != source_.end()) {
+      table_attributes[source_it->second.first].emplace_back(source_it->second.second);
+    }
+  }
+
+  std::unordered_set<E::ExprId> erased_grouping_attr_ids;
+  std::vector<std::pair<P::TableReferencePtr, E::AttributeReferencePtr>> hoisted_tables;
+
+  // For each group (i.e. each source table), if it is profitable then we pull
+  // the table up the aggregation.
+  for (const auto &pair : table_attributes) {
+    const P::TableReferencePtr table = pair.first;
+    const std::vector<E::AttributeReferencePtr> &attributes = pair.second;
+    // TODO(jianqiao): find a cost-based metic instead of hard-coding the threshold
+    // number of group-by attributes.
+    if (attributes.size() <= FLAGS_reduce_group_by_attributes_threshold) {
+      continue;
+    }
+
+    std::vector<AttributeInfo> attr_infos;
+    for (const auto &attr : attributes) {
+      attr_infos.emplace_back(attr,
+                              cost_model_->impliesUniqueAttributes(table, {attr}),
+                              !attr->getValueType().isVariableLength(),
+                              attr->getValueType().maximumByteLength());
+    }
+
+    std::vector<const AttributeInfo *> attr_info_refs;
+    for (const auto &info : attr_infos) {
+      attr_info_refs.emplace_back(&info);
+    }
+    std::sort(attr_info_refs.begin(),
+              attr_info_refs.end(),
+              AttributeInfo::IsBetterThan);
+
+    const AttributeInfo &best_candidate = *attr_info_refs.front();
+    if (!best_candidate.is_unique) {
+      // Cannot find a key attribute. Give up pulling this table up.
+      continue;
+    }
+
+    const E::AttributeReferencePtr key_attribute = best_candidate.attribute;
+    hoisted_tables.emplace_back(table, key_attribute);
+
+    for (const auto &attr : attributes) {
+      if (attr->id() != key_attribute->id()) {
+        erased_grouping_attr_ids.emplace(attr->id());
+      }
+    }
+  }
+
+  if (erased_grouping_attr_ids.empty()) {
+    return input;
+  }
+
+  // Reconstuct the Aggregate node with reduced group-by attributes and then
+  // construct HashJoin nodes on top of the Aggregate.
+  std::vector<E::NamedExpressionPtr> reduced_grouping_expressions;
+  for (const auto &expr : aggregate->grouping_expressions()) {
+    if (erased_grouping_attr_ids.find(expr->id()) == erased_grouping_attr_ids.end()) {
+      reduced_grouping_expressions.emplace_back(expr);
+    }
+  }
+
+  const P::AggregatePtr new_aggregate =
+      P::Aggregate::Create(aggregate->input(),
+                           reduced_grouping_expressions,
+                           aggregate->aggregate_expressions(),
+                           aggregate->filter_predicate());
+
+  P::PhysicalPtr output = new_aggregate;
+  std::vector<E::NamedExpressionPtr> project_expressions =
+      E::ToNamedExpressions(output->getOutputAttributes());
+  for (const auto &pair : hoisted_tables) {
+    const P::TableReferencePtr &source_table = pair.first;
+    const E::AttributeReferencePtr &probe_attribute = pair.second;
+
+    E::AttributeReferencePtr build_attribute;
+    std::vector<E::AttributeReferencePtr> new_attribute_list;
+    for (const auto &attr : source_table->attribute_list()) {
+      if (attr->id() == probe_attribute->id()) {
+        build_attribute =
+          E::AttributeReference::Create(optimizer_context_->nextExprId(),
+                                        attr->attribute_name(),
+                                        attr->attribute_alias(),
+                                        attr->relation_name(),
+                                        attr->getValueType(),
+                                        E::AttributeReferenceScope::kLocal);
+        new_attribute_list.emplace_back(build_attribute);
+      } else {
+        new_attribute_list.emplace_back(attr);
+        project_expressions.emplace_back(attr);
+      }
+    }
+
+    DCHECK(build_attribute != nullptr);
+    const P::TableReferencePtr build_side_table =
+        P::TableReference::Create(source_table->relation(),
+                                  source_table->relation()->getName(),
+                                  new_attribute_list);
+    output = P::HashJoin::Create(output,
+                                 build_side_table,
+                                 {probe_attribute},
+                                 {build_attribute},
+                                 nullptr,
+                                 project_expressions,
+                                 P::HashJoin::JoinType::kInnerJoin);
+  }
+
+  return output;
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/ReduceGroupByAttributes.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReduceGroupByAttributes.hpp b/query_optimizer/rules/ReduceGroupByAttributes.hpp
new file mode 100644
index 0000000..5a1f295
--- /dev/null
+++ b/query_optimizer/rules/ReduceGroupByAttributes.hpp
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+class OptimizerContext;
+
+/**
+ * @brief Rule that applies to a physical plan to reduce the number of group-by
+ *        attributes for Aggregate nodes (to improve performance) by pulling
+ *        joins up the aggregations.
+ *
+ * For example, let R be a relation with PRIMARY KEY x and attributes y, z. Let
+ * S be a relation with FOREIGN KEY u refering to R(x) and attribute v. Then the
+ * optimization rule will transform the physical plan:
+ *   Aggregate(
+ *     [input relation]: HashJoin(
+ *                         [probe relation]: S
+ *                         [build relation]: R
+ *                         [join expression]: S.u = R.x
+ *                         [project attributes]: v, x, y, z
+ *                       )
+ *     [aggregate expression]: SUM(v) AS sum_v
+ *     [group-by attributes]: x, y, z
+ *   )
+ *
+ * into:
+ *   HashJoin(
+ *     [probe relation]: Aggregate(
+ *                         [input relation]: S
+ *                         [aggregate expression]: SUM(v) AS sum_v
+ *                         [group-by attribute]: u
+ *                       ) AS T
+ *     [build relation]: R
+ *     [join expression]: T.u = R.x
+ *     [project attributes]: sum_v, x, y, z
+ *   )
+ */
+class ReduceGroupByAttributes : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param optimizer_context The optimizer context.
+   */
+  explicit ReduceGroupByAttributes(OptimizerContext *optimizer_context)
+      : optimizer_context_(optimizer_context) {}
+
+  ~ReduceGroupByAttributes() override {}
+
+  std::string getName() const override {
+    return "ReduceGroupByAttributes";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct AttributeInfo {
+    AttributeInfo(const expressions::AttributeReferencePtr &attribute_in,
+                  const bool is_unique_in,
+                  const bool is_fixed_length_in,
+                  const std::size_t maximum_size_in)
+        : attribute(attribute_in),
+          is_unique(is_unique_in),
+          is_fixed_length(is_fixed_length_in),
+          maximum_size(maximum_size_in) {}
+
+    // In the situation that there are multiple attributes that can serve as the
+    // group-by key, we define an ordering based on aggregation performance (e.g.
+    // it is faster to do aggregation with a fix-length attribute as the group-by
+    // key than with a variable-length attribute).
+    inline static bool IsBetterThan(const AttributeInfo *lhs,
+                                    const AttributeInfo *rhs) {
+      if (lhs->is_unique != rhs->is_unique) {
+        return lhs->is_unique;
+      }
+      if (lhs->is_fixed_length != rhs->is_fixed_length) {
+        return lhs->is_fixed_length;
+      }
+      if (lhs->maximum_size != rhs->maximum_size) {
+        return lhs->maximum_size < rhs->maximum_size;
+      }
+      return lhs->attribute->id() < rhs->attribute->id();
+    }
+
+    const expressions::AttributeReferencePtr attribute;
+    const bool is_unique;
+    const bool is_fixed_length;
+    const std::size_t maximum_size;
+  };
+
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input);
+  physical::PhysicalPtr applyToNode(const physical::PhysicalPtr &input);
+
+  OptimizerContext *optimizer_context_;
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  // Maps an attribute's id to the TableReference that generates the attribute.
+  std::unordered_map<expressions::ExprId,
+                     std::pair<physical::TableReferencePtr,
+                               expressions::AttributeReferencePtr>> source_;
+
+  DISALLOW_COPY_AND_ASSIGN(ReduceGroupByAttributes);
+};
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTest.cpp b/query_optimizer/tests/OptimizerTest.cpp
index 3838638..7eb7a11 100644
--- a/query_optimizer/tests/OptimizerTest.cpp
+++ b/query_optimizer/tests/OptimizerTest.cpp
@@ -62,7 +62,7 @@ OptimizerTest::OptimizerTest()
       catalog_database_(
           new CatalogDatabase(catalog_.get(), "TestDatabase" /* name */, 0)),
       optimizer_context_(new OptimizerContext),
-      physical_generator_(new PhysicalGenerator()) {}
+      physical_generator_(new PhysicalGenerator(optimizer_context_.get())) {}
 
 E::AliasPtr OptimizerTest::createAlias(const E::ExpressionPtr &expression,
                                        const std::string &attribute_name,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTextTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.cpp b/query_optimizer/tests/OptimizerTextTestRunner.cpp
index b9238c9..cb8f153 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.cpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.cpp
@@ -80,7 +80,7 @@ void OptimizerTextTestRunner::runTestCase(const std::string &input,
       }
       if (output_physical_plan) {
         physical_plan =
-            generatePhysicalPlan(optimized_logical_plan);
+            generatePhysicalPlan(optimized_logical_plan, &optimizer_context);
         ++num_options;
       }
 
@@ -126,8 +126,9 @@ logical::LogicalPtr OptimizerTextTestRunner::generateLogicalPlan(
 }
 
 physical::PhysicalPtr OptimizerTextTestRunner::generatePhysicalPlan(
-    const logical::LogicalPtr &logical_plan) {
-  PhysicalGenerator physical_generator;
+    const logical::LogicalPtr &logical_plan,
+    OptimizerContext *optimizer_context) {
+  PhysicalGenerator physical_generator(optimizer_context);
   return physical_generator.generatePlan(logical_plan);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTextTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.hpp b/query_optimizer/tests/OptimizerTextTestRunner.hpp
index 27fa14f..d8f604b 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.hpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.hpp
@@ -73,7 +73,8 @@ class OptimizerTextTestRunner : public TextBasedTestRunner {
                                           OptimizerContext *optimizer_context);
 
   physical::PhysicalPtr generatePhysicalPlan(
-      const logical::LogicalPtr &logical_plan);
+      const logical::LogicalPtr &logical_plan,
+      OptimizerContext *optimizer_context);
 
   SqlParserWrapper sql_parser_;
   TestDatabaseLoader test_database_loader_;


[24/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
deleted file mode 100644
index 4a82a62..0000000
--- a/storage/FastHashTable.hpp
+++ /dev/null
@@ -1,2403 +0,0 @@
-/**
- * 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_STORAGE_FAST_HASH_TABLE_HPP_
-#define QUICKSTEP_STORAGE_FAST_HASH_TABLE_HPP_
-
-#include <atomic>
-#include <cstddef>
-#include <cstdlib>
-#include <type_traits>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTableBase.hpp"
-#include "storage/StorageBlob.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageConstants.hpp"
-#include "storage/StorageManager.hpp"
-#include "storage/TupleReference.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "threading/SpinMutex.hpp"
-#include "threading/SpinSharedMutex.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/HashPair.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-/** \addtogroup Storage
- *  @{
- */
-
-/**
- * @brief Base class for the hash table implementation in which the payload can
- *        be just a bunch of bytes. This implementation is suitable for
- *        aggregation hash table with multiple aggregation handles (e.g. SUM,
- *        MAX, MIN etc).
- *
- * At present there is one implementation for this base class.
- *      1. SeparateChainingHashTable - Keys/values are stored in a separate
- *         region of memory from the base hash table slot array. Every bucket
- *         has a "next" pointer so that entries that collide (i.e. map to the
- *         same base slot) form chains of pointers with each other. Although
- *         this implementation has some extra indirection compared to
- *         LinearOpenAddressingHashTable, it does not have the same
- *         vulnerabilities to key skew, and it additionally supports a very
- *         efficient bucket-preallocation mechanism that minimizes cache
- *         coherency overhead when multiple threads are building a HashTable.
- *
- * @note If you need to create a HashTable and not just use it as a client, see
- *       HashTableFactory, which simplifies the process of creating a
- *       HashTable.
- *
- * @param resizable Whether this hash table is resizable (using memory from a
- *        StorageManager) or not (using a private, fixed memory allocation).
- * @param serializable If true, this hash table can safely be saved to and
- *        loaded from disk. If false, some out of band memory may be used (e.g.
- *        to store variable length keys).
- * @param force_key_copy If true, inserted keys are always copied into this
- *        HashTable's memory. If false, pointers to external values may be
- *        stored instead. force_key_copy should be true if the hash table will
- *        outlive the external key values which are inserted into it. Note that
- *        if serializable is true and force_key_copy is false, then relative
- *        offsets will be used instead of absolute pointers to keys, meaning
- *        that the pointed-to keys must be serialized and deserialized in
- *        exactly the same relative byte order (e.g. as part of the same
- *        StorageBlock), and keys must not change position relative to this
- *        HashTable (beware TupleStorageSubBlocks that may self-reorganize when
- *        modified). If serializable and resizable are both true, then
- *        force_key_copy must also be true.
- * @param allow_duplicate_keys If true, multiple values can be mapped to the
- *        same key. If false, one and only one value may be mapped.
- **/
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-class FastHashTable : public HashTableBase<resizable,
-                                           serializable,
-                                           force_key_copy,
-                                           allow_duplicate_keys> {
-  static_assert(!(serializable && resizable && !force_key_copy),
-                "A HashTable must have force_key_copy=true when serializable "
-                "and resizable are both true.");
-
- public:
-  // Shadow template parameters. This is useful for shared test harnesses.
-  static constexpr bool template_resizable = resizable;
-  static constexpr bool template_serializable = serializable;
-  static constexpr bool template_force_key_copy = force_key_copy;
-  static constexpr bool template_allow_duplicate_keys = allow_duplicate_keys;
-
-  // Some HashTable implementations (notably LinearOpenAddressingHashTable)
-  // use a special hash code to represent an empty bucket, and another special
-  // code to indicate that a bucket is currently being inserted into. For those
-  // HashTables, this is a surrogate hash value for empty buckets. Keys which
-  // actually hash to this value should have their hashes mutated (e.g. by
-  // adding 1). We use zero, since we will often be using memory which is
-  // already zeroed-out and this saves us the trouble of a memset. This has
-  // some downside, as the hash function we use is the identity hash for
-  // integers, and the integer 0 is common in many data sets and must be
-  // adjusted (and will then spuriously collide with 1). Nevertheless, this
-  // expense is outweighed by no longer having to memset large regions of
-  // memory when initializing a HashTable.
-  static constexpr unsigned char kEmptyHashByte = 0x0;
-  static constexpr std::size_t kEmptyHash = 0x0;
-
-  // A surrogate hash value for a bucket which is currently being inserted
-  // into. As with kEmptyHash, keys which actually hash to this value should
-  // have their hashes adjusted.
-  static constexpr std::size_t kPendingHash = ~kEmptyHash;
-
-  /**
-   * @brief Virtual destructor.
-   **/
-  virtual ~FastHashTable() {
-    if (resizable) {
-      if (blob_.valid()) {
-        if (serializable) {
-          DEV_WARNING(
-              "Destroying a resizable serializable HashTable's underlying "
-              "StorageBlob.");
-        }
-        const block_id blob_id = blob_->getID();
-        blob_.release();
-        storage_manager_->deleteBlockOrBlobFile(blob_id);
-      }
-    }
-  }
-
-  /**
-   * @brief Get the ID of the StorageBlob used to store a resizable HashTable.
-   *
-   * @warning This method must not be used for a non-resizable HashTable.
-   *
-   * @return The ID of the StorageBlob used to store this HashTable.
-   **/
-  inline block_id getBlobId() const {
-    DEBUG_ASSERT(resizable);
-    return blob_->getID();
-  }
-
-  /**
-   * @brief Erase all entries in this hash table.
-   *
-   * @warning This method is not guaranteed to be threadsafe.
-   **/
-  virtual void clear() = 0;
-
-  /**
-   * @brief Add a new entry into the hash table.
-   *
-   * @warning The key must not be null.
-   * @warning This method is threadsafe with regard to other calls to put(),
-   *          putCompositeKey(), putValueAccessor(), and
-   *          putValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
-   * @note This version is for single scalar keys, see also putCompositeKey().
-   * @note If the hash table is (close to) full and resizable is true, this
-   *       routine might result in rebuilding the entire hash table.
-   *
-   * @param key The key.
-   * @param value The value payload.
-   * @return HashTablePutResult::kOK if an entry was successfully inserted,
-   *         HashTablePutResult::kDuplicateKey if allow_duplicate_keys is false
-   *         and key was a duplicate, or HashTablePutResult::kOutOfSpace if
-   *         resizable is false and storage space for the hash table has been
-   *         exhausted.
-   **/
-  HashTablePutResult put(const TypedValue &key, const std::uint8_t &value);
-
-  /**
-   * @brief Add a new entry into the hash table (composite key version).
-   *
-   * @warning No component of the key may be null.
-   * @warning This method is threadsafe with regard to other calls to put(),
-   *          putCompositeKey(), putValueAccessor(), and
-   *          putValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
-   * @note This version is for composite keys, see also put().
-   * @note If the hash table is (close to) full and resizable is true, this
-   *       routine might result in rebuilding the entire hash table.
-   *
-   * @param key The components of the key.
-   * @param value The value payload.
-   * @return HashTablePutResult::kOK if an entry was successfully inserted,
-   *         HashTablePutResult::kDuplicateKey if allow_duplicate_keys is false
-   *         and key was a duplicate, or HashTablePutResult::kOutOfSpace if
-   *         resizable is false and storage space for the hash table has been
-   *         exhausted.
-   **/
-
-  HashTablePutResult putCompositeKey(const std::vector<TypedValue> &key,
-                                         const std::uint8_t *value_ptr);
-
-  /**
-   * @brief Add (multiple) new entries into the hash table from a
-   *        ValueAccessor.
-   *
-   * @warning This method is threadsafe with regard to other calls to put(),
-   *          putCompositeKey(), putValueAccessor(), and
-   *          putValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
-   * @note This version is for scalar keys, see also
-   *       putValueAccessorCompositeKey().
-   * @note If the hash table fills up while this call is in progress and
-   *       resizable is true, this might result in rebuilding the entire hash
-   *       table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before inserting it (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator that takes const ValueAccessor& as an argument (or better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as an argument) and returns either
-   *        a ValueT or a reference to a ValueT. The functor should generate
-   *        the appropriate mapped value for the current tuple the accessor is
-   *        iterating on.
-   * @return HashTablePutResult::kOK if all keys and generated values from
-   *         accessor were successfully inserted.
-   *         HashTablePutResult::kOutOfSpace is returned if this hash-table is
-   *         non-resizable and ran out of space (note that some entries may
-   *         still have been inserted, and accessor's iteration will be left on
-   *         the first tuple which could not be inserted).
-   *         HashTablePutResult::kDuplicateKey is returned if
-   *         allow_duplicate_keys is false and a duplicate key is encountered
-   *         (as with HashTablePutResult::kOutOfSpace, some entries may have
-   *         been inserted, and accessor will be left on the tuple with a
-   *         duplicate key).
-   **/
-  template <typename FunctorT>
-  HashTablePutResult putValueAccessor(ValueAccessor *accessor,
-                                      const attribute_id key_attr_id,
-                                      const bool check_for_null_keys,
-                                      FunctorT *functor);
-
-  /**
-   * @brief Add (multiple) new entries into the hash table from a
-   *        ValueAccessor (composite key version).
-   *
-   * @warning This method is threadsafe with regard to other calls to put(),
-   *          putCompositeKey(), putValueAccessor(), and
-   *          putValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
-   * @note This version is for composite keys, see also putValueAccessor().
-   * @note If the hash table fills up while this call is in progress and
-   *       resizable is true, this might result in rebuilding the entire hash
-   *       table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_ids The attribute IDs of each key component to be read
-   *        from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        has a null component before inserting it (null keys are skipped).
-   *        This must be set to true if some of the keys that will be read from
-   *        accessor may be null.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator that takes const ValueAccessor& as an argument (or better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as an argument) and returns either
-   *        a ValueT or a reference to a ValueT. The functor should generate
-   *        the appropriate mapped value for the current tuple the accessor is
-   *        iterating on.
-   * @return HashTablePutResult::kOK if all keys and generated values from
-   *         accessor were successfully inserted.
-   *         HashTablePutResult::kOutOfSpace is returned if this hash-table is
-   *         non-resizable and ran out of space (note that some entries may
-   *         still have been inserted, and accessor's iteration will be left on
-   *         the first tuple which could not be inserted).
-   *         HashTablePutResult::kDuplicateKey is returned if
-   *         allow_duplicate_keys is false and a duplicate key is encountered
-   *         (as with HashTablePutResult::kOutOfSpace, some entries may have
-   *         been inserted, and accessor will be left on the tuple with a
-   *         duplicate key).
-   **/
-  template <typename FunctorT>
-  HashTablePutResult putValueAccessorCompositeKey(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      FunctorT *functor);
-
-  /**
-   * @brief Apply a functor to the value mapped to a key, first inserting a new
-   *        value if one is not already present.
-   *
-   * @warning The key must not be null.
-   * @warning This method is only usable if allow_duplicate_keys is false.
-   * @warning This method is threadsafe with regard to other calls to upsert(),
-   *          upsertCompositeKey(), upsertValueAccessor(), and
-   *          upsertValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
-   *          or putValueAccessorCompositeKey().
-   * @warning The ValueT* pointer passed to functor's call operator is only
-   *          guaranteed to be valid for the duration of the call. The functor
-   *          should not store a copy of the pointer and assume that it remains
-   *          valid.
-   * @warning Although this method itself is threadsafe, the ValueT object
-   *          accessed by functor is not guaranteed to be (although it is
-   *          guaranteed that its initial insertion will be atomic). If it is
-   *          possible for multiple threads to call upsert() with the same key
-   *          at the same time, then their access to ValueT should be made
-   *          threadsafe (e.g. with the use of atomic types, mutexes, or some
-   *          other external synchronization).
-   * @note This version is for single scalar keys, see also
-   *       upsertCompositeKey().
-   * @note If the hash table is (close to) full and resizable is true, this
-   *       routine might result in rebuilding the entire hash table.
-   *
-   * @param key The key.
-   * @param initial_value If there was not already a preexisting entry in this
-   *        HashTable for the specified key, then the value will be initialized
-   *        with a copy of initial_value. This parameter is ignored if a value
-   *        is already present for key.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator which takes ValueT* as an argument. The call operator will
-   *        be invoked once on the value corresponding to key (which may be
-   *        newly inserted and default-constructed).
-   * @return True on success, false if upsert failed because there was not
-   *         enough space to insert a new entry in this HashTable.
-   **/
-  template <typename FunctorT>
-  bool upsert(const TypedValue &key,
-              const std::uint8_t *initial_value_ptr,
-              FunctorT *functor);
-
-  /**
-   * @brief Apply a functor to the value mapped to a key, first inserting a new
-   *        value if one is not already present.
-   *
-   * @warning The key must not be null.
-   * @warning This method is only usable if allow_duplicate_keys is false.
-   * @warning This method is threadsafe with regard to other calls to upsert(),
-   *          upsertCompositeKey(), upsertValueAccessor(), and
-   *          upsertValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
-   *          or putValueAccessorCompositeKey().
-   * @warning The ValueT* pointer passed to functor's call operator is only
-   *          guaranteed to be valid for the duration of the call. The functor
-   *          should not store a copy of the pointer and assume that it remains
-   *          valid.
-   * @warning Although this method itself is threadsafe, the ValueT object
-   *          accessed by functor is not guaranteed to be (although it is
-   *          guaranteed that its initial insertion will be atomic). If it is
-   *          possible for multiple threads to call upsertCompositeKey() with
-   *          the same key at the same time, then their access to ValueT should
-   *          be made threadsafe (e.g. with the use of atomic types, mutexes,
-   *          or some other external synchronization).
-   * @note This version is for composite keys, see also upsert().
-   * @note If the hash table is (close to) full and resizable is true, this
-   *       routine might result in rebuilding the entire hash table.
-   *
-   * @param key The key.
-   * @param initial_value If there was not already a preexisting entry in this
-   *        HashTable for the specified key, then the value will be initialized
-   *        with a copy of initial_value. This parameter is ignored if a value
-   *        is already present for key.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator which takes ValueT* as an argument. The call operator will
-   *        be invoked once on the value corresponding to key (which may be
-   *        newly inserted and default-constructed).
-   * @return True on success, false if upsert failed because there was not
-   *         enough space to insert a new entry in this HashTable.
-   **/
-  template <typename FunctorT>
-  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                              const std::uint8_t *init_value_ptr,
-                              FunctorT *functor);
-
-  template <typename FunctorT>
-  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                              const std::uint8_t *init_value_ptr,
-                              FunctorT *functor,
-                              int index);
-
-  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                              const std::uint8_t *init_value_ptr,
-                              const std::uint8_t *source_state);
-
-  /**
-   * @brief Apply a functor to (multiple) entries in this hash table, with keys
-   *        drawn from a ValueAccessor. New values are first inserted if not
-   *        already present.
-   *
-   * @warning This method is only usable if allow_duplicate_keys is false.
-   * @warning This method is threadsafe with regard to other calls to upsert(),
-   *          upsertCompositeKey(), upsertValueAccessor(), and
-   *          upsertValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
-   *          or putValueAccessorCompositeKey().
-   * @warning The ValueAccessor reference and ValueT* pointer passed to
-   *          functor's call operator are only guaranteed to be valid for the
-   *          duration of the call. The functor should not store a copy of
-   *          these pointers and assume that they remain valid.
-   * @warning Although this method itself is threadsafe, the ValueT object
-   *          accessed by functor is not guaranteed to be (although it is
-   *          guaranteed that its initial insertion will be atomic). If it is
-   *          possible for multiple threads to call upsertValueAccessor() with
-   *          the same key at the same time, then their access to ValueT should
-   *          be made threadsafe (e.g. with the use of atomic types, mutexes,
-   *          or some other external synchronization).
-   * @note This version is for single scalar keys, see also
-   *       upsertValueAccessorCompositeKey().
-   * @note If the hash table is (close to) full and resizable is true, this
-   *       routine might result in rebuilding the entire hash table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before upserting it (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator that takes two arguments: const ValueAccessor& (or better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as its first argument) and ValueT*.
-   *        The call operator will be invoked once for every tuple with a
-   *        non-null key in accessor.
-   * @return True on success, false if upsert failed because there was not
-   *         enough space to insert new entries for all the keys in accessor
-   *         (note that some entries may still have been upserted, and
-   *         accessor's iteration will be left on the first tuple which could
-   *         not be inserted).
-   **/
-  bool upsertValueAccessorFast(
-      const std::vector<attribute_id> &argument_ids,
-      ValueAccessor *accessor,
-      const attribute_id key_attr_id,
-      const bool check_for_null_keys);
-
-  /**
-   * @brief Apply a functor to (multiple) entries in this hash table, with keys
-   *        drawn from a ValueAccessor. New values are first inserted if not
-   *        already present. Composite key version.
-   *
-   * @warning This method is only usable if allow_duplicate_keys is false.
-   * @warning This method is threadsafe with regard to other calls to upsert(),
-   *          upsertCompositeKey(), upsertValueAccessor(), and
-   *          upsertValueAccessorCompositeKey(), but should not be used
-   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
-   *          or putValueAccessorCompositeKey().
-   * @warning The ValueAccessor reference and ValueT* pointer passed to
-   *          functor's call operator are only guaranteed to be valid for the
-   *          duration of the call. The functor should not store a copy of
-   *          these pointers and assume that they remain valid.
-   * @warning Although this method itself is threadsafe, the ValueT object
-   *          accessed by functor is not guaranteed to be (although it is
-   *          guaranteed that its initial insertion will be atomic). If it is
-   *          possible for multiple threads to call upsertValueAccessor() with
-   *          the same key at the same time, then their access to ValueT should
-   *          be made threadsafe (e.g. with the use of atomic types, mutexes,
-   *          or some other external synchronization).
-   * @note This version is for composite keys, see also upsertValueAccessor().
-   * @note If the hash table is (close to) full and resizable is true, this
-   *       routine might result in rebuilding the entire hash table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_ids The attribute IDs of each key component to be read
-   *        from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before upserting it (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator that takes two arguments: const ValueAccessor& (or better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as its first argument) and ValueT*.
-   *        The call operator will be invoked once for every tuple with a
-   *        non-null key in accessor.
-   * @return True on success, false if upsert failed because there was not
-   *         enough space to insert new entries for all the keys in accessor
-   *         (note that some entries may still have been upserted, and
-   *         accessor's iteration will be left on the first tuple which could
-   *         not be inserted).
-   **/
-  bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<attribute_id> &argument,
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) override;
-
-  /**
-   * @brief Determine the number of entries (key-value pairs) contained in this
-   *        HashTable.
-   * @note For some HashTable implementations, this is O(1), but for others it
-   *       may be O(n) where n is the number of buckets.
-   *
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   *
-   * @return The number of entries in this HashTable.
-   **/
-  virtual std::size_t numEntries() const = 0;
-
-  /**
-   * @brief Lookup a key against this hash table to find a matching entry.
-   *
-   * @warning Only usable with the hash table that does not allow duplicate
-   *          keys.
-   * @warning The key must not be null.
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for single scalar keys. See also
-   *       getSingleCompositeKey().
-   *
-   * @param key The key to look up.
-   * @return The value of a matched entry if a matching key is found.
-   *         Otherwise, return NULL.
-   **/
-  virtual const std::uint8_t* getSingle(const TypedValue &key) const = 0;
-
-  /**
-   * @brief Lookup a composite key against this hash table to find a matching
-   *        entry.
-   *
-   * @warning Only usable with the hash table that does not allow duplicate
-   *          keys.
-   * @warning The key must not be null.
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for composite keys. See also getSingle().
-   *
-   * @param key The key to look up.
-   * @return The value of a matched entry if a matching key is found.
-   *         Otherwise, return NULL.
-   **/
-  virtual const std::uint8_t* getSingleCompositeKey(
-      const std::vector<TypedValue> &key) const = 0;
-  virtual const std::uint8_t *getSingleCompositeKey(
-      const std::vector<TypedValue> &key, int index) const = 0;
-
-  /**
-   * @brief Lookup a key against this hash table to find matching entries.
-   *
-   * @warning The key must not be null.
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note It is more efficient to call getSingle() if the hash table does not
-   *       allow duplicate keys.
-   * @note This version is for single scalar keys. See also
-   *       getAllCompositeKey().
-   *
-   * @param key The key to look up.
-   * @param values A vector to hold values of all matching entries. Matches
-   *        will be appended to the vector.
-   **/
-  virtual void getAll(const TypedValue &key,
-                      std::vector<const std::uint8_t *> *values) const = 0;
-
-  /**
-   * @brief Lookup a composite key against this hash table to find matching
-   *        entries.
-   *
-   * @warning The key must not be null.
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note It is more efficient to call getSingleCompositeKey() if the hash
-   *       table does not allow duplicate keys.
-   * @note This version is for composite keys. See also getAll().
-   *
-   * @param key The key to look up.
-   * @param values A vector to hold values of all matching entries. Matches
-   *        will be appended to the vector.
-   **/
-  virtual void getAllCompositeKey(
-      const std::vector<TypedValue> &key,
-      std::vector<const std::uint8_t *> *values) const = 0;
-
-  /**
-   * @brief Lookup (multiple) keys from a ValueAccessor and apply a functor to
-   *        the matching values.
-   *
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for single scalar keys. See also
-   *       getAllFromValueAccessorCompositeKey().
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator that takes 2 arguments: const ValueAccessor& (or better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as its first argument) and
-   *        const ValueT&. The functor will be invoked once for each pair of a
-   *        key taken from accessor and matching value.
-   **/
-  template <typename FunctorT>
-  void getAllFromValueAccessor(ValueAccessor *accessor,
-                               const attribute_id key_attr_id,
-                               const bool check_for_null_keys,
-                               FunctorT *functor) const;
-
-  /**
-   * @brief Lookup (multiple) keys from a ValueAccessor, apply a functor to the
-   *        matching values and additionally call a recordMatch() function of
-   *        the functor when the first match for a key is found.
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for single scalar keys. See also
-   *       getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch().
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor, which should provide two functions:
-   *        1) An operator that takes 2 arguments: const ValueAccessor& (or
-   * better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as its first argument) and
-   *        const ValueT&. The operator will be invoked once for each pair of a
-   *        key taken from accessor and matching value.
-   *        2) A function hasMatch that takes 1 argument: const ValueAccessor&.
-   *        The function will be called only once for a key from accessor when
-   *        the first match is found.
-   */
-  template <typename FunctorT>
-  void getAllFromValueAccessorWithExtraWorkForFirstMatch(
-      ValueAccessor *accessor,
-      const attribute_id key_attr_id,
-      const bool check_for_null_keys,
-      FunctorT *functor) const;
-
-  /**
-   * @brief Lookup (multiple) keys from a ValueAccessor, apply a functor to the
-   *        matching values and additionally call a recordMatch() function of
-   *        the functor when the first match for a key is found. Composite key
-   *        version.
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor, which should provide two functions:
-   *        1) An operator that takes 2 arguments: const ValueAccessor& (or
-   * better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as its first argument) and
-   *        const ValueT&. The operator will be invoked once for each pair of a
-   *        key taken from accessor and matching value.
-   *        2) A function hasMatch that takes 1 argument: const ValueAccessor&.
-   *        The function will be called only once for a key from accessor when
-   *        the first match is found.
-   */
-  template <typename FunctorT>
-  void getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      FunctorT *functor) const;
-
-  /**
-   * @brief Lookup (multiple) keys from a ValueAccessor and apply a functor to
-   *        the matching values. Composite key version.
-   *
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for composite keys. See also
-   *       getAllFromValueAccessor().
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_ids The attribute IDs of each key component to be read
-   *        from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        has a null component before inserting it (null keys are skipped).
-   *        This must be set to true if some of the keys that will be read from
-   *        accessor may be null.
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator that takes 2 arguments: const ValueAccessor& (or better
-   *        yet, a templated call operator which takes a const reference to
-   *        some subclass of ValueAccessor as its first argument) and
-   *        const ValueT&. The functor will be invoked once for each pair of a
-   *        key taken from accessor and matching value.
-   **/
-  template <typename FunctorT>
-  void getAllFromValueAccessorCompositeKey(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      FunctorT *functor) const;
-
-  /**
-   * @brief Apply the functor to each key with a match in the hash table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor which should provide an operator that
-   *        takes 1 argument: const ValueAccessor&. The operator will be called
-   *        only once for a key from accessor if there is a match.
-   */
-  template <typename FunctorT>
-  void runOverKeysFromValueAccessorIfMatchFound(ValueAccessor *accessor,
-                                                const attribute_id key_attr_id,
-                                                const bool check_for_null_keys,
-                                                FunctorT *functor) const {
-    return runOverKeysFromValueAccessor<true>(
-        accessor, key_attr_id, check_for_null_keys, functor);
-  }
-
-  /**
-   * @brief Apply the functor to each key with a match in the hash table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor which should provide an operator that
-   *        takes 1 argument: const ValueAccessor&. The operator will be called
-   *        only once for a key from accessor if there is a match.
-   */
-  template <typename FunctorT>
-  void runOverKeysFromValueAccessorIfMatchFoundCompositeKey(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      FunctorT *functor) const {
-    return runOverKeysFromValueAccessorCompositeKey<true>(
-        accessor, key_attr_ids, check_for_null_keys, functor);
-  }
-
-  /**
-   * @brief Apply the functor to each key without a match in the hash table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor which should provide an operator that
-   *        takes 1 argument: const ValueAccessor&. The operator will be called
-   *        only once for a key from accessor if there is no match.
-   */
-  template <typename FunctorT>
-  void runOverKeysFromValueAccessorIfMatchNotFound(
-      ValueAccessor *accessor,
-      const attribute_id key_attr_id,
-      const bool check_for_null_keys,
-      FunctorT *functor) const {
-    return runOverKeysFromValueAccessor<false>(
-        accessor, key_attr_id, check_for_null_keys, functor);
-  }
-
-  /**
-   * @brief Apply the functor to each key without a match in the hash table.
-   *
-   * @param accessor A ValueAccessor which will be used to access keys.
-   *        beginIteration() should be called on accessor before calling this
-   *        method.
-   * @param key_attr_id The attribute ID of the keys to be read from accessor.
-   * @param check_for_null_keys If true, each key will be checked to see if it
-   *        is null before looking it up (null keys are skipped). This must be
-   *        set to true if some of the keys that will be read from accessor may
-   *        be null.
-   * @param functor A pointer to a functor which should provide an operator that
-   *        takes 1 argument: const ValueAccessor&. The operator will be called
-   *        only once for a key from accessor if there is no match.
-   */
-  template <typename FunctorT>
-  void runOverKeysFromValueAccessorIfMatchNotFoundCompositeKey(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      FunctorT *functor) const {
-    return runOverKeysFromValueAccessorCompositeKey<false>(
-        accessor, key_attr_ids, check_for_null_keys, functor);
-  }
-
-  /**
-   * @brief Apply a functor to each key, value pair in this hash table.
-   *
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for single scalar keys. See also
-   *       forEachCompositeKey().
-   *
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator which takes 2 arguments: const TypedValue&, const ValueT&.
-   *        The call operator will be invoked once on each key, value pair in
-   *        this hash table (note that if allow_duplicate_keys is true,
-   *        the call may occur multiple times for the same key with different
-   *        values).
-   * @return The number of key-value pairs visited.
-   **/
-  template <typename FunctorT>
-  std::size_t forEach(FunctorT *functor) const;
-
-  /**
-   * @brief Apply a functor to each key, value pair in this hash table.
-   *
-   * @warning This method assumes that no concurrent calls to put(),
-   *          putCompositeKey(), putValueAccessor(),
-   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
-   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
-   *          taking place (i.e. that this HashTable is immutable for the
-   *          duration of the call and as long as the returned pointer may be
-   *          dereferenced). Concurrent calls to getSingle(),
-   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
-   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
-   *          forEach(), and forEachCompositeKey() are safe.
-   * @note This version is for composite keys. See also forEach().
-   *
-   * @param functor A pointer to a functor, which should provide a call
-   *        operator which takes 2 arguments: const std::vector<TypedValue>&,
-   *        const ValueT&. The call operator will be invoked once on each key,
-   *        value pair in this hash table (note that if allow_duplicate_keys is
-   *        true, the call may occur multiple times for the same key with
-   *        different values).
-   * @return The number of key-value pairs visited.
-   **/
-  template <typename FunctorT>
-  std::size_t forEachCompositeKeyFast(FunctorT *functor) const;
-
-  template <typename FunctorT>
-  std::size_t forEachCompositeKeyFast(FunctorT *functor, int index) const;
-
- protected:
-  /**
-   * @brief Constructor for new resizable hash table.
-   *
-   * @param key_types A vector of one or more types (>1 indicates a composite
-   *        key).
-   * @param num_entries The estimated number of entries this hash table will
-   *        hold.
-   * @param storage_manager The StorageManager to use (a StorageBlob will be
-   *        allocated to hold this hash table's contents).
-   * @param adjust_hashes If true, the hash of a key should be modified by
-   *        applying AdjustHash() so that it does not collide with one of the
-   *        special values kEmptyHash or kPendingHash. If false, the hash is
-   *        used as-is.
-   * @param use_scalar_literal_hash If true, the key is a single scalar literal
-   *        (non-composite) that it is safe to use the simplified hash function
-   *        TypedValue::getHashScalarLiteral() on. If false, the generic
-   *        TypedValue::getHash() method will be used.
-   * @param preallocate_supported If true, this HashTable overrides
-   *        preallocateForBulkInsert() to allow bulk-allocation of resources
-   *        (i.e. buckets and variable-length key storage) in a single up-front
-   *        pass when bulk-inserting entries. If false, resources are allocated
-   *        on the fly for each entry.
-   **/
-  FastHashTable(const std::vector<const Type *> &key_types,
-                const std::size_t num_entries,
-                const std::vector<AggregationHandle *> &handles,
-                const std::vector<std::size_t> &payload_sizes,
-                StorageManager *storage_manager,
-                const bool adjust_hashes,
-                const bool use_scalar_literal_hash,
-                const bool preallocate_supported)
-      : key_types_(key_types),
-        scalar_key_inline_(true),
-        key_inline_(nullptr),
-        adjust_hashes_(adjust_hashes),
-        use_scalar_literal_hash_(use_scalar_literal_hash),
-        preallocate_supported_(preallocate_supported),
-        handles_(handles),
-        num_handles_(handles.size()),
-        total_payload_size_(std::accumulate(
-            payload_sizes.begin(), payload_sizes.end(), sizeof(SpinMutex))),
-        storage_manager_(storage_manager),
-        hash_table_memory_(nullptr),
-        hash_table_memory_size_(0) {
-    DEBUG_ASSERT(resizable);
-    std::size_t running_sum = sizeof(SpinMutex);
-    for (auto size : payload_sizes) {
-      payload_offsets_.emplace_back(running_sum);
-      running_sum += size;
-    }
-  }
-
-  /**
-   * @brief Constructor for non-resizable hash table.
-   *
-   * @param key_types A vector of one or more types (>1 indicates a composite
-   *        key).
-   * @param hash_table_memory A pointer to memory to use for this hash table.
-   * @param hash_table_memory_size The size of hash_table_memory in bytes.
-   * @param new_hash_table If true, this hash table is being constructed for
-   *        the first time and hash_table_memory will be cleared. If false,
-   *        reload a pre-existing hash table.
-   * @param hash_table_memory_zeroed If new_hash_table is true, setting this to
-   *        true means that this HashTable will assume that hash_table_memory
-   *        has already been zeroed-out (any newly-allocated block or blob
-   *        memory from StorageManager is zeroed-out). If false, this HashTable
-   *        will explicitly zero-fill its memory as neccessary. This parameter
-   *        has no effect when new_hash_table is false.
-   * @param adjust_hashes If true, the hash of a key should be modified by
-   *        applying AdjustHash() so that it does not collide with one of the
-   *        special values kEmptyHash or kPendingHash. If false, the hash is
-   *        used as-is.
-   * @param use_scalar_literal_hash If true, the key is a single scalar literal
-   *        (non-composite) that it is safe to use the simplified hash function
-   *        TypedValue::getHashScalarLiteral() on. If false, the generic
-   *        TypedValue::getHash() method will be used.
-   * @param preallocate_supported If true, this HashTable overrides
-   *        preallocateForBulkInsert() to allow bulk-allocation of resources
-   *        (i.e. buckets and variable-length key storage) in a single up-front
-   *        pass when bulk-inserting entries. If false, resources are allocated
-   *        on the fly for each entry.
-   **/
-  FastHashTable(const std::vector<const Type *> &key_types,
-                void *hash_table_memory,
-                const std::size_t hash_table_memory_size,
-                const bool new_hash_table,
-                const bool hash_table_memory_zeroed,
-                const bool adjust_hashes,
-                const bool use_scalar_literal_hash,
-                const bool preallocate_supported)
-      : key_types_(key_types),
-        scalar_key_inline_(true),
-        key_inline_(nullptr),
-        adjust_hashes_(adjust_hashes),
-        use_scalar_literal_hash_(use_scalar_literal_hash),
-        preallocate_supported_(preallocate_supported),
-        storage_manager_(nullptr),
-        hash_table_memory_(hash_table_memory),
-        hash_table_memory_size_(hash_table_memory_size) {
-    DEBUG_ASSERT(!resizable);
-  }
-
-  // Adjust 'hash' so that it is not exactly equal to either of the special
-  // values kEmptyHash or kPendingHash.
-  inline constexpr static std::size_t AdjustHash(const std::size_t hash) {
-    return hash + (hash == kEmptyHash) - (hash == kPendingHash);
-  }
-
-  // Set information about which key components are stored inline. This usually
-  // comes from a HashTableKeyManager, and is set by the constructor of a
-  // subclass of HashTable.
-  inline void setKeyInline(const std::vector<bool> *key_inline) {
-    scalar_key_inline_ = key_inline->front();
-    key_inline_ = key_inline;
-  }
-
-  // Generate a hash for a composite key by hashing each component of 'key' and
-  // mixing their bits with CombineHashes().
-  inline std::size_t hashCompositeKey(const std::vector<TypedValue> &key) const;
-
-  // If 'force_key_copy' is true and some part of a composite key is
-  // variable-length, calculate the total number of bytes for variable-length
-  // key components that need to be copied. Otherwise, return 0 to indicate
-  // that no variable-length copy is required.
-  inline std::size_t calculateVariableLengthCompositeKeyCopySize(
-      const std::vector<TypedValue> &key) const;
-
-  // Helpers for put. If this HashTable is resizable, 'resize_shared_mutex_'
-  // should be locked in shared mode before calling either of these methods.
-  virtual HashTablePutResult putInternal(
-      const TypedValue &key,
-      const std::size_t variable_key_size,
-      const std::uint8_t &value,
-      HashTablePreallocationState *prealloc_state) = 0;
-
-  virtual HashTablePutResult putCompositeKeyInternalFast(
-      const std::vector<TypedValue> &key,
-      const std::size_t variable_key_size,
-      const std::uint8_t *init_value_ptr,
-      HashTablePreallocationState *prealloc_state) = 0;
-
-  // Helpers for upsert. Both return a pointer to the value corresponding to
-  // 'key'. If this HashTable is resizable, 'resize_shared_mutex_' should be
-  // locked in shared mode while calling and using the returned pointer. May
-  // return NULL if there is not enough space to insert a new key, in which
-  // case a resizable HashTable should release the 'resize_shared_mutex_' and
-  // call resize(), then try again.
-  virtual std::uint8_t *upsertInternalFast(
-      const TypedValue &key,
-      const std::size_t variable_key_size,
-      const std::uint8_t *init_value_ptr) = 0;
-
-  virtual std::uint8_t *upsertCompositeKeyInternalFast(
-      const std::vector<TypedValue> &key,
-      const std::uint8_t *init_value_ptr,
-      const std::size_t variable_key_size) = 0;
-
-  // Helpers for forEach. Each return true on success, false if no more entries
-  // exist to iterate over. After a successful call, '*key' is overwritten with
-  // the key of the next entry, '*value' points to the associated value, and
-  // '*entry_num' is incremented to the next (implementation defined) entry to
-  // check ('*entry_num' should initially be set to zero).
-  virtual bool getNextEntry(TypedValue *key,
-                            const std::uint8_t **value,
-                            std::size_t *entry_num) const = 0;
-  virtual bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                                        const std::uint8_t **value,
-                                        std::size_t *entry_num) const = 0;
-
-  // Helpers for getAllFromValueAccessor. Each return true on success, false if
-  // no more entries exist for the specified key. After a successful call,
-  // '*value' points to the associated value, and '*entry_num' is incremented
-  // to the next (implementation defined) entry to check ('*entry_num' should
-  // initially be set to zero).
-  virtual bool getNextEntryForKey(const TypedValue &key,
-                                  const std::size_t hash_code,
-                                  const std::uint8_t **value,
-                                  std::size_t *entry_num) const = 0;
-  virtual bool getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
-                                           const std::size_t hash_code,
-                                           const std::uint8_t **value,
-                                           std::size_t *entry_num) const = 0;
-
-  // Return true if key exists in the hash table.
-  virtual bool hasKey(const TypedValue &key) const = 0;
-  virtual bool hasCompositeKey(const std::vector<TypedValue> &key) const = 0;
-
-  // For a resizable HashTable, grow to accomodate more entries. If
-  // 'extra_buckets' is not zero, it may serve as a "hint" to implementations
-  // that at least the requested number of extra buckets are required when
-  // resizing (mainly used in putValueAccessor() and
-  // putValueAccessorCompositeKey() when 'preallocate_supported_' is true).
-  // Implementations are free to ignore 'extra_buckets'. If
-  // 'extra_variable_storage' is not zero, implementations will attempt to
-  // allocate at least enough additional variable-key storage space to
-  // accomodate the number of bytes specified. 'retry_num' is intended ONLY for
-  // when resize() recursively calls itself and should not be set to nonzero by
-  // any other caller.
-  virtual void resize(const std::size_t extra_buckets,
-                      const std::size_t extra_variable_storage,
-                      const std::size_t retry_num = 0) = 0;
-
-  // In the case where 'allow_duplicate_keys' is true, it is possible to
-  // pre-calculate the number of key-value entries and the amount of
-  // variable-length key storage that will be needed to insert all the
-  // entries from a ValueAccessor in putValueAccessor() or
-  // putValueAccessorCompositeKey() before actually inserting anything. Some
-  // HashTable implemetations (notably SeparateChainingHashTable) can achieve
-  // better performance by ammortizing the cost of allocating certain resources
-  // (buckets and variable-length key storage) in one up-front allocation. This
-  // method is intended to support that. Returns true and fills in
-  // '*prealloc_state' if pre-allocation was successful. Returns false if a
-  // resize() is needed.
-  virtual bool preallocateForBulkInsert(
-      const std::size_t total_entries,
-      const std::size_t total_variable_key_size,
-      HashTablePreallocationState *prealloc_state) {
-    FATAL_ERROR(
-        "Called HashTable::preallocateForBulkInsert() on a HashTable "
-        "implementation that does not support preallocation.");
-  }
-
-  // Type(s) of keys.
-  const std::vector<const Type *> key_types_;
-
-  // Information about whether key components are stored inline or in a
-  // separate variable-length storage region. This is usually determined by a
-  // HashTableKeyManager and set by calling setKeyInline().
-  bool scalar_key_inline_;
-  const std::vector<bool> *key_inline_;
-
-  // Whether hashes should be adjusted by AdjustHash() before being used.
-  const bool adjust_hashes_;
-  // Whether it is safe to use the simplified TypedValue::getHashScalarLiteral()
-  // method instead of the generic TypedValue::getHash() method.
-  const bool use_scalar_literal_hash_;
-  // Whether preallocateForBulkInsert() is supported by this HashTable.
-  const bool preallocate_supported_;
-
-  const std::vector<AggregationHandle *> handles_;
-  const unsigned int num_handles_;
-  const std::size_t total_payload_size_;
-  std::vector<std::size_t> payload_offsets_;
-
-  // Used only when resizable is true:
-  StorageManager *storage_manager_;
-  MutableBlobReference blob_;
-  // Locked in shared mode for most operations, exclusive mode during resize.
-  // Not locked at all for non-resizable HashTables.
-  alignas(kCacheLineBytes) SpinSharedMutex<true> resize_shared_mutex_;
-
-  // Used only when resizable is false:
-  void *hash_table_memory_;
-  const std::size_t hash_table_memory_size_;
-
- private:
-  // Assign '*key_vector' with the attribute values specified by 'key_attr_ids'
-  // at the current position of 'accessor'. If 'check_for_null_keys' is true,
-  // stops and returns true if any of the values is null, otherwise returns
-  // false.
-  template <typename ValueAccessorT>
-  inline static bool GetCompositeKeyFromValueAccessor(
-      const ValueAccessorT &accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      std::vector<TypedValue> *key_vector) {
-    for (std::vector<attribute_id>::size_type key_idx = 0;
-         key_idx < key_attr_ids.size();
-         ++key_idx) {
-      (*key_vector)[key_idx] = accessor.getTypedValue(key_attr_ids[key_idx]);
-      if (check_for_null_keys && (*key_vector)[key_idx].isNull()) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  // If run_if_match_found is true, apply the functor to each key if a match is
-  // found; otherwise, apply the functor if no match is found.
-  template <bool run_if_match_found, typename FunctorT>
-  void runOverKeysFromValueAccessor(ValueAccessor *accessor,
-                                    const attribute_id key_attr_id,
-                                    const bool check_for_null_keys,
-                                    FunctorT *functor) const;
-
-  template <bool run_if_match_found, typename FunctorT>
-  void runOverKeysFromValueAccessorCompositeKey(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      FunctorT *functor) const;
-
-  // Method containing the actual logic implementing getAllFromValueAccessor().
-  // Has extra template parameters that control behavior to avoid some
-  // inner-loop branching.
-  template <typename FunctorT,
-            bool check_for_null_keys,
-            bool adjust_hashes_template,
-            bool use_scalar_literal_hash_template>
-  void getAllFromValueAccessorImpl(ValueAccessor *accessor,
-                                   const attribute_id key_attr_id,
-                                   FunctorT *functor) const;
-
-  DISALLOW_COPY_AND_ASSIGN(FastHashTable);
-};
-
-/**
- * @brief An instantiation of the HashTable template for use in aggregations.
- * @note This has force_key_copy = true, so that we don't have dangling pointers
- * to blocks that are evicted.
- **/
-using AggregationStateFastHashTable = FastHashTable<true, false, true, false>;
-
-/** @} */
-
-// ----------------------------------------------------------------------------
-// Implementations of template class methods follow.
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-HashTablePutResult
-FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
-    put(const TypedValue &key, const std::uint8_t &value) {
-  const std::size_t variable_size =
-      (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-  if (resizable) {
-    HashTablePutResult result = HashTablePutResult::kOutOfSpace;
-    while (result == HashTablePutResult::kOutOfSpace) {
-      {
-        SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-        result = putInternal(key, variable_size, value, nullptr);
-      }
-      if (result == HashTablePutResult::kOutOfSpace) {
-        resize(0, variable_size);
-      }
-    }
-    return result;
-  } else {
-    return putInternal(key, variable_size, value, nullptr);
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-HashTablePutResult
-FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
-    putCompositeKey(const std::vector<TypedValue> &key,
-                    const std::uint8_t *init_value_ptr) {
-  const std::size_t variable_size =
-      calculateVariableLengthCompositeKeyCopySize(key);
-  if (resizable) {
-    HashTablePutResult result = HashTablePutResult::kOutOfSpace;
-    while (result == HashTablePutResult::kOutOfSpace) {
-      {
-        SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-        result = putCompositeKeyInternalFast(
-            key, variable_size, init_value_ptr, nullptr);
-      }
-      if (result == HashTablePutResult::kOutOfSpace) {
-        resize(0, variable_size);
-      }
-    }
-    return result;
-  } else {
-    return putCompositeKeyInternalFast(
-        key, variable_size, init_value_ptr, nullptr);
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-template <typename FunctorT>
-HashTablePutResult
-FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
-    putValueAccessor(ValueAccessor *accessor,
-                     const attribute_id key_attr_id,
-                     const bool check_for_null_keys,
-                     FunctorT *functor) {
-  HashTablePutResult result = HashTablePutResult::kOutOfSpace;
-  std::size_t variable_size;
-  HashTablePreallocationState prealloc_state;
-  bool using_prealloc = allow_duplicate_keys && preallocate_supported_;
-  return InvokeOnAnyValueAccessor(
-      accessor,
-      [&](auto *accessor) -> HashTablePutResult {  // NOLINT(build/c++11)
-        if (using_prealloc) {
-          std::size_t total_entries = 0;
-          std::size_t total_variable_key_size = 0;
-          if (check_for_null_keys || (force_key_copy && !scalar_key_inline_)) {
-            // If we need to filter out nulls OR make variable copies, make a
-            // prepass over the ValueAccessor.
-            while (accessor->next()) {
-              TypedValue key = accessor->getTypedValue(key_attr_id);
-              if (check_for_null_keys && key.isNull()) {
-                continue;
-              }
-              ++total_entries;
-              total_variable_key_size += (force_key_copy && !scalar_key_inline_)
-                                             ? key.getDataSize()
-                                             : 0;
-            }
-            accessor->beginIteration();
-          } else {
-            total_entries = accessor->getNumTuples();
-          }
-          if (resizable) {
-            bool prealloc_succeeded = false;
-            while (!prealloc_succeeded) {
-              {
-                SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-                prealloc_succeeded = this->preallocateForBulkInsert(
-                    total_entries, total_variable_key_size, &prealloc_state);
-              }
-              if (!prealloc_succeeded) {
-                this->resize(total_entries, total_variable_key_size);
-              }
-            }
-          } else {
-            using_prealloc = this->preallocateForBulkInsert(
-                total_entries, total_variable_key_size, &prealloc_state);
-          }
-        }
-        if (resizable) {
-          while (result == HashTablePutResult::kOutOfSpace) {
-            {
-              result = HashTablePutResult::kOK;
-              SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-              while (accessor->next()) {
-                TypedValue key = accessor->getTypedValue(key_attr_id);
-                if (check_for_null_keys && key.isNull()) {
-                  continue;
-                }
-                variable_size = (force_key_copy && !scalar_key_inline_)
-                                    ? key.getDataSize()
-                                    : 0;
-                result = this->putInternal(
-                    key,
-                    variable_size,
-                    (*functor)(*accessor),
-                    using_prealloc ? &prealloc_state : nullptr);
-                if (result == HashTablePutResult::kDuplicateKey) {
-                  DEBUG_ASSERT(!using_prealloc);
-                  return result;
-                } else if (result == HashTablePutResult::kOutOfSpace) {
-                  DEBUG_ASSERT(!using_prealloc);
-                  break;
-                }
-              }
-            }
-            if (result == HashTablePutResult::kOutOfSpace) {
-              this->resize(0, variable_size);
-              accessor->previous();
-            }
-          }
-        } else {
-          while (accessor->next()) {
-            TypedValue key = accessor->getTypedValue(key_attr_id);
-            if (check_for_null_keys && key.isNull()) {
-              continue;
-            }
-            variable_size =
-                (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-            result =
-                this->putInternal(key,
-                                  variable_size,
-                                  (*functor)(*accessor),
-                                  using_prealloc ? &prealloc_state : nullptr);
-            if (result != HashTablePutResult::kOK) {
-              return result;
-            }
-          }
-        }
-
-        return HashTablePutResult::kOK;
-      });
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-template <typename FunctorT>
-HashTablePutResult
-FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
-    putValueAccessorCompositeKey(ValueAccessor *accessor,
-                                 const std::vector<attribute_id> &key_attr_ids,
-                                 const bool check_for_null_keys,
-                                 FunctorT *functor) {
-  DEBUG_ASSERT(key_types_.size() == key_attr_ids.size());
-  HashTablePutResult result = HashTablePutResult::kOutOfSpace;
-  std::size_t variable_size;
-  HashTablePreallocationState prealloc_state;
-  bool using_prealloc = allow_duplicate_keys && preallocate_supported_;
-  std::vector<TypedValue> key_vector;
-  key_vector.resize(key_attr_ids.size());
-  return InvokeOnAnyValueAccessor(
-      accessor,
-      [&](auto *accessor) -> HashTablePutResult {  // NOLINT(build/c++11)
-        if (using_prealloc) {
-          std::size_t total_entries = 0;
-          std::size_t total_variable_key_size = 0;
-          if (check_for_null_keys || force_key_copy) {
-            // If we need to filter out nulls OR make variable copies, make a
-            // prepass over the ValueAccessor.
-            while (accessor->next()) {
-              if (this->GetCompositeKeyFromValueAccessor(*accessor,
-                                                         key_attr_ids,
-                                                         check_for_null_keys,
-                                                         &key_vector)) {
-                continue;
-              }
-              ++total_entries;
-              total_variable_key_size +=
-                  this->calculateVariableLengthCompositeKeyCopySize(key_vector);
-            }
-            accessor->beginIteration();
-          } else {
-            total_entries = accessor->getNumTuples();
-          }
-          if (resizable) {
-            bool prealloc_succeeded = false;
-            while (!prealloc_succeeded) {
-              {
-                SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-                prealloc_succeeded = this->preallocateForBulkInsert(
-                    total_entries, total_variable_key_size, &prealloc_state);
-              }
-              if (!prealloc_succeeded) {
-                this->resize(total_entries, total_variable_key_size);
-              }
-            }
-          } else {
-            using_prealloc = this->preallocateForBulkInsert(
-                total_entries, total_variable_key_size, &prealloc_state);
-          }
-        }
-        if (resizable) {
-          while (result == HashTablePutResult::kOutOfSpace) {
-            {
-              result = HashTablePutResult::kOK;
-              SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-              while (accessor->next()) {
-                if (this->GetCompositeKeyFromValueAccessor(*accessor,
-                                                           key_attr_ids,
-                                                           check_for_null_keys,
-                                                           &key_vector)) {
-                  continue;
-                }
-                variable_size =
-                    this->calculateVariableLengthCompositeKeyCopySize(
-                        key_vector);
-                result = this->putCompositeKeyInternal(
-                    key_vector,
-                    variable_size,
-                    (*functor)(*accessor),
-                    using_prealloc ? &prealloc_state : nullptr);
-                if (result == HashTablePutResult::kDuplicateKey) {
-                  DEBUG_ASSERT(!using_prealloc);
-                  return result;
-                } else if (result == HashTablePutResult::kOutOfSpace) {
-                  DEBUG_ASSERT(!using_prealloc);
-                  break;
-                }
-              }
-            }
-            if (result == HashTablePutResult::kOutOfSpace) {
-              this->resize(0, variable_size);
-              accessor->previous();
-            }
-          }
-        } else {
-          while (accessor->next()) {
-            if (this->GetCompositeKeyFromValueAccessor(*accessor,
-                                                       key_attr_ids,
-                                                       check_for_null_keys,
-                                                       &key_vector)) {
-              continue;
-            }
-            variable_size =
-                this->calculateVariableLengthCompositeKeyCopySize(key_vector);
-            result = this->putCompositeKeyInternal(
-                key_vector,
-                variable_size,
-                (*functor)(*accessor),
-                using_prealloc ? &prealloc_state : nullptr);
-            if (result != HashTablePutResult::kOK) {
-              return result;
-            }
-          }
-        }
-
-        return HashTablePutResult::kOK;
-      });
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-template <typename FunctorT>
-bool FastHashTable<resizable,
-                   serializable,
-                   force_key_copy,
-                   allow_duplicate_keys>::upsert(const TypedValue &key,
-                                                 const std::uint8_t
-                                                     *initial_value_ptr,
-                                                 FunctorT *functor) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  const std::size_t variable_size =
-      (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        std::uint8_t *value =
-            upsertInternalFast(key, variable_size, initial_value_ptr);
-        if (value != nullptr) {
-          (*functor)(value);
-          return true;
-        }
-      }
-      resize(0, force_key_copy && !scalar_key_inline_ ? key.getDataSize() : 0);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertInternalFast(key, variable_size, initial_value_ptr);
-    if (value == nullptr) {
-      return false;
-    } else {
-      (*functor)(value);
-      return true;
-    }
-  }
-}
-
-class HashTableMergerFast {
- public:
-  /**
-   * @brief Constructor
-   *
-   * @param handle The Aggregation handle being used.
-   * @param destination_hash_table The destination hash table to which other
-   *        hash tables will be merged.
-   **/
-  explicit HashTableMergerFast(
-      AggregationStateHashTableBase *destination_hash_table)
-      : destination_hash_table_(
-            static_cast<FastHashTable<true, false, true, false> *>(
-                destination_hash_table)) {}
-
-  /**
-   * @brief The operator for the functor.
-   *
-   * @param group_by_key The group by key being merged.
-   * @param source_state The aggregation state for the given key in the source
-   *        aggregation hash table.
-   **/
-  inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const std::uint8_t *source_state) {
-    const std::uint8_t *original_state =
-        destination_hash_table_->getSingleCompositeKey(group_by_key);
-    if (original_state != nullptr) {
-      // The CHECK is required as upsertCompositeKey can return false if the
-      // hash table runs out of space during the upsert process. The ideal
-      // solution will be to retry again if the upsert fails.
-      CHECK(destination_hash_table_->upsertCompositeKeyFast(
-          group_by_key, original_state, source_state));
-    } else {
-      destination_hash_table_->putCompositeKey(group_by_key, source_state);
-    }
-  }
-
- private:
-  FastHashTable<true, false, true, false> *destination_hash_table_;
-
-  DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast);
-};
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-template <typename FunctorT>
-bool FastHashTable<resizable,
-                   serializable,
-                   force_key_copy,
-                   allow_duplicate_keys>::
-    upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                           const std::uint8_t *init_value_ptr,
-                           FunctorT *functor) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  const std::size_t variable_size =
-      calculateVariableLengthCompositeKeyCopySize(key);
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        std::uint8_t *value =
-            upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-        if (value != nullptr) {
-          (*functor)(value);
-          return true;
-        }
-      }
-      resize(0, variable_size);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-    if (value == nullptr) {
-      return false;
-    } else {
-      (*functor)(value);
-      return true;
-    }
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-template <typename FunctorT>
-bool FastHashTable<resizable,
-                   serializable,
-                   force_key_copy,
-                   allow_duplicate_keys>::
-    upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                           const std::uint8_t *init_value_ptr,
-                           FunctorT *functor,
-                           int index) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  const std::size_t variable_size =
-      calculateVariableLengthCompositeKeyCopySize(key);
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        std::uint8_t *value =
-            upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-        if (value != nullptr) {
-          (*functor)(value + payload_offsets_[index]);
-          return true;
-        }
-      }
-      resize(0, variable_size);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-    if (value == nullptr) {
-      return false;
-    } else {
-      (*functor)(value + payload_offsets_[index]);
-      return true;
-    }
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastHashTable<resizable,
-                   serializable,
-                   force_key_copy,
-                   allow_duplicate_keys>::
-    upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                           const std::uint8_t *init_value_ptr,
-                           const std::uint8_t *source_state) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  const std::size_t variable_size =
-      calculateVariableLengthCompositeKeyCopySize(key);
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        std::uint8_t *value =
-            upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-        if (value != nullptr) {
-          SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
-          for (unsigned int k = 0; k < num_handles_; ++k) {
-            handles_[k]->mergeStatesFast(source_state + payload_offsets_[k],
-                                         value + payload_offsets_[k]);
-          }
-          return true;
-        }
-      }
-      resize(0, variable_size);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-    if (value == nullptr) {
-      return false;
-    } else {
-      SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
-      for (unsigned int k = 0; k < num_handles_; ++k) {
-        handles_[k]->mergeStatesFast(source_state + payload_offsets_[k],
-                                     value + payload_offsets_[k]);
-      }
-      return true;
-    }
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastHashTable<resizable,
-                   serializable,
-                   force_key_copy,
-                   allow_duplicate_keys>::
-    upsertValueAccessorFast(
-        const std::vector<attribute_id> &argument_ids,
-        ValueAccessor *accessor,
-        const attribute_id key_attr_id,
-        const bool check_for_null_keys) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  std::size_t variable_size;
-  return InvokeOnAnyValueAccessor(
-      accessor,
-      [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
-        if (resizable) {
-          bool continuing = true;
-          while (continuing) {
-            {
-              continuing = false;
-              SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-              while (accessor->next()) {
-                TypedValue key = accessor->getTypedValue(key_attr_id);
-                if (check_for_null_keys && key.isNull()) {
-                  continue;
-                }
-                variable_size = (force_key_copy && !scalar_key_inline_)
-                                    ? key.getDataSize()
-                                    : 0;
-                std::uint8_t *value =
-                    this->upsertInternalFast(key, variable_size, nullptr);
-                if (value == nullptr) {
-                  continuing = true;
-                  break;
-                } else {
-                  SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
-                  for (unsigned int k = 0; k < num_handles_; ++k) {
-                    if (argument_ids[k] != kInvalidAttributeID) {
-                      handles_[k]->updateStateUnary(
-                          accessor->getTypedValue(argument_ids[k]),
-                          value + payload_offsets_[k]);
-                    } else {
-                      handles_[k]->updateStateNullary(value +
-                                                      payload_offsets_[k]);
-                    }
-                  }
-                }
-              }
-            }
-            if (continuing) {
-              this->resize(0, variable_size);
-              accessor->previous();
-            }
-          }
-        } else {
-          while (accessor->next()) {
-            TypedValue key = accessor->getTypedValue(key_attr_id);
-            if (check_for_null_keys && key.isNull()) {
-              continue;
-            }
-            variable_size =
-                (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-            std::uint8_t *value =
-                this->upsertInternalFast(key, variable_size, nullptr);
-            if (value == nullptr) {
-              return false;
-            } else {
-              SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
-              for (unsigned int k = 0; k < num_handles_; ++k) {
-                if (argument_ids[k] != kInvalidAttributeID) {
-                  handles_[k]->updateStateUnary(
-                      accessor->getTypedValue(argument_ids[k]),
-                      value + payload_offsets_[k]);
-                } else {
-                  handles_[k]->updateStateNullary(value +
-            

<TRUNCATED>


[41/50] [abbrv] incubator-quickstep git commit: Defined TMB Message Poll Interval as a gflag.

Posted by ji...@apache.org.
Defined TMB Message Poll Interval as a gflag.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 1cfc1c40e2bcf7ff8671d5b899a8304c9e9fd455
Parents: a28b1e4
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Feb 6 12:26:56 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 10:31:08 2017 -0800

----------------------------------------------------------------------
 third_party/src/tmb/CMakeLists.txt            | 41 ++++++++++++----------
 third_party/src/tmb/include/tmb/message_bus.h |  5 ---
 third_party/src/tmb/src/message_bus.cc        | 20 +++++++++--
 3 files changed, 40 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1cfc1c40/third_party/src/tmb/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/third_party/src/tmb/CMakeLists.txt b/third_party/src/tmb/CMakeLists.txt
index 14e467b..a280a93 100644
--- a/third_party/src/tmb/CMakeLists.txt
+++ b/third_party/src/tmb/CMakeLists.txt
@@ -379,6 +379,26 @@ if (ENABLE_ZOOKEEPER)
   include_directories(${ZOOKEEPER_INCLUDE_DIRS})
 endif()
 
+set_gflags_lib_name ()
+
+# NOTE(chasseur): We only add gflags and gtest to the build if those targets
+# don't already exist, so that TMB can be brought in to a build that already
+# uses one or both of those libraries with add_subdirectory() and not cause
+# name collisions.
+
+# Build GFlags command-line processing library if needed.
+if ((NOT TARGET ${GFLAGS_LIB_NAME}) AND (BUILD_BENCHMARKS OR ENABLE_NATIVENET))
+  add_subdirectory(third_party/gflags)
+  include_directories(${CMAKE_CURRENT_BINARY_DIR}/third_party/gflags/include)
+endif()
+
+# Googletest Framework For Unit Testing
+if (NOT TARGET gtest)
+  add_subdirectory(third_party/gtest)
+  include_directories(third_party/gtest/include)
+  enable_testing()
+endif()
+
 # Include path for TMB.
 include_directories(${PROJECT_SOURCE_DIR}/include)
 set(TMB_INCLUDE_DIRS ${TMB_INCLUDE_DIRS} CACHE STRING
@@ -391,7 +411,8 @@ link_directories(${tmb_BINARY_DIR}/src)
 add_library(tmb
             ${TMB_SRCS})
 target_link_libraries(tmb
-                      ${CMAKE_THREAD_LIBS_INIT})
+                      ${CMAKE_THREAD_LIBS_INIT}
+                      ${GFLAGS_LIB_NAME})
 
 if (ENABLE_LEVELDB)
   target_link_libraries(tmb
@@ -418,24 +439,6 @@ if (ENABLE_ZOOKEEPER)
                         ${ZOOKEEPER_LIBRARIES})
 endif()
 
-# NOTE(chasseur): We only add gflags and gtest to the build if those targets
-# don't already exist, so that TMB can be brought in to a build that already
-# uses one or both of those libraries with add_subdirectory() and not cause
-# name collisions.
-
-# Build GFlags command-line processing library if needed.
-if ((NOT TARGET gflags_nothreads-static) AND (BUILD_BENCHMARKS OR ENABLE_NATIVENET))
-  add_subdirectory(third_party/gflags)
-  include_directories(${CMAKE_CURRENT_BINARY_DIR}/third_party/gflags/include)
-endif()
-
-# Googletest Framework For Unit Testing
-if (NOT TARGET gtest)
-  add_subdirectory(third_party/gtest)
-  include_directories(third_party/gtest/include)
-  enable_testing()
-endif()
-
 # Build the tmb_net_server executable if enabled.
 if (ENABLE_NATIVENET)
   add_executable(tmb_net_server src/tmb_net_server.cc)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1cfc1c40/third_party/src/tmb/include/tmb/message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/src/tmb/include/tmb/message_bus.h b/third_party/src/tmb/include/tmb/message_bus.h
index a4ca525..74e298d 100644
--- a/third_party/src/tmb/include/tmb/message_bus.h
+++ b/third_party/src/tmb/include/tmb/message_bus.h
@@ -496,11 +496,6 @@ class MessageBus {
       internal::IteratorAdapter<const AnnotatedMessage> *adapter) = 0;
 
  private:
-  // The number of milliseconds to sleep between calls to
-  // ReceiveIfAvailableImpl() in the default active-polling implementation of
-  // ReceiveImpl().
-  static const unsigned int kReceivePollIntervalMS = 100;
-
   // Disallow copy and assign:
   MessageBus(const MessageBus &orig) = delete;
   MessageBus& operator=(const MessageBus &rhs) = delete;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1cfc1c40/third_party/src/tmb/src/message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/src/tmb/src/message_bus.cc b/third_party/src/tmb/src/message_bus.cc
index 44324ec..7fd0efb 100644
--- a/third_party/src/tmb/src/message_bus.cc
+++ b/third_party/src/tmb/src/message_bus.cc
@@ -24,9 +24,25 @@
 #include <cstdlib>
 #include <thread>  // NOLINT(build/c++11)
 
+#include "gflags/gflags.h"
+
 namespace tmb {
 
-const unsigned int MessageBus::kReceivePollIntervalMS;
+static bool ValidateTmbReceivePollInterval(const char *flagname,
+                                           std::int32_t value) {
+  if (value > 0) {
+    return true;
+  } else {
+    std::fprintf(stderr, "--%s must be at least 1\n", flagname);
+    return false;
+  }
+}
+DEFINE_int32(tmb_receive_poll_interval, 50,
+             "The number of milliseconds to sleep between calls to ReceiveIfAvailableImpl() "
+             "in the default active-polling implementation of ReceiveImpl().");
+static const bool tmb_receive_poll_interval_dummy = gflags::RegisterFlagValidator(
+    &FLAGS_tmb_receive_poll_interval,
+    &ValidateTmbReceivePollInterval);
 
 internal::NetMessageRemovalInterface*
     MessageBus::GetNetMessageRemovalInterface() {
@@ -49,7 +65,7 @@ std::size_t MessageBus::ReceiveImpl(const client_id receiver_id,
                                                 pusher);
   while (received == 0) {
     std::this_thread::sleep_for(
-        std::chrono::milliseconds(kReceivePollIntervalMS));
+        std::chrono::milliseconds(FLAGS_tmb_receive_poll_interval));
     received = ReceiveIfAvailableImpl(receiver_id,
                                       minimum_priority,
                                       max_messages,


[08/50] [abbrv] incubator-quickstep git commit: Add BitVectorExactFilter as a LIP filter and supports Join-to-Semijoin transformation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/rules/InjectJoinFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/InjectJoinFilters.cpp b/query_optimizer/rules/InjectJoinFilters.cpp
new file mode 100644
index 0000000..0fcd06b
--- /dev/null
+++ b/query_optimizer/rules/InjectJoinFilters.cpp
@@ -0,0 +1,438 @@
+/**
+ * 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 "query_optimizer/rules/InjectJoinFilters.hpp"
+
+#include <cstddef>
+#include <cstdint>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/LIPFilterConfiguration.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/FilterJoin.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/Selection.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/rules/PruneColumns.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "utility/lip_filter/LIPFilter.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr InjectJoinFilters::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  const P::TopLevelPlanPtr top_level_plan =
+     std::static_pointer_cast<const P::TopLevelPlan>(input);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(
+          top_level_plan->shared_subplans()));
+  lip_filter_configuration_.reset(new P::LIPFilterConfiguration());
+
+  // Step 1. Transform applicable HashJoin nodes to FilterJoin nodes.
+  P::PhysicalPtr output = transformHashJoinToFilters(input);
+
+  // Step 2. Push down FilterJoin nodes to be evaluated early.
+  output = pushDownFilters(output);
+
+  // Step 3. Add Selection nodes for attaching the LIPFilters, if necessary.
+  output = addFilterAnchors(output, false);
+
+  // Step 4. Because of the pushdown of FilterJoin nodes, there are optimization
+  // opportunities for projecting columns early.
+  output = PruneColumns().apply(output);
+
+  // Step 5. For each FilterJoin node, attach its corresponding LIPFilter to
+  // proper nodes.
+  concretizeAsLIPFilters(output, nullptr);
+
+  if (!lip_filter_configuration_->getBuildInfoMap().empty() ||
+      !lip_filter_configuration_->getProbeInfoMap().empty()) {
+    output = std::static_pointer_cast<const P::TopLevelPlan>(output)
+        ->copyWithLIPFilterConfiguration(
+              P::LIPFilterConfigurationPtr(lip_filter_configuration_.release()));
+  }
+
+  return output;
+}
+
+bool InjectJoinFilters::isTransformable(
+    const physical::HashJoinPtr &hash_join) const {
+  // Conditions for replacing a HashJoin with a FilterJoin:
+
+  // No residual predicate.
+  if (hash_join->residual_predicate() != nullptr) {
+    return false;
+  }
+  // Single attribute equi-join.
+  if (hash_join->right_join_attributes().size() > 1) {
+    return false;
+  }
+  // All the output attributes must be from the probe side.
+  if (!E::SubsetOfExpressions(hash_join->getOutputAttributes(),
+                              hash_join->left()->getOutputAttributes())) {
+    return false;
+  }
+  switch (hash_join->join_type()) {
+    case P::HashJoin::JoinType::kInnerJoin: {
+      // In the case of inner join, the build side join attributes must be unique.
+      if (!cost_model_->impliesUniqueAttributes(hash_join->right(),
+                                                hash_join->right_join_attributes())) {
+        return false;
+      }
+      break;
+    }
+    case P::HashJoin::JoinType::kLeftSemiJoin:  // Fall through
+    case P::HashJoin::JoinType::kLeftAntiJoin:
+      break;
+    default:
+      return false;
+  }
+
+  // The build side join attribute has integer type and its values are exactly
+  // within a reasonable range.
+  std::int64_t min_cpp_value;
+  std::int64_t max_cpp_value;
+  const bool has_exact_min_max_stats =
+      findExactMinMaxValuesForAttributeHelper(hash_join->right(),
+                                              hash_join->right_join_attributes().front(),
+                                              &min_cpp_value,
+                                              &max_cpp_value);
+  if (!has_exact_min_max_stats) {
+    return false;
+  }
+
+  // TODO(jianqiao): implement SimpleHashSetExactFilter to relax this requirement.
+  // Note that 1G bits = 128MB.
+  const std::int64_t value_range = max_cpp_value - min_cpp_value;
+  DCHECK_GE(value_range, 0);
+  if (value_range > kMaxFilterSize) {
+    return false;
+  }
+
+  return true;
+}
+
+P::PhysicalPtr InjectJoinFilters::transformHashJoinToFilters(
+    const P::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  bool has_changed_children = false;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = transformHashJoinToFilters(child);
+    if (child != new_child && !has_changed_children) {
+      has_changed_children = true;
+    }
+    new_children.push_back(new_child);
+  }
+
+  P::HashJoinPtr hash_join;
+  if (P::SomeHashJoin::MatchesWithConditionalCast(input, &hash_join) &&
+      isTransformable(hash_join)) {
+    const bool is_anti_join =
+        hash_join->join_type() == P::HashJoin::JoinType::kLeftAntiJoin;
+
+    DCHECK_EQ(2u, new_children.size());
+    P::PhysicalPtr build_child = new_children[1];
+    E::PredicatePtr build_side_filter_predicate = nullptr;
+    P::SelectionPtr selection;
+    if (P::SomeSelection::MatchesWithConditionalCast(build_child, &selection) &&
+        E::SubsetOfExpressions(hash_join->right_join_attributes(),
+                               selection->input()->getOutputAttributes())) {
+      build_child = selection->input();
+      build_side_filter_predicate = selection->filter_predicate();
+    }
+
+    return P::FilterJoin::Create(new_children[0],
+                                 build_child,
+                                 hash_join->left_join_attributes(),
+                                 hash_join->right_join_attributes(),
+                                 hash_join->project_expressions(),
+                                 build_side_filter_predicate,
+                                 is_anti_join);
+  }
+
+  if (has_changed_children) {
+    return input->copyWithNewChildren(new_children);
+  } else {
+    return input;
+  }
+}
+
+physical::PhysicalPtr InjectJoinFilters::pushDownFilters(
+    const physical::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  bool has_changed_children = false;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = pushDownFilters(child);
+    if (child != new_child && !has_changed_children) {
+      has_changed_children = true;
+    }
+    new_children.push_back(new_child);
+  }
+
+  P::FilterJoinPtr filter_join;
+  if (P::SomeFilterJoin::MatchesWithConditionalCast(input, &filter_join)) {
+    DCHECK_EQ(2u, new_children.size());
+    return pushDownFiltersInternal(
+        new_children[0], new_children[1], filter_join);
+  }
+
+  if (has_changed_children) {
+    return input->copyWithNewChildren(new_children);
+  } else {
+    return input;
+  }
+}
+
+physical::PhysicalPtr InjectJoinFilters::pushDownFiltersInternal(
+    const physical::PhysicalPtr &probe_child,
+    const physical::PhysicalPtr &build_child,
+    const physical::FilterJoinPtr &filter_join) const {
+  switch (probe_child->getPhysicalType()) {
+    case P::PhysicalType::kAggregate:  // Fall through
+    case P::PhysicalType::kHashJoin:
+    case P::PhysicalType::kSample:
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kSort:
+    case P::PhysicalType::kWindowAggregate: {
+      DCHECK_GE(probe_child->getNumChildren(), 1u);
+      const P::PhysicalPtr child = probe_child->children()[0];
+      if (E::SubsetOfExpressions(filter_join->probe_attributes(),
+                                 child->getOutputAttributes())) {
+        const P::PhysicalPtr new_child =
+            pushDownFiltersInternal(child, build_child, filter_join);
+        if (new_child != child) {
+          std::vector<P::PhysicalPtr> new_children = probe_child->children();
+          new_children[0] = new_child;
+          return probe_child->copyWithNewChildren(new_children);
+        }
+      }
+    }
+    default:
+      break;
+  }
+
+  if (probe_child != filter_join->left()) {
+    // TODO(jianqiao): may need to update probe_attributes.
+    return P::FilterJoin::Create(probe_child,
+                                 build_child,
+                                 filter_join->probe_attributes(),
+                                 filter_join->build_attributes(),
+                                 E::ToNamedExpressions(probe_child->getOutputAttributes()),
+                                 filter_join->build_side_filter_predicate(),
+                                 filter_join->is_anti_join());
+  } else {
+    return filter_join;
+  }
+}
+
+
+physical::PhysicalPtr InjectJoinFilters::addFilterAnchors(
+    const physical::PhysicalPtr &input,
+    const bool ancestor_can_anchor_filter) const {
+  std::vector<P::PhysicalPtr> new_children;
+
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(input);
+      new_children.emplace_back(
+          addFilterAnchors(aggregate->input(), true));
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      const P::SelectionPtr &selection =
+          std::static_pointer_cast<const P::Selection>(input);
+      new_children.emplace_back(
+          addFilterAnchors(selection->input(), true));
+      break;
+    }
+    // NOTE(jianqiao): Some of the SSB/TPCH queries slow down significantly if
+    // we attach converted filters to parent HashJoin nodes. E.g. one HashJoin +
+    // one attached LIPFilter is slower than the original two HashJoins. This is
+    // due to some implementation issues with the current HashJoinOperator. So
+    // currently we disable the anchoring of filters to HashJoin nodes. That is,
+    // in the case that a FilterJoin's parent node (or ancestor node, if there
+    // is a chain of FilterJoins) is a HashJoin, we create an extra Selection
+    // before the parent HashJoin as anchoring node to attach the filters. This
+    // guarantees non-degrading performance.
+    /*
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr &hash_join =
+          std::static_pointer_cast<const P::HashJoin>(input);
+      new_children.emplace_back(
+          addFilterAnchors(hash_join->left(), true));
+      new_children.emplace_back(
+          addFilterAnchors(hash_join->right(), false));
+      break;
+    }
+    */
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(input);
+      new_children.emplace_back(
+          addFilterAnchors(filter_join->left(), true));
+      new_children.emplace_back(
+          addFilterAnchors(filter_join->right(), true));
+      break;
+    }
+    default: {
+      for (const P::PhysicalPtr &child : input->children()) {
+        new_children.emplace_back(addFilterAnchors(child, false));
+      }
+    }
+  }
+
+  DCHECK_EQ(new_children.size(), input->children().size());
+  const P::PhysicalPtr output_with_new_children =
+      new_children == input->children()
+          ? input
+          : input->copyWithNewChildren(new_children);
+
+  if (input->getPhysicalType() == P::PhysicalType::kFilterJoin &&
+      !ancestor_can_anchor_filter) {
+    const P::FilterJoinPtr &filter_join =
+        std::static_pointer_cast<const P::FilterJoin>(output_with_new_children);
+    return P::Selection::Create(filter_join,
+                                filter_join->project_expressions(),
+                                nullptr);
+  } else {
+    return output_with_new_children;
+  }
+}
+
+void InjectJoinFilters::concretizeAsLIPFilters(
+    const P::PhysicalPtr &input,
+    const P::PhysicalPtr &anchor_node) const {
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(input);
+      concretizeAsLIPFilters(aggregate->input(), aggregate);
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      const P::SelectionPtr &selection =
+          std::static_pointer_cast<const P::Selection>(input);
+      concretizeAsLIPFilters(selection->input(), selection);
+      break;
+    }
+    // Currently we disable the attachment of filters to HashJoin nodes. See the
+    // comments in InjectJoinFilters::addFilterAnchors().
+    /*
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr &hash_join =
+          std::static_pointer_cast<const P::HashJoin>(input);
+      concretizeAsLIPFilters(hash_join->left(), hash_join);
+      concretizeAsLIPFilters(hash_join->right(), nullptr);
+      break;
+    }
+    */
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(input);
+      DCHECK_EQ(1u, filter_join->build_attributes().size());
+      const E::AttributeReferencePtr &build_attr =
+          filter_join->build_attributes().front();
+
+      std::int64_t min_cpp_value;
+      std::int64_t max_cpp_value;
+      const bool has_exact_min_max_stats =
+          findExactMinMaxValuesForAttributeHelper(filter_join,
+                                                  build_attr,
+                                                  &min_cpp_value,
+                                                  &max_cpp_value);
+      DCHECK(has_exact_min_max_stats);
+      DCHECK_GE(max_cpp_value, min_cpp_value);
+      DCHECK_LE(max_cpp_value - min_cpp_value, kMaxFilterSize);
+      CHECK(anchor_node != nullptr);
+
+      lip_filter_configuration_->addBuildInfo(
+          P::BitVectorExactFilterBuildInfo::Create(build_attr,
+                                                   min_cpp_value,
+                                                   max_cpp_value,
+                                                   filter_join->is_anti_join()),
+          filter_join);
+      lip_filter_configuration_->addProbeInfo(
+          P::LIPFilterProbeInfo::Create(filter_join->probe_attributes().front(),
+                                        build_attr,
+                                        filter_join),
+          anchor_node);
+
+      concretizeAsLIPFilters(filter_join->left(), anchor_node);
+      concretizeAsLIPFilters(filter_join->right(), filter_join);
+      break;
+    }
+    default: {
+      for (const P::PhysicalPtr &child : input->children()) {
+        concretizeAsLIPFilters(child, nullptr);
+      }
+    }
+  }
+}
+
+bool InjectJoinFilters::findExactMinMaxValuesForAttributeHelper(
+    const physical::PhysicalPtr &physical_plan,
+    const expressions::AttributeReferencePtr &attribute,
+    std::int64_t *min_cpp_value,
+    std::int64_t *max_cpp_value) const {
+  bool min_value_is_exact;
+  bool max_value_is_exact;
+
+  const TypedValue min_value =
+      cost_model_->findMinValueStat(physical_plan, attribute, &min_value_is_exact);
+  const TypedValue max_value =
+      cost_model_->findMaxValueStat(physical_plan, attribute, &max_value_is_exact);
+  if (min_value.isNull() || max_value.isNull() ||
+      (!min_value_is_exact) || (!max_value_is_exact)) {
+    return false;
+  }
+
+  switch (attribute->getValueType().getTypeID()) {
+    case TypeID::kInt: {
+      *min_cpp_value = min_value.getLiteral<int>();
+      *max_cpp_value = max_value.getLiteral<int>();
+      return true;
+    }
+    case TypeID::kLong: {
+      *min_cpp_value = min_value.getLiteral<std::int64_t>();
+      *max_cpp_value = max_value.getLiteral<std::int64_t>();
+      return true;
+    }
+    default:
+      return false;
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/rules/InjectJoinFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/InjectJoinFilters.hpp b/query_optimizer/rules/InjectJoinFilters.hpp
new file mode 100644
index 0000000..c5250b3
--- /dev/null
+++ b/query_optimizer/rules/InjectJoinFilters.hpp
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_INJECT_JOIN_FILTERS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_INJECT_JOIN_FILTERS_HPP_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/LIPFilterConfiguration.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to transform HashJoin nodes into
+ *        FilterJoin nodes.
+ * 
+ * This is an optimization that strength-reduces HashJoins to FilterJoins
+ * (implemented as LIPFilters attached to some anchoring operators where the
+ * filters get applied). Briefly speaking, the idea is that in the case that
+ * (1) the join attribute has consecutive integer values bounded in a reasonably
+ * small range AND (2) the output attributes are all from the probe-side table,
+ * we can eliminate the HashJoin by building a BitVector on the build-side
+ * attribute and using the BitVector to filter the probe-side table.
+ */
+class InjectJoinFilters : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  InjectJoinFilters() {}
+
+  ~InjectJoinFilters() override {}
+
+  std::string getName() const override {
+    return "TransformFilterJoins";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  // Check whether a HashJoin can be transformed into a FilterJoin.
+  bool isTransformable(const physical::HashJoinPtr &hash_join) const;
+
+  // Transform applicable HashJoin nodes into FilterJoin nodes.
+  physical::PhysicalPtr transformHashJoinToFilters(
+      const physical::PhysicalPtr &input) const;
+
+  // Push down FilterJoin nodes to be evaluated early.
+  physical::PhysicalPtr pushDownFilters(const physical::PhysicalPtr &input) const;
+
+  // Add Selection node, if necessary, for anchoring the LIP filters built by
+  // FilterJoin nodes.
+  physical::PhysicalPtr addFilterAnchors(const physical::PhysicalPtr &input,
+                                         const bool ancestor_can_anchor_filter) const;
+
+  // Setup lip_filter_configuration_ with the transformed plan tree.
+  void concretizeAsLIPFilters(const physical::PhysicalPtr &input,
+                              const physical::PhysicalPtr &anchor_node) const;
+
+  physical::PhysicalPtr pushDownFiltersInternal(
+      const physical::PhysicalPtr &probe_child,
+      const physical::PhysicalPtr &build_child,
+      const physical::FilterJoinPtr &filter_join) const;
+
+  bool findExactMinMaxValuesForAttributeHelper(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      std::int64_t *min_cpp_value,
+      std::int64_t *max_cpp_value) const;
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+  std::unique_ptr<physical::LIPFilterConfiguration> lip_filter_configuration_;
+
+  // TODO(jianqiao): Add this threshold as a gflag.
+  // Note that 1G bits = 128MB
+  static constexpr std::int64_t kMaxFilterSize = 1000000000L;
+
+  DISALLOW_COPY_AND_ASSIGN(InjectJoinFilters);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_INJECT_JOIN_FILTERS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/query_optimizer/tests/OptimizerTextTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTest.cpp b/query_optimizer/tests/OptimizerTextTest.cpp
index e17f5c4..07accf6 100644
--- a/query_optimizer/tests/OptimizerTextTest.cpp
+++ b/query_optimizer/tests/OptimizerTextTest.cpp
@@ -34,6 +34,7 @@ namespace optimizer {
 DECLARE_bool(reorder_columns);
 DECLARE_bool(reorder_hash_joins);
 DECLARE_bool(use_lip_filters);
+DECLARE_bool(use_filter_joins);
 
 }
 }
@@ -64,6 +65,7 @@ int main(int argc, char** argv) {
   quickstep::optimizer::FLAGS_reorder_columns = false;
   quickstep::optimizer::FLAGS_reorder_hash_joins = false;
   quickstep::optimizer::FLAGS_use_lip_filters = false;
+  quickstep::optimizer::FLAGS_use_filter_joins = false;
 
   ::testing::InitGoogleTest(&argc, argv);
   int success = RUN_ALL_TESTS();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/relational_operators/BuildLIPFilterOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildLIPFilterOperator.cpp b/relational_operators/BuildLIPFilterOperator.cpp
new file mode 100644
index 0000000..f7c09cd
--- /dev/null
+++ b/relational_operators/BuildLIPFilterOperator.cpp
@@ -0,0 +1,154 @@
+/**
+ * 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 "relational_operators/BuildLIPFilterOperator.hpp"
+
+#include <memory>
+#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/StorageBlock.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/TupleIdSequence.hpp"
+#include "storage/TupleStorageSubBlock.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
+#include "utility/lip_filter/LIPFilterBuilder.hpp"
+#include "utility/lip_filter/LIPFilterUtil.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool BuildLIPFilterOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  DCHECK(query_context != nullptr);
+
+  const Predicate *build_side_predicate =
+      query_context->getPredicate(build_side_predicate_index_);
+
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addNormalWorkOrder(
+            new BuildLIPFilterWorkOrder(
+                query_id_,
+                input_relation_,
+                input_block_id,
+                build_side_predicate,
+                storage_manager,
+                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context),
+                CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
+            op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addNormalWorkOrder(
+          new BuildLIPFilterWorkOrder(
+              query_id_,
+              input_relation_,
+              input_relation_block_ids_[num_workorders_generated_],
+              build_side_predicate,
+              storage_manager,
+              CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context),
+              CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+bool BuildLIPFilterOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id block : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(block), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* BuildLIPFilterOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::BUILD_LIP_FILTER);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::BuildLIPFilterWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::BuildLIPFilterWorkOrder::build_block_id, block);
+  proto->SetExtension(serialization::BuildLIPFilterWorkOrder::build_side_predicate_index,
+                      build_side_predicate_index_);
+  proto->SetExtension(serialization::BuildLIPFilterWorkOrder::lip_deployment_index, lip_deployment_index_);
+
+  return proto;
+}
+
+void BuildLIPFilterWorkOrder::execute() {
+  BlockReference block(
+      storage_manager_->getBlock(build_block_id_, input_relation_));
+
+  // Apply the predicate first.
+  std::unique_ptr<TupleIdSequence> predicate_matches;
+  if (build_side_predicate_ != nullptr) {
+    predicate_matches.reset(block->getMatchesForPredicate(build_side_predicate_));
+  }
+
+  std::unique_ptr<ValueAccessor> accessor(
+      block->getTupleStorageSubBlock().createValueAccessor(predicate_matches.get()));
+
+  if (lip_filter_adaptive_prober_ != nullptr) {
+    // Probe the LIP filters if there are any. Note that the LIP filters to be
+    // probed are for filtering the input relation. They are distinct from the
+    // target LIP filters we are building.
+    std::unique_ptr<TupleIdSequence> matches(
+        lip_filter_adaptive_prober_->filterValueAccessor(accessor.get()));
+    std::unique_ptr<ValueAccessor> filtered_accessor(
+        accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+
+    lip_filter_builder_->insertValueAccessor(filtered_accessor.get());
+  } else {
+    lip_filter_builder_->insertValueAccessor(accessor.get());
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/relational_operators/BuildLIPFilterOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildLIPFilterOperator.hpp b/relational_operators/BuildLIPFilterOperator.hpp
new file mode 100644
index 0000000..5192b40
--- /dev/null
+++ b/relational_operators/BuildLIPFilterOperator.hpp
@@ -0,0 +1,200 @@
+/**
+ * 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_RELATIONAL_OPERATORS_BUILD_LIP_FILTER_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_BUILD_LIP_FILTER_OPERATOR_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/Macros.hpp"
+#include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
+#include "utility/lip_filter/LIPFilterBuilder.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class CatalogRelationSchema;
+class Predicate;
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+namespace serialization { class WorkOrder; }
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+/**
+ * @brief An operator which builds LIPFilters on one relation.
+ **/
+class BuildLIPFilterOperator : public RelationalOperator {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @note The LIPFilters' information are not passed explicitly as parameters
+   *       to this constructor, but attached later via RelationalOperator::deployLIPFilters().
+   *
+   * @param query_id The ID of the query to which this operator belongs.
+   * @param input_relation The relation to build LIP filters on.
+   * @param build_side_predicate_index The index of the predicate in QueryContext
+   *        where the predicate is to be applied to the input relation before
+   *        building the LIP filters (or kInvalidPredicateId if no predicate is
+   *        to be applied).
+   * @param input_relation_is_stored If input_relation is a stored relation and
+   *        is fully available to the operator before it can start generating
+   *        workorders.
+   **/
+  BuildLIPFilterOperator(const std::size_t query_id,
+                         const CatalogRelation &input_relation,
+                         const QueryContext::predicate_id build_side_predicate_index,
+                         const bool input_relation_is_stored)
+    : RelationalOperator(query_id),
+      input_relation_(input_relation),
+      build_side_predicate_index_(build_side_predicate_index),
+      input_relation_is_stored_(input_relation_is_stored),
+      input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
+                                                         : std::vector<block_id>()),
+      num_workorders_generated_(0),
+      started_(false) {}
+
+  ~BuildLIPFilterOperator() override {}
+
+  /**
+   * @return The input relation to build LIP filters on.
+   */
+  const CatalogRelation& input_relation() const {
+    return input_relation_;
+  }
+
+  std::string getName() const override {
+    return "BuildLIPFilterOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+  void feedInputBlock(const block_id input_block_id,
+                      const relation_id input_relation_id,
+                      const partition_id part_id) override {
+    input_relation_block_ids_.push_back(input_block_id);
+  }
+
+ private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
+  const CatalogRelation &input_relation_;
+  const QueryContext::predicate_id build_side_predicate_index_;
+  const bool input_relation_is_stored_;
+
+  std::vector<block_id> input_relation_block_ids_;
+  std::vector<block_id>::size_type num_workorders_generated_;
+
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildLIPFilterOperator);
+};
+
+/**
+ * @brief A WorkOrder produced by BuildLIPFilterOperator.
+ **/
+class BuildLIPFilterWorkOrder : public WorkOrder {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
+   * @param input_relation The relation to build LIP filters on.
+   * @param build_block_id The block id.
+   * @param build_side_predicate The predicate to be applied to filter the input
+   *        relation before building the LIP filters (or nullptr if no predicate
+   *        is to be applied).
+   * @param storage_manager The StorageManager to use.
+   * @param lip_filter_adaptive_prober The attached LIP filter prober.
+   * @param lip_filter_builder The attached LIP filter builder.
+   **/
+  BuildLIPFilterWorkOrder(const std::size_t query_id,
+                          const CatalogRelationSchema &input_relation,
+                          const block_id build_block_id,
+                          const Predicate *build_side_predicate,
+                          StorageManager *storage_manager,
+                          LIPFilterAdaptiveProber *lip_filter_adaptive_prober,
+                          LIPFilterBuilder *lip_filter_builder)
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
+        build_block_id_(build_block_id),
+        build_side_predicate_(build_side_predicate),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)),
+        lip_filter_adaptive_prober_(lip_filter_adaptive_prober),
+        lip_filter_builder_(DCHECK_NOTNULL(lip_filter_builder)) {}
+
+  ~BuildLIPFilterWorkOrder() override {}
+
+  /**
+   * @return The input relation to build LIP filters on.
+   */
+  const CatalogRelationSchema& input_relation() const {
+    return input_relation_;
+  }
+
+  void execute() override;
+
+ private:
+  const CatalogRelationSchema &input_relation_;
+  const block_id build_block_id_;
+  const Predicate *build_side_predicate_;
+
+  StorageManager *storage_manager_;
+
+  std::unique_ptr<LIPFilterAdaptiveProber> lip_filter_adaptive_prober_;
+  std::unique_ptr<LIPFilterBuilder> lip_filter_builder_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildLIPFilterWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_BUILD_LIP_FILTER_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c8447f3..c18dc77 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -34,6 +34,7 @@ set_gflags_lib_name ()
 # Declare micro-libs:
 add_library(quickstep_relationaloperators_AggregationOperator AggregationOperator.cpp AggregationOperator.hpp)
 add_library(quickstep_relationaloperators_BuildHashOperator BuildHashOperator.cpp BuildHashOperator.hpp)
+add_library(quickstep_relationaloperators_BuildLIPFilterOperator BuildLIPFilterOperator.cpp BuildLIPFilterOperator.hpp)
 add_library(quickstep_relationaloperators_CreateIndexOperator CreateIndexOperator.cpp CreateIndexOperator.hpp)
 add_library(quickstep_relationaloperators_CreateTableOperator CreateTableOperator.cpp CreateTableOperator.hpp)
 add_library(quickstep_relationaloperators_DestroyAggregationStateOperator
@@ -113,6 +114,27 @@ target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       quickstep_utility_lipfilter_LIPFilterBuilder
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_BuildLIPFilterOperator
+                      glog
+                      quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_storage_StorageBlock
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageManager
+                      quickstep_storage_TupleIdSequence
+                      quickstep_storage_TupleStorageSubBlock
+                      quickstep_storage_ValueAccessor
+                      quickstep_utility_Macros
+                      quickstep_utility_lipfilter_LIPFilterAdaptiveProber
+                      quickstep_utility_lipfilter_LIPFilterBuilder
+                      quickstep_utility_lipfilter_LIPFilterUtil
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_CreateIndexOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -483,6 +505,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       quickstep_queryexecution_QueryContext
                       quickstep_relationaloperators_AggregationOperator
                       quickstep_relationaloperators_BuildHashOperator
+                      quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_DeleteOperator
                       quickstep_relationaloperators_DestroyAggregationStateOperator
                       quickstep_relationaloperators_DestroyHashOperator
@@ -515,6 +538,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrder_proto
 add_library(quickstep_relationaloperators ../empty_src.cpp RelationalOperatorsModule.hpp)
 target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_AggregationOperator
+                      quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_CreateIndexOperator
                       quickstep_relationaloperators_CreateTableOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index f8d9246..76753d2 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -24,25 +24,26 @@ import "relational_operators/SortMergeRunOperator.proto";
 enum WorkOrderType {
   AGGREGATION = 1;
   BUILD_HASH = 2;
-  CREATE_INDEX = 3;  // Placeholder.
-  CREATE_TABLE = 4;  // Placeholder.
-  DELETE = 5;
-  DESTROY_HASH = 6;
-  DROP_TABLE = 7;
-  FINALIZE_AGGREGATION = 8;
-  HASH_JOIN = 9;
-  INSERT = 10;
-  NESTED_LOOP_JOIN = 11;
-  SAMPLE = 12;
-  SAVE_BLOCKS = 13;
-  SELECT = 14;
-  SORT_MERGE_RUN = 15;
-  SORT_RUN_GENERATION = 16;
-  TABLE_GENERATOR = 17;
-  TEXT_SCAN = 18;
-  UPDATE = 19;
-  WINDOW_AGGREGATION = 20;
-  DESTROY_AGGREGATION_STATE = 21;
+  BUILD_LIP_FILTER = 3;
+  CREATE_INDEX = 4;  // Placeholder.
+  CREATE_TABLE = 5;  // Placeholder.
+  DELETE = 6;
+  DESTROY_HASH = 7;
+  DROP_TABLE = 8;
+  FINALIZE_AGGREGATION = 9;
+  HASH_JOIN = 10;
+  INSERT = 11;
+  NESTED_LOOP_JOIN = 12;
+  SAMPLE = 13;
+  SAVE_BLOCKS = 14;
+  SELECT = 15;
+  SORT_MERGE_RUN = 16;
+  SORT_RUN_GENERATION = 17;
+  TABLE_GENERATOR = 18;
+  TEXT_SCAN = 19;
+  UPDATE = 20;
+  WINDOW_AGGREGATION = 21;
+  DESTROY_AGGREGATION_STATE = 22;
 }
 
 message WorkOrder {
@@ -77,6 +78,16 @@ message BuildHashWorkOrder {
   }
 }
 
+message BuildLIPFilterWorkOrder {
+  extend WorkOrder {
+    // All required.
+    optional int32 relation_id = 48;
+    optional fixed64 build_block_id = 49;
+    optional int32 build_side_predicate_index = 50;
+    optional int32 lip_deployment_index = 51;
+  }
+}
+
 message DeleteWorkOrder {
   extend WorkOrder {
     // All required.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index a6cba02..5e8d03d 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -30,6 +30,7 @@
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/AggregationOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
+#include "relational_operators/BuildLIPFilterOperator.hpp"
 #include "relational_operators/DeleteOperator.hpp"
 #include "relational_operators/DestroyAggregationStateOperator.hpp"
 #include "relational_operators/DestroyHashOperator.hpp"
@@ -90,6 +91,23 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           CreateLIPFilterAdaptiveProberHelper(
               proto.GetExtension(serialization::AggregationWorkOrder::lip_deployment_index), query_context));
     }
+    case serialization::BUILD_LIP_FILTER: {
+      LOG(INFO) << "Creating BuildLIPFilterWorkOrder in Shiftboss " << shiftboss_index;
+
+      const QueryContext::lip_deployment_id lip_deployment_index =
+          proto.GetExtension(serialization::BuildLIPFilterWorkOrder::lip_deployment_index);
+
+      return new BuildLIPFilterWorkOrder(
+          proto.query_id(),
+          catalog_database->getRelationSchemaById(
+              proto.GetExtension(serialization::BuildLIPFilterWorkOrder::relation_id)),
+          proto.GetExtension(serialization::BuildLIPFilterWorkOrder::build_block_id),
+          query_context->getPredicate(
+              proto.GetExtension(serialization::BuildLIPFilterWorkOrder::build_side_predicate_index)),
+          storage_manager,
+          CreateLIPFilterAdaptiveProberHelper(lip_deployment_index, query_context),
+          CreateLIPFilterBuilderHelper(lip_deployment_index, query_context));
+    }
     case serialization::BUILD_HASH: {
       LOG(INFO) << "Creating BuildHashWorkOrder in Shiftboss " << shiftboss_index;
       vector<attribute_id> join_key_attributes;
@@ -541,6 +559,33 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
                  proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index),
                  proto.GetExtension(serialization::BuildHashWorkOrder::partition_id));
     }
+    case serialization::BUILD_LIP_FILTER: {
+      if (!proto.HasExtension(serialization::BuildLIPFilterWorkOrder::relation_id)) {
+        return false;
+      }
+
+      const relation_id rel_id =
+          proto.GetExtension(serialization::BuildLIPFilterWorkOrder::relation_id);
+      if (!catalog_database.hasRelationWithId(rel_id)) {
+        return false;
+      }
+
+      if (!proto.HasExtension(serialization::BuildLIPFilterWorkOrder::lip_deployment_index)) {
+        return false;
+      } else {
+        const QueryContext::lip_deployment_id lip_deployment_index =
+            proto.GetExtension(serialization::BuildLIPFilterWorkOrder::lip_deployment_index);
+        if (lip_deployment_index != QueryContext::kInvalidLIPDeploymentId &&
+            !query_context.isValidLIPDeploymentId(lip_deployment_index)) {
+          return false;
+        }
+      }
+
+      return proto.HasExtension(serialization::BuildLIPFilterWorkOrder::build_block_id) &&
+             proto.HasExtension(serialization::BuildLIPFilterWorkOrder::build_side_predicate_index) &&
+             query_context.isValidPredicate(
+                 proto.GetExtension(serialization::BuildLIPFilterWorkOrder::build_side_predicate_index));
+    }
     case serialization::DELETE: {
       return proto.HasExtension(serialization::DeleteWorkOrder::relation_id) &&
              catalog_database.hasRelationWithId(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 8571149..aeff388 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -270,6 +270,7 @@ target_link_libraries(quickstep_utility_PlanVisualizer
                       quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_LIPFilterConfiguration
                       quickstep_queryoptimizer_physical_Physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index df7a20c..f8bf6f8 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -32,6 +32,7 @@
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -58,6 +59,8 @@ std::string PlanVisualizer::visualize(const P::PhysicalPtr &input) {
 
   color_map_["TableReference"] = "skyblue";
   color_map_["Selection"] = "#90EE90";
+  color_map_["FilterJoin"] = "pink";
+  color_map_["FilterJoin(Anti)"] = "pink";
   color_map_["HashJoin"] = "red";
   color_map_["HashLeftOuterJoin"] = "orange";
   color_map_["HashLeftSemiJoin"] = "orange";
@@ -126,7 +129,8 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     edge_info.dst_node_id = node_id;
     edge_info.dashed = false;
 
-    if (input->getPhysicalType() == P::PhysicalType::kHashJoin &&
+    if ((input->getPhysicalType() == P::PhysicalType::kHashJoin ||
+         input->getPhysicalType() == P::PhysicalType::kFilterJoin) &&
         child == input->children()[1]) {
       edge_info.dashed = true;
     }
@@ -165,6 +169,20 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
       }
       break;
     }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(input);
+      node_info.labels.emplace_back(input->getName());
+
+      const auto &probe_attributes = filter_join->probe_attributes();
+      const auto &build_attributes = filter_join->build_attributes();
+      for (std::size_t i = 0; i < probe_attributes.size(); ++i) {
+        node_info.labels.emplace_back(
+            probe_attributes[i]->attribute_alias() + " = " +
+                build_attributes[i]->attribute_alias());
+      }
+      break;
+    }
     default: {
       node_info.labels.emplace_back(input->getName());
       break;
@@ -177,7 +195,7 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     if (build_it != build_filters.end()) {
       for (const auto &build_info : build_it->second) {
         node_info.labels.emplace_back(
-            std::string("[LIP build] ") + build_info.build_attribute->attribute_alias());
+            std::string("[LIP build] ") + build_info->build_attribute()->attribute_alias());
       }
     }
     const auto &probe_filters = lip_filter_conf_->getProbeInfoMap();
@@ -185,7 +203,7 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     if (probe_it != probe_filters.end()) {
       for (const auto &probe_info : probe_it->second) {
         node_info.labels.emplace_back(
-            std::string("[LIP probe] ") + probe_info.probe_attribute->attribute_alias());
+            std::string("[LIP probe] ") + probe_info->probe_attribute()->attribute_alias());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/BitVectorExactFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/BitVectorExactFilter.hpp b/utility/lip_filter/BitVectorExactFilter.hpp
new file mode 100644
index 0000000..6ad0567
--- /dev/null
+++ b/utility/lip_filter/BitVectorExactFilter.hpp
@@ -0,0 +1,202 @@
+/**
+ * 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_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_
+#define QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_
+
+#include <atomic>
+#include <cstdint>
+#include <cstring>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "utility/Macros.hpp"
+#include "utility/lip_filter/LIPFilter.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A LIP filter that tests the EXACT memberships of elements, i.e. there
+ *        will be neither false positives nor false negatives. The implementation
+ *        is to simply reinterpret_cast a value's byte stream into the specified
+ *        CppType as the underlying bit vector's index. Therefore, to use this
+ *        filter, the corresponding LIP attribute's values must be bounded in a
+ *        reasonably small integer range.
+ */
+template <typename CppType, bool is_anti_filter>
+class BitVectorExactFilter : public LIPFilter {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param min_value The minimum possible value for this filter to set.
+   * @param max_value The maximum possible value for this filter to set.
+   */
+  explicit BitVectorExactFilter(const std::int64_t min_value,
+                                const std::int64_t max_value)
+      : LIPFilter(LIPFilterType::kBitVectorExactFilter),
+        min_value_(static_cast<CppType>(min_value)),
+        max_value_(static_cast<CppType>(max_value)),
+        bit_array_(GetByteSize(max_value - min_value + 1)) {
+    DCHECK_EQ(min_value, static_cast<std::int64_t>(min_value_));
+    DCHECK_EQ(max_value, static_cast<std::int64_t>(max_value_));
+    DCHECK_GE(max_value_, min_value_);
+
+    std::memset(bit_array_.data(),
+                0x0,
+                sizeof(std::atomic<std::uint8_t>) * GetByteSize(max_value - min_value + 1));
+  }
+
+  void insertValueAccessor(ValueAccessor *accessor,
+                           const attribute_id attr_id,
+                           const Type *attr_type) override {
+    InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      if (attr_type->isNullable()) {
+        this->insertValueAccessorInternal<true>(accessor, attr_id);
+      } else {
+        this->insertValueAccessorInternal<false>(accessor, attr_id);
+      }
+    });
+  }
+
+  std::size_t filterBatch(ValueAccessor *accessor,
+                          const attribute_id attr_id,
+                          const bool is_attr_nullable,
+                          std::vector<tuple_id> *batch,
+                          const std::size_t batch_size) const override {
+    DCHECK(batch != nullptr);
+    DCHECK_LE(batch_size, batch->size());
+
+    return InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> std::size_t {  // NOLINT(build/c++11)
+      if (is_attr_nullable) {
+        return this->filterBatchInternal<true>(accessor, attr_id, batch, batch_size);
+      } else {
+        return this->filterBatchInternal<false>(accessor, attr_id, batch, batch_size);
+      }
+    });
+  }
+
+ private:
+  /**
+   * @brief Round up bit_size to multiples of 8.
+   */
+  inline static std::size_t GetByteSize(const std::size_t bit_size) {
+    return (bit_size + 7u) / 8u;
+  }
+
+  /**
+   * @brief Iterate through the accessor and hash values into the internal bit
+   *        array.
+   */
+  template <bool is_attr_nullable, typename ValueAccessorT>
+  inline void insertValueAccessorInternal(ValueAccessorT *accessor,
+                                          const attribute_id attr_id) {
+    accessor->beginIteration();
+    while (accessor->next()) {
+      const void *value = accessor->template getUntypedValue<is_attr_nullable>(attr_id);
+      if (!is_attr_nullable || value != nullptr) {
+        insert(value);
+      }
+    }
+  }
+
+  /**
+   * @brief Filter the given batch of tuples from a ValueAccessor. Write the
+   *        tuple ids which survive in the filtering back to \p batch.
+   */
+  template <bool is_attr_nullable, typename ValueAccessorT>
+  inline std::size_t filterBatchInternal(const ValueAccessorT *accessor,
+                                         const attribute_id attr_id,
+                                         std::vector<tuple_id> *batch,
+                                         const std::size_t batch_size) const {
+    std::size_t out_size = 0;
+    for (std::size_t i = 0; i < batch_size; ++i) {
+      const tuple_id tid = batch->at(i);
+      const void *value =
+          accessor->template getUntypedValueAtAbsolutePosition(attr_id, tid);
+      if (is_attr_nullable && value == nullptr) {
+        continue;
+      }
+      if (contains(value)) {
+        batch->at(out_size) = tid;
+        ++out_size;
+      }
+    }
+    return out_size;
+  }
+
+  /**
+   * @brief Inserts a given value into the exact filter.
+   */
+  inline void insert(const void *key_begin) {
+    const CppType value = *reinterpret_cast<const CppType *>(key_begin);
+    DCHECK_GE(value, min_value_);
+    DCHECK_LE(value, max_value_);
+
+    const CppType loc = value - min_value_;
+    bit_array_[loc >> 3u].fetch_or(1u << (loc & 7u), std::memory_order_relaxed);
+  }
+
+  /**
+   * @brief Test membership of a given value in the exact filter.
+   */
+  inline bool contains(const void *key_begin) const {
+    const CppType value = *reinterpret_cast<const CppType *>(key_begin);
+    if (value < min_value_ || value > max_value_) {
+      return is_anti_filter;
+    }
+
+    const CppType loc = value - min_value_;
+    const bool is_bit_set =
+        (bit_array_[loc >> 3u].load(std::memory_order_relaxed) & (1u << (loc & 7u))) != 0;
+
+    if (is_anti_filter) {
+      return !is_bit_set;
+    } else {
+      return is_bit_set;
+    }
+  }
+
+  const CppType min_value_;
+  const CppType max_value_;
+  alignas(kCacheLineBytes) std::vector<std::atomic<std::uint8_t>> bit_array_;
+
+  DISALLOW_COPY_AND_ASSIGN(BitVectorExactFilter);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/lip_filter/CMakeLists.txt b/utility/lip_filter/CMakeLists.txt
index 23b3763..edd0d24 100644
--- a/utility/lip_filter/CMakeLists.txt
+++ b/utility/lip_filter/CMakeLists.txt
@@ -20,6 +20,7 @@ QS_PROTOBUF_GENERATE_CPP(utility_lipfilter_LIPFilter_proto_srcs
                          LIPFilter.proto)
 
 # Declare micro-libs:
+add_library(quickstep_utility_lipfilter_BitVectorExactFilter ../../empty_src.cpp BitVectorExactFilter.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilter ../../empty_src.cpp LIPFilter.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilterAdaptiveProber ../../empty_src.cpp LIPFilterAdaptiveProber.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilterBuilder ../../empty_src.cpp LIPFilterBuilder.hpp)
@@ -31,6 +32,15 @@ add_library(quickstep_utility_lipfilter_LIPFilter_proto
 add_library(quickstep_utility_lipfilter_SingleIdentityHashFilter ../../empty_src.cpp SingleIdentityHashFilter.hpp)
 
 # Link dependencies:
+target_link_libraries(quickstep_utility_lipfilter_BitVectorExactFilter
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_utility_lipfilter_LIPFilter
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilter
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
@@ -56,6 +66,7 @@ target_link_libraries(quickstep_utility_lipfilter_LIPFilterDeployment
                       quickstep_utility_lipfilter_LIPFilterBuilder
                       quickstep_utility_lipfilter_LIPFilter_proto)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilterFactory
+                      quickstep_utility_lipfilter_BitVectorExactFilter
                       quickstep_utility_lipfilter_LIPFilter_proto
                       quickstep_utility_lipfilter_SingleIdentityHashFilter
                       quickstep_utility_Macros)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/LIPFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilter.hpp b/utility/lip_filter/LIPFilter.hpp
index 682d69f..ba38264 100644
--- a/utility/lip_filter/LIPFilter.hpp
+++ b/utility/lip_filter/LIPFilter.hpp
@@ -37,8 +37,8 @@ class ValueAccessor;
  */
 
 enum class LIPFilterType {
+  kBitVectorExactFilter,
   kBloomFilter,
-  kExactFilter,
   kSingleIdentityHashFilter
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/LIPFilter.proto
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilter.proto b/utility/lip_filter/LIPFilter.proto
index def13dd..45843f3 100644
--- a/utility/lip_filter/LIPFilter.proto
+++ b/utility/lip_filter/LIPFilter.proto
@@ -22,8 +22,8 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 enum LIPFilterType {
-  BLOOM_FILTER = 1;
-  EXACT_FILTER = 2;
+  BIT_VECTOR_EXACT_FILTER = 1;
+  BLOOM_FILTER = 2;
   SINGLE_IDENTITY_HASH_FILTER = 3;
 }
 
@@ -33,17 +33,22 @@ message LIPFilter {
   extensions 16 to max;
 }
 
-message SingleIdentityHashFilter {
+message BitVectorExactFilter {
   extend LIPFilter {
     // All required
-    optional uint64 filter_cardinality = 16;
-    optional uint64 attribute_size = 17;
+    optional int64 min_value = 16;
+    optional int64 max_value = 17;
+    optional uint64 attribute_size = 18;
+    optional bool is_anti_filter = 19;
   }
 }
 
-enum LIPFilterActionType {
-  BUILD = 1;
-  PROBE = 2;
+message SingleIdentityHashFilter {
+  extend LIPFilter {
+    // All required
+    optional uint64 filter_cardinality = 24;
+    optional uint64 attribute_size = 25;
+  }
 }
 
 message LIPFilterDeployment {
@@ -53,6 +58,6 @@ message LIPFilterDeployment {
     required Type attribute_type = 3;
   }
 
-  required LIPFilterActionType action_type = 1;
-  repeated Entry entries = 2;
+  repeated Entry build_entries = 1;
+  repeated Entry probe_entries = 2;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/LIPFilterDeployment.cpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterDeployment.cpp b/utility/lip_filter/LIPFilterDeployment.cpp
index cd4d90f..5721496 100644
--- a/utility/lip_filter/LIPFilterDeployment.cpp
+++ b/utility/lip_filter/LIPFilterDeployment.cpp
@@ -28,45 +28,49 @@
 #include "utility/lip_filter/LIPFilterBuilder.hpp"
 #include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
 
-#include "glog/logging.h"
-
 namespace quickstep {
 
 LIPFilterDeployment::LIPFilterDeployment(
     const serialization::LIPFilterDeployment &proto,
     const std::vector<std::unique_ptr<LIPFilter>> &lip_filters) {
-  switch (proto.action_type()) {
-    case serialization::LIPFilterActionType::BUILD:
-      action_type_ = LIPFilterActionType::kBuild;
-      break;
-    case serialization::LIPFilterActionType::PROBE:
-      action_type_ = LIPFilterActionType::kProbe;
-      break;
-    default:
-      LOG(FATAL) << "Unsupported LIPFilterActionType: "
-                 << serialization::LIPFilterActionType_Name(proto.action_type());
+  if (proto.build_entries_size() > 0) {
+    build_.reset(new LIPFilterDeploymentInfo());
+    for (int i = 0; i < proto.build_entries_size(); ++i) {
+      const auto &entry_proto = proto.build_entries(i);
+      build_->lip_filters_.emplace_back(
+          lip_filters.at(entry_proto.lip_filter_id()).get());
+      build_->attr_ids_.emplace_back(entry_proto.attribute_id());
+      build_->attr_types_.emplace_back(
+          &TypeFactory::ReconstructFromProto(entry_proto.attribute_type()));
+    }
   }
 
-  for (int i = 0; i < proto.entries_size(); ++i) {
-    const auto &entry_proto = proto.entries(i);
-    lip_filters_.emplace_back(lip_filters.at(entry_proto.lip_filter_id()).get());
-    attr_ids_.emplace_back(entry_proto.attribute_id());
-    attr_types_.emplace_back(&TypeFactory::ReconstructFromProto(entry_proto.attribute_type()));
+  if (proto.probe_entries_size() > 0) {
+    probe_.reset(new LIPFilterDeploymentInfo());
+    for (int i = 0; i < proto.probe_entries_size(); ++i) {
+      const auto &entry_proto = proto.probe_entries(i);
+      probe_->lip_filters_.emplace_back(
+          lip_filters.at(entry_proto.lip_filter_id()).get());
+      probe_->attr_ids_.emplace_back(entry_proto.attribute_id());
+      probe_->attr_types_.emplace_back(
+          &TypeFactory::ReconstructFromProto(entry_proto.attribute_type()));
+    }
   }
 }
 
 bool LIPFilterDeployment::ProtoIsValid(
     const serialization::LIPFilterDeployment &proto) {
-  if (proto.action_type() != serialization::LIPFilterActionType::BUILD &&
-      proto.action_type() != serialization::LIPFilterActionType::PROBE) {
-    LOG(FATAL) << "Unsupported LIPFilterActionType: "
-               << serialization::LIPFilterActionType_Name(proto.action_type());
-  }
-  if (proto.entries_size() == 0) {
+  if (proto.build_entries_size() == 0 && proto.probe_entries_size() == 0) {
     return false;
   }
-  for (int i = 0; i < proto.entries_size(); ++i) {
-    const auto &entry_proto = proto.entries(i);
+  for (int i = 0; i < proto.build_entries_size(); ++i) {
+    const auto &entry_proto = proto.build_entries(i);
+    if (!TypeFactory::ProtoIsValid(entry_proto.attribute_type())) {
+      return false;
+    }
+  }
+  for (int i = 0; i < proto.probe_entries_size(); ++i) {
+    const auto &entry_proto = proto.probe_entries(i);
     if (!TypeFactory::ProtoIsValid(entry_proto.attribute_type())) {
       return false;
     }
@@ -75,13 +79,23 @@ bool LIPFilterDeployment::ProtoIsValid(
 }
 
 LIPFilterBuilder* LIPFilterDeployment::createLIPFilterBuilder() const {
-  DCHECK(action_type_ == LIPFilterActionType::kBuild);
-  return new LIPFilterBuilder(lip_filters_, attr_ids_, attr_types_);
+  if (build_ == nullptr) {
+    return nullptr;
+  }
+
+  return new LIPFilterBuilder(build_->lip_filters_,
+                              build_->attr_ids_,
+                              build_->attr_types_);
 }
 
 LIPFilterAdaptiveProber* LIPFilterDeployment::createLIPFilterAdaptiveProber() const {
-  DCHECK(action_type_ == LIPFilterActionType::kProbe);
-  return new LIPFilterAdaptiveProber(lip_filters_, attr_ids_, attr_types_);
+  if (probe_ == nullptr) {
+    return nullptr;
+  }
+
+  return new LIPFilterAdaptiveProber(probe_->lip_filters_,
+                                     probe_->attr_ids_,
+                                     probe_->attr_types_);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/LIPFilterDeployment.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterDeployment.hpp b/utility/lip_filter/LIPFilterDeployment.hpp
index 9b37f88..ab1259b 100644
--- a/utility/lip_filter/LIPFilterDeployment.hpp
+++ b/utility/lip_filter/LIPFilterDeployment.hpp
@@ -39,11 +39,6 @@ class Type;
  *  @{
  */
 
-enum class LIPFilterActionType {
-  kBuild = 0,
-  kProbe
-};
-
 /**
  * @brief Helper class for organizing a group of LIPFilters in the backend.
  *        Each LIPFilterDeployment object is attached to a RelationalOperator.
@@ -69,16 +64,6 @@ class LIPFilterDeployment {
   static bool ProtoIsValid(const serialization::LIPFilterDeployment &proto);
 
   /**
-   * @brief Get the action type for this group of LIPFilters (i.e. whether
-   *        to build or probe the filters).
-   *
-   * @return The action type.
-   */
-  LIPFilterActionType getActionType() const {
-    return action_type_;
-  }
-
-  /**
    * @brief Create a LIPFilterBuilder for this group of LIPFilters.
    *
    * @return A new LIPFilterBuilder object for this group of LIPFilters.
@@ -95,11 +80,14 @@ class LIPFilterDeployment {
   LIPFilterAdaptiveProber* createLIPFilterAdaptiveProber() const;
 
  private:
-  LIPFilterActionType action_type_;
-
-  std::vector<LIPFilter *> lip_filters_;
-  std::vector<attribute_id> attr_ids_;
-  std::vector<const Type *> attr_types_;
+  struct LIPFilterDeploymentInfo {
+    std::vector<LIPFilter *> lip_filters_;
+    std::vector<attribute_id> attr_ids_;
+    std::vector<const Type *> attr_types_;
+  };
+
+  std::unique_ptr<LIPFilterDeploymentInfo> build_;
+  std::unique_ptr<LIPFilterDeploymentInfo> probe_;
 
   DISALLOW_COPY_AND_ASSIGN(LIPFilterDeployment);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4ba819c5/utility/lip_filter/LIPFilterFactory.cpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterFactory.cpp b/utility/lip_filter/LIPFilterFactory.cpp
index ebc4a0e..f69d8b0 100644
--- a/utility/lip_filter/LIPFilterFactory.cpp
+++ b/utility/lip_filter/LIPFilterFactory.cpp
@@ -23,6 +23,7 @@
 #include <cstdint>
 
 #include "utility/lip_filter/LIPFilter.pb.h"
+#include "utility/lip_filter/BitVectorExactFilter.hpp"
 #include "utility/lip_filter/SingleIdentityHashFilter.hpp"
 
 #include "glog/logging.h"
@@ -31,6 +32,46 @@ namespace quickstep {
 
 LIPFilter* LIPFilterFactory::ReconstructFromProto(const serialization::LIPFilter &proto) {
   switch (proto.lip_filter_type()) {
+    case serialization::LIPFilterType::BIT_VECTOR_EXACT_FILTER: {
+      const std::size_t attr_size =
+          proto.GetExtension(serialization::BitVectorExactFilter::attribute_size);
+      const std::int64_t min_value =
+          proto.GetExtension(serialization::BitVectorExactFilter::min_value);
+      const std::int64_t max_value =
+          proto.GetExtension(serialization::BitVectorExactFilter::max_value);
+      const bool is_anti_filter =
+          proto.GetExtension(serialization::BitVectorExactFilter::is_anti_filter);
+
+      switch (attr_size) {
+        case 1:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::int8_t, true>(min_value, max_value);
+          } else {
+            return new BitVectorExactFilter<std::int8_t, false>(min_value, max_value);
+          }
+        case 2:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::int16_t, true>(min_value, max_value);
+          } else {
+            return new BitVectorExactFilter<std::int16_t, false>(min_value, max_value);
+          }
+        case 4:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::int32_t, true>(min_value, max_value);
+          } else {
+            return new BitVectorExactFilter<std::int32_t, false>(min_value, max_value);
+          }
+        case 8:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::int64_t, true>(min_value, max_value);
+          } else {
+            return new BitVectorExactFilter<std::int64_t, false>(min_value, max_value);
+          }
+        default:
+          LOG(FATAL) << "Invalid attribute size for BitVectorExactFilter: "
+                     << attr_size;
+      }
+    }
     case serialization::LIPFilterType::SINGLE_IDENTITY_HASH_FILTER: {
       const std::size_t attr_size =
           proto.GetExtension(serialization::SingleIdentityHashFilter::attribute_size);
@@ -57,6 +98,15 @@ LIPFilter* LIPFilterFactory::ReconstructFromProto(const serialization::LIPFilter
 
 bool LIPFilterFactory::ProtoIsValid(const serialization::LIPFilter &proto) {
   switch (proto.lip_filter_type()) {
+    case serialization::LIPFilterType::BIT_VECTOR_EXACT_FILTER: {
+      const std::size_t attr_size =
+          proto.GetExtension(serialization::BitVectorExactFilter::attribute_size);
+      const std::int64_t min_value =
+          proto.GetExtension(serialization::BitVectorExactFilter::min_value);
+      const std::int64_t max_value =
+          proto.GetExtension(serialization::BitVectorExactFilter::max_value);
+      return (attr_size != 0 && max_value >= min_value);
+    }
     case serialization::LIPFilterType::SINGLE_IDENTITY_HASH_FILTER: {
       const std::size_t attr_size =
           proto.GetExtension(serialization::SingleIdentityHashFilter::attribute_size);


[32/50] [abbrv] incubator-quickstep git commit: Fixed a memory leak in the distributed version.

Posted by ji...@apache.org.
Fixed a memory leak in 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/3b65b0fd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/3b65b0fd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/3b65b0fd

Branch: refs/heads/LIP-time-decomposition
Commit: 3b65b0fdf40efa47862323834917cc19f2478ba3
Parents: 9a95c23
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 00:35:43 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:35:52 2017 -0800

----------------------------------------------------------------------
 query_execution/PolicyEnforcerDistributed.cpp | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3b65b0fd/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 38b8a34..49a1d9a 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -240,6 +240,7 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
     TaggedMessage message(static_cast<const void*>(proto_bytes),
                           proto_length,
                           kQueryTeardownMessage);
+    free(proto_bytes);
 
     DLOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
                << "') to all Shiftbosses";


[02/50] [abbrv] incubator-quickstep git commit: Minor refactor for InsertDestinations.

Posted by ji...@apache.org.
Minor refactor for InsertDestinations.


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

Branch: refs/heads/LIP-time-decomposition
Commit: f2e77266edeaff38a60650b48836ff6ddb3b84ca
Parents: 0f4938c
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Jan 30 15:24:03 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jan 30 15:24:03 2017 -0800

----------------------------------------------------------------------
 storage/InsertDestination.cpp          | 17 ++++-------------
 storage/InsertDestination.hpp          |  4 +++-
 storage/InsertDestinationInterface.hpp |  2 +-
 storage/StorageBlock.hpp               |  2 +-
 4 files changed, 9 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 944998f..714e6e5 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -290,7 +290,6 @@ void InsertDestination::bulkInsertTuplesFromValueAccessors(
       ValueAccessor *accessor = p.first;
       std::vector<attribute_id> attribute_map = p.second;
 
-
       InvokeOnAnyValueAccessor(
           accessor,
           [&](auto *accessor) -> void {  // NOLINT(build/c++11)
@@ -621,11 +620,10 @@ void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor,
        &always_mark_full,
        &num_partitions](auto *accessor) -> void {  // NOLINT(build/c++11)
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
-    partition_membership.resize(num_partitions);
 
     // Create a tuple-id sequence for each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      partition_membership[partition].reset(new TupleIdSequence(accessor->getEndPosition()));
+      partition_membership.emplace_back(std::make_unique<TupleIdSequence>(accessor->getEndPosition()));
     }
 
     // Iterate over ValueAccessor for each tuple,
@@ -641,9 +639,8 @@ void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor,
     // TupleIdSequence.
     std::vector<std::unique_ptr<typename std::remove_pointer<
         decltype(accessor->createSharedTupleIdSequenceAdapter(*partition_membership.front()))>::type>> adapter;
-    adapter.resize(num_partitions);
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      adapter[partition].reset(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
+      adapter.emplace_back(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
     }
 
     // Bulk-insert into a block belonging to the partition.
@@ -678,11 +675,10 @@ void PartitionAwareInsertDestination::bulkInsertTuplesWithRemappedAttributes(
        &always_mark_full,
        &num_partitions](auto *accessor) -> void {  // NOLINT(build/c++11)
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
-    partition_membership.resize(num_partitions);
 
     // Create a tuple-id sequence for each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      partition_membership[partition].reset(new TupleIdSequence(accessor->getEndPosition()));
+      partition_membership.emplace_back(std::make_unique<TupleIdSequence>(accessor->getEndPosition()));
     }
 
     // Iterate over ValueAccessor for each tuple,
@@ -698,9 +694,8 @@ void PartitionAwareInsertDestination::bulkInsertTuplesWithRemappedAttributes(
     // TupleIdSequence.
     std::vector<std::unique_ptr<typename std::remove_pointer<
         decltype(accessor->createSharedTupleIdSequenceAdapter(*partition_membership.front()))>::type>> adapter;
-    adapter.resize(num_partitions);
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      adapter[partition].reset(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
+      adapter.emplace_back(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
     }
 
     // Bulk-insert into a block belonging to the partition.
@@ -742,10 +737,6 @@ void PartitionAwareInsertDestination::insertTuplesFromVector(std::vector<Tuple>:
   }
 }
 
-MutableBlockReference PartitionAwareInsertDestination::getBlockForInsertion() {
-  FATAL_ERROR("PartitionAwareInsertDestination::getBlockForInsertion needs a partition id as an argument.");
-}
-
 MutableBlockReference PartitionAwareInsertDestination::getBlockForInsertionInPartition(const partition_id part_id) {
   DCHECK_LT(part_id, partition_scheme_header_->getNumPartitions());
   SpinMutexLock lock(mutexes_for_partition_[part_id]);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index c3c40bd..6707192 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -539,7 +539,9 @@ class PartitionAwareInsertDestination : public InsertDestination {
                               std::vector<Tuple>::const_iterator end) override;
 
  protected:
-  MutableBlockReference getBlockForInsertion() override;
+  MutableBlockReference getBlockForInsertion() override {
+    LOG(FATAL) << "PartitionAwareInsertDestination::getBlockForInsertion needs a partition id as an argument.";
+  }
 
   /**
    * @brief Get a block to use for insertion from a partition.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/InsertDestinationInterface.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestinationInterface.hpp b/storage/InsertDestinationInterface.hpp
index b62d3e5..be6b0c2 100644
--- a/storage/InsertDestinationInterface.hpp
+++ b/storage/InsertDestinationInterface.hpp
@@ -131,7 +131,7 @@ class InsertDestinationInterface {
    *
    * @param accessor_attribute_map A vector of pairs of ValueAccessor and
    *        corresponding attribute map
-   *        The i-th attribute ID in the attr map for a value accessor is "n" 
+   *        The i-th attribute ID in the attr map for a value accessor is "n"
    *        if the attribute_id "i" in the output relation
    *        is the attribute_id "n" in corresponding input value accessor.
    *        Set the i-th element to kInvalidCatalogId if it doesn't come from

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index ed252c5..16ea50f 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -325,7 +325,7 @@ class StorageBlock : public StorageBlockBase {
    *       function with the appropriate attribute_map for each value
    *       accessor (InsertDestination::bulkInsertTuplesFromValueAccessors
    *       handles all the details) to insert tuples without an extra temp copy.
-   * 
+   *
    * @warning Must call bulkInsertPartialTuplesFinalize() to update the header,
    *          until which point, the insertion is not visible to others.
    * @warning The inserted tuples may be placed in sub-optimal locations in this


[22/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/PackedPayloadHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.cpp b/storage/PackedPayloadHashTable.cpp
new file mode 100644
index 0000000..bf5eaee
--- /dev/null
+++ b/storage/PackedPayloadHashTable.cpp
@@ -0,0 +1,463 @@
+/**
+ * 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 "storage/PackedPayloadHashTable.hpp"
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "storage/HashTableKeyManager.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "threading/SpinMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "types/Type.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/Alignment.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PrimeNumber.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+PackedPayloadHashTable::PackedPayloadHashTable(
+    const std::vector<const Type *> &key_types,
+    const std::size_t num_entries,
+    const std::vector<AggregationHandle *> &handles,
+    StorageManager *storage_manager)
+    : key_types_(key_types),
+      num_handles_(handles.size()),
+      handles_(handles),
+      total_payload_size_(ComputeTotalPayloadSize(handles)),
+      storage_manager_(storage_manager),
+      kBucketAlignment(alignof(std::atomic<std::size_t>)),
+      kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
+      key_manager_(key_types_, kValueOffset + total_payload_size_),
+      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
+  std::size_t payload_offset_running_sum = sizeof(SpinMutex);
+  for (const auto *handle : handles) {
+    payload_offsets_.emplace_back(payload_offset_running_sum);
+    payload_offset_running_sum += handle->getPayloadSize();
+  }
+
+  // NOTE(jianqiao): Potential memory leak / double freeing by copying from
+  // init_payload to buckets if payload contains out of line data.
+  init_payload_ =
+      static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
+  DCHECK(init_payload_ != nullptr);
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    handles_[i]->initPayload(init_payload_ + payload_offsets_[i]);
+  }
+
+  // Bucket size always rounds up to the alignment requirement of the atomic
+  // size_t "next" pointer at the front or a ValueT, whichever is larger.
+  //
+  // Give base HashTable information about what key components are stored
+  // inline from 'key_manager_'.
+  setKeyInline(key_manager_.getKeyInline());
+
+  // Pick out a prime number of slots and calculate storage requirements.
+  std::size_t num_slots_tmp =
+      get_next_prime_number(num_entries * kHashTableLoadFactor);
+  std::size_t required_memory =
+      sizeof(Header) + num_slots_tmp * sizeof(std::atomic<std::size_t>) +
+      (num_slots_tmp / kHashTableLoadFactor) *
+          (bucket_size_ + key_manager_.getEstimatedVariableKeySize());
+  std::size_t num_storage_slots =
+      this->storage_manager_->SlotsNeededForBytes(required_memory);
+  if (num_storage_slots == 0) {
+    FATAL_ERROR(
+        "Storage requirement for SeparateChainingHashTable "
+        "exceeds maximum allocation size.");
+  }
+
+  // Get a StorageBlob to hold the hash table.
+  const block_id blob_id =
+      this->storage_manager_->createBlob(num_storage_slots);
+  this->blob_ = this->storage_manager_->getBlobMutable(blob_id);
+
+  void *aligned_memory_start = this->blob_->getMemoryMutable();
+  std::size_t available_memory = num_storage_slots * kSlotSizeBytes;
+  if (align(alignof(Header),
+            sizeof(Header),
+            aligned_memory_start,
+            available_memory) == nullptr) {
+    // With current values from StorageConstants.hpp, this should be
+    // impossible. A blob is at least 1 MB, while a Header has alignment
+    // requirement of just kCacheLineBytes (64 bytes).
+    FATAL_ERROR(
+        "StorageBlob used to hold resizable "
+        "SeparateChainingHashTable is too small to meet alignment "
+        "requirements of SeparateChainingHashTable::Header.");
+  } else if (aligned_memory_start != this->blob_->getMemoryMutable()) {
+    // This should also be impossible, since the StorageManager allocates slots
+    // aligned to kCacheLineBytes.
+    DEV_WARNING("StorageBlob memory adjusted by "
+                << (num_storage_slots * kSlotSizeBytes - available_memory)
+                << " bytes to meet alignment requirement for "
+                << "SeparateChainingHashTable::Header.");
+  }
+
+  // Locate the header.
+  header_ = static_cast<Header *>(aligned_memory_start);
+  aligned_memory_start =
+      static_cast<char *>(aligned_memory_start) + sizeof(Header);
+  available_memory -= sizeof(Header);
+
+  // Recompute the number of slots & buckets using the actual available memory.
+  // Most likely, we got some extra free bucket space due to "rounding up" to
+  // the storage blob's size. It's also possible (though very unlikely) that we
+  // will wind up with fewer buckets than we initially wanted because of screwy
+  // alignment requirements for ValueT.
+  std::size_t num_buckets_tmp =
+      available_memory /
+      (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) + bucket_size_ +
+       key_manager_.getEstimatedVariableKeySize());
+  num_slots_tmp =
+      get_previous_prime_number(num_buckets_tmp * kHashTableLoadFactor);
+  num_buckets_tmp = num_slots_tmp / kHashTableLoadFactor;
+  DEBUG_ASSERT(num_slots_tmp > 0);
+  DEBUG_ASSERT(num_buckets_tmp > 0);
+
+  // Locate the slot array.
+  slots_ = static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         sizeof(std::atomic<std::size_t>) * num_slots_tmp;
+  available_memory -= sizeof(std::atomic<std::size_t>) * num_slots_tmp;
+
+  // Locate the buckets.
+  buckets_ = aligned_memory_start;
+  // Extra-paranoid: If ValueT has an alignment requirement greater than that
+  // of std::atomic<std::size_t>, we may need to adjust the start of the bucket
+  // array.
+  if (align(kBucketAlignment, bucket_size_, buckets_, available_memory) ==
+      nullptr) {
+    FATAL_ERROR(
+        "StorageBlob used to hold resizable "
+        "SeparateChainingHashTable is too small to meet "
+        "alignment requirements of buckets.");
+  } else if (buckets_ != aligned_memory_start) {
+    DEV_WARNING(
+        "Bucket array start position adjusted to meet alignment "
+        "requirement for SeparateChainingHashTable's value type.");
+    if (num_buckets_tmp * bucket_size_ > available_memory) {
+      --num_buckets_tmp;
+    }
+  }
+
+  // Fill in the header.
+  header_->num_slots = num_slots_tmp;
+  header_->num_buckets = num_buckets_tmp;
+  header_->buckets_allocated.store(0, std::memory_order_relaxed);
+  header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
+  available_memory -= bucket_size_ * (header_->num_buckets);
+
+  // Locate variable-length key storage region, and give it all the remaining
+  // bytes in the blob.
+  key_manager_.setVariableLengthStorageInfo(
+      static_cast<char *>(buckets_) + header_->num_buckets * bucket_size_,
+      available_memory,
+      &(header_->variable_length_bytes_allocated));
+}
+
+PackedPayloadHashTable::~PackedPayloadHashTable() {
+  if (blob_.valid()) {
+    const block_id blob_id = blob_->getID();
+    blob_.release();
+    storage_manager_->deleteBlockOrBlobFile(blob_id);
+  }
+  std::free(init_payload_);
+}
+
+void PackedPayloadHashTable::clear() {
+  const std::size_t used_buckets =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
+  // Destroy existing values, if necessary.
+  destroyPayload();
+
+  // Zero-out slot array.
+  std::memset(
+      slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
+
+  // Zero-out used buckets.
+  std::memset(buckets_, 0x0, used_buckets * bucket_size_);
+
+  header_->buckets_allocated.store(0, std::memory_order_relaxed);
+  header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
+  key_manager_.zeroNextVariableLengthKeyOffset();
+}
+
+void PackedPayloadHashTable::destroyPayload() {
+  const std::size_t num_buckets =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
+  void *bucket_ptr = static_cast<char *>(buckets_) + kValueOffset;
+  for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
+    for (std::size_t handle_id = 0; handle_id < num_handles_; ++handle_id) {
+      void *value_internal_ptr =
+          static_cast<char *>(bucket_ptr) + this->payload_offsets_[handle_id];
+      handles_[handle_id]->destroyPayload(static_cast<std::uint8_t *>(value_internal_ptr));
+    }
+    bucket_ptr = static_cast<char *>(bucket_ptr) + bucket_size_;
+  }
+}
+
+bool PackedPayloadHashTable::upsertValueAccessorCompositeKey(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_attr_ids,
+    const ValueAccessorMultiplexer &accessor_mux) {
+  ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+  ValueAccessor *derived_accessor = accessor_mux.getDerivedAccessor();
+
+  base_accessor->beginIterationVirtual();
+  if (derived_accessor == nullptr) {
+    return upsertValueAccessorCompositeKeyInternal<false>(
+        argument_ids,
+        key_attr_ids,
+        base_accessor,
+        nullptr);
+  } else {
+    DCHECK(derived_accessor->getImplementationType()
+               == ValueAccessor::Implementation::kColumnVectors);
+    derived_accessor->beginIterationVirtual();
+    return upsertValueAccessorCompositeKeyInternal<true>(
+        argument_ids,
+        key_attr_ids,
+        base_accessor,
+        static_cast<ColumnVectorsValueAccessor *>(derived_accessor));
+  }
+}
+
+void PackedPayloadHashTable::resize(const std::size_t extra_buckets,
+                                    const std::size_t extra_variable_storage,
+                                    const std::size_t retry_num) {
+  // A retry should never be necessary with this implementation of HashTable.
+  // Separate chaining ensures that any resized hash table with more buckets
+  // than the original table will be able to hold more entries than the
+  // original.
+  DEBUG_ASSERT(retry_num == 0);
+
+  SpinSharedMutexExclusiveLock<true> write_lock(this->resize_shared_mutex_);
+
+  // Recheck whether the hash table is still full. Note that multiple threads
+  // might wait to rebuild this hash table simultaneously. Only the first one
+  // should do the rebuild.
+  if (!isFull(extra_variable_storage)) {
+    return;
+  }
+
+  // Approximately double the number of buckets and slots.
+  //
+  // TODO(chasseur): It may be worth it to more than double the number of
+  // buckets here so that we can maintain a good, sparse fill factor for a
+  // longer time as more values are inserted. Such behavior should take into
+  // account kHashTableLoadFactor.
+  std::size_t resized_num_slots = get_next_prime_number(
+      (header_->num_buckets + extra_buckets / 2) * kHashTableLoadFactor * 2);
+  std::size_t variable_storage_required =
+      (resized_num_slots / kHashTableLoadFactor) *
+      key_manager_.getEstimatedVariableKeySize();
+  const std::size_t original_variable_storage_used =
+      header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+  // If this resize was triggered by a too-large variable-length key, bump up
+  // the variable-length storage requirement.
+  if ((extra_variable_storage > 0) &&
+      (extra_variable_storage + original_variable_storage_used >
+       key_manager_.getVariableLengthKeyStorageSize())) {
+    variable_storage_required += extra_variable_storage;
+  }
+
+  const std::size_t resized_memory_required =
+      sizeof(Header) + resized_num_slots * sizeof(std::atomic<std::size_t>) +
+      (resized_num_slots / kHashTableLoadFactor) * bucket_size_ +
+      variable_storage_required;
+  const std::size_t resized_storage_slots =
+      this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
+  if (resized_storage_slots == 0) {
+    FATAL_ERROR(
+        "Storage requirement for resized SeparateChainingHashTable "
+        "exceeds maximum allocation size.");
+  }
+
+  // Get a new StorageBlob to hold the resized hash table.
+  const block_id resized_blob_id =
+      this->storage_manager_->createBlob(resized_storage_slots);
+  MutableBlobReference resized_blob =
+      this->storage_manager_->getBlobMutable(resized_blob_id);
+
+  // Locate data structures inside the new StorageBlob.
+  void *aligned_memory_start = resized_blob->getMemoryMutable();
+  std::size_t available_memory = resized_storage_slots * kSlotSizeBytes;
+  if (align(alignof(Header),
+            sizeof(Header),
+            aligned_memory_start,
+            available_memory) == nullptr) {
+    // Should be impossible, as noted in constructor.
+    FATAL_ERROR(
+        "StorageBlob used to hold resized SeparateChainingHashTable "
+        "is too small to meet alignment requirements of "
+        "LinearOpenAddressingHashTable::Header.");
+  } else if (aligned_memory_start != resized_blob->getMemoryMutable()) {
+    // Again, should be impossible.
+    DEV_WARNING("In SeparateChainingHashTable::resize(), StorageBlob "
+                << "memory adjusted by "
+                << (resized_num_slots * kSlotSizeBytes - available_memory)
+                << " bytes to meet alignment requirement for "
+                << "LinearOpenAddressingHashTable::Header.");
+  }
+
+  Header *resized_header = static_cast<Header *>(aligned_memory_start);
+  aligned_memory_start =
+      static_cast<char *>(aligned_memory_start) + sizeof(Header);
+  available_memory -= sizeof(Header);
+
+  // As in constructor, recompute the number of slots and buckets using the
+  // actual available memory.
+  std::size_t resized_num_buckets =
+      (available_memory - extra_variable_storage) /
+      (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) + bucket_size_ +
+       key_manager_.getEstimatedVariableKeySize());
+  resized_num_slots =
+      get_previous_prime_number(resized_num_buckets * kHashTableLoadFactor);
+  resized_num_buckets = resized_num_slots / kHashTableLoadFactor;
+
+  // Locate slot array.
+  std::atomic<std::size_t> *resized_slots =
+      static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         sizeof(std::atomic<std::size_t>) * resized_num_slots;
+  available_memory -= sizeof(std::atomic<std::size_t>) * resized_num_slots;
+
+  // As in constructor, we will be extra paranoid and use align() to locate the
+  // start of the array of buckets, as well.
+  void *resized_buckets = aligned_memory_start;
+  if (align(
+          kBucketAlignment, bucket_size_, resized_buckets, available_memory) ==
+      nullptr) {
+    FATAL_ERROR(
+        "StorageBlob used to hold resized SeparateChainingHashTable "
+        "is too small to meet alignment requirements of buckets.");
+  } else if (resized_buckets != aligned_memory_start) {
+    DEV_WARNING(
+        "Bucket array start position adjusted to meet alignment "
+        "requirement for SeparateChainingHashTable's value type.");
+    if (resized_num_buckets * bucket_size_ + variable_storage_required >
+        available_memory) {
+      --resized_num_buckets;
+    }
+  }
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         resized_num_buckets * bucket_size_;
+  available_memory -= resized_num_buckets * bucket_size_;
+
+  void *resized_variable_length_key_storage = aligned_memory_start;
+  const std::size_t resized_variable_length_key_storage_size = available_memory;
+
+  const std::size_t original_buckets_used =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
+
+  // Initialize the header.
+  resized_header->num_slots = resized_num_slots;
+  resized_header->num_buckets = resized_num_buckets;
+  resized_header->buckets_allocated.store(original_buckets_used,
+                                          std::memory_order_relaxed);
+  resized_header->variable_length_bytes_allocated.store(
+      original_variable_storage_used, std::memory_order_relaxed);
+
+  // Bulk-copy buckets. This is safe because:
+  //     1. The "next" pointers will be adjusted when rebuilding chains below.
+  //     2. The hash codes will stay the same.
+  //     3. For key components:
+  //       a. Inline keys will stay exactly the same.
+  //       b. Offsets into variable-length storage will remain valid, because
+  //          we also do a byte-for-byte copy of variable-length storage below.
+  //       c. Absolute external pointers will still point to the same address.
+  //       d. Relative pointers are not used with resizable hash tables.
+  //     4. If values are not trivially copyable, then we invoke ValueT's copy
+  //        or move constructor with placement new.
+  // NOTE(harshad) - Regarding point 4 above, as this is a specialized
+  // hash table implemented for aggregation, the values are trivially copyable,
+  // therefore we don't need to invoke payload values' copy/move constructors.
+  std::memcpy(resized_buckets, buckets_, original_buckets_used * bucket_size_);
+
+  // Copy over variable-length key components, if any.
+  if (original_variable_storage_used > 0) {
+    DEBUG_ASSERT(original_variable_storage_used ==
+                 key_manager_.getNextVariableLengthKeyOffset());
+    DEBUG_ASSERT(original_variable_storage_used <=
+                 resized_variable_length_key_storage_size);
+    std::memcpy(resized_variable_length_key_storage,
+                key_manager_.getVariableLengthKeyStorage(),
+                original_variable_storage_used);
+  }
+
+  destroyPayload();
+
+  // Make resized structures active.
+  std::swap(this->blob_, resized_blob);
+  header_ = resized_header;
+  slots_ = resized_slots;
+  buckets_ = resized_buckets;
+  key_manager_.setVariableLengthStorageInfo(
+      resized_variable_length_key_storage,
+      resized_variable_length_key_storage_size,
+      &(resized_header->variable_length_bytes_allocated));
+
+  // Drop the old blob.
+  const block_id old_blob_id = resized_blob->getID();
+  resized_blob.release();
+  this->storage_manager_->deleteBlockOrBlobFile(old_blob_id);
+
+  // Rebuild chains.
+  void *current_bucket = buckets_;
+  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used;
+       ++bucket_num) {
+    std::atomic<std::size_t> *next_ptr =
+        static_cast<std::atomic<std::size_t> *>(current_bucket);
+    const std::size_t hash_code = *reinterpret_cast<const std::size_t *>(
+        static_cast<const char *>(current_bucket) +
+        sizeof(std::atomic<std::size_t>));
+
+    const std::size_t slot_number = hash_code % header_->num_slots;
+    std::size_t slot_ptr_value = 0;
+    if (slots_[slot_number].compare_exchange_strong(
+            slot_ptr_value, bucket_num + 1, std::memory_order_relaxed)) {
+      // This bucket is the first in the chain for this block, so reset its
+      // next pointer to 0.
+      next_ptr->store(0, std::memory_order_relaxed);
+    } else {
+      // A chain already exists starting from this slot, so put this bucket at
+      // the head.
+      next_ptr->store(slot_ptr_value, std::memory_order_relaxed);
+      slots_[slot_number].store(bucket_num + 1, std::memory_order_relaxed);
+    }
+    current_bucket = static_cast<char *>(current_bucket) + bucket_size_;
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/PackedPayloadHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.hpp b/storage/PackedPayloadHashTable.hpp
new file mode 100644
index 0000000..f87a1de
--- /dev/null
+++ b/storage/PackedPayloadHashTable.hpp
@@ -0,0 +1,995 @@
+/**
+ * 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_STORAGE_PACKED_PAYLOAD_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_PACKED_PAYLOAD_HASH_TABLE_HPP_
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <limits>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/HashTableKeyManager.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "threading/SpinMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class StorageManager;
+class Type;
+class ValueAccessor;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+/**
+ * @brief Aggregation hash table implementation in which the payload can be just
+ *        a bunch of bytes. This implementation is suitable for aggregation with
+ *        multiple aggregation handles (e.g. SUM, MAX, MIN etc).
+ *
+ * At present the hash table uses separate chaining to resolve collisions, i.e.
+ * Keys/values are stored in a separate region of memory from the base hash
+ * table slot array. Every bucket has a "next" pointer so that entries that
+ * collide (i.e. map to the same base slot) form chains of pointers with each
+ * other.
+ **/
+class PackedPayloadHashTable : public AggregationStateHashTableBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key).
+   * @param num_entries The estimated number of entries this hash table will
+   *        hold.
+   * @param handles The aggregation handles.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold this hash table's contents).
+   **/
+  PackedPayloadHashTable(
+      const std::vector<const Type *> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  ~PackedPayloadHashTable() override;
+
+  /**
+   * @brief Erase all entries in this hash table.
+   *
+   * @warning This method is not guaranteed to be threadsafe.
+   **/
+  void clear();
+
+  void destroyPayload() override;
+
+  /**
+   * @brief Use aggregation handles to update (multiple) aggregation states in
+   *        this hash table, with group-by keys and arguments drawn from the
+   *        given ValueAccessors. New states are first inserted if not already
+   *        present.
+   *
+   * @note This method is threadsafe with regard to other calls to
+   *       upsertCompositeKey() and upsertValueAccessorCompositeKey().
+   *
+   * @param argument_ids The multi-source attribute IDs of each argument
+   *        component to be read from \p accessor_mux.
+   * @param key_ids The multi-source attribute IDs of each group-by key
+   *        component to be read from \p accessor_mux.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors which will be used to access keys. beginIteration()
+   *        should be called on the accessors before calling this method.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert new entries for all the keys in accessor
+   *         (note that some entries may still have been upserted, and
+   *         accessors' iterations will be left on the first tuple which could
+   *         not be inserted).
+   **/
+  bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux) override;
+
+  /**
+   * @return The ID of the StorageBlob used to store this hash table.
+   **/
+  inline block_id getBlobId() const {
+    return blob_->getID();
+  }
+
+  /**
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call).
+   *          Concurrent calls to getSingleCompositeKey(), forEach(), and
+   *          forEachCompositeKey() are safe.
+   *
+   * @return The number of entries in this HashTable.
+   **/
+  inline std::size_t numEntries() const {
+    return header_->buckets_allocated.load(std::memory_order_relaxed);
+  }
+
+  /**
+   * @brief Use aggregation handles to merge the given aggregation states into
+   *        the aggregation states mapped to the given key. New states are first
+   *        inserted if not already present.
+   *
+   * @warning The key must not be null.
+   * @note This method is threadsafe with regard to other calls to
+   *       upsertCompositeKey() and upsertValueAccessorCompositeKey().
+   *
+   * @param key The key.
+   * @param source_state The source aggregation states to be merged into this
+   *        hash table.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert a new entry in this hash table.
+   **/
+  inline bool upsertCompositeKey(const std::vector<TypedValue> &key,
+                                 const std::uint8_t *source_state);
+
+  /**
+   * @brief Apply a functor to an aggregation state mapped to the given key.
+   *        First inserting a new state if one is not already present.
+   *
+   * @warning The key must not be null.
+   * @note This method is threadsafe with regard to other calls to
+   *       upsertCompositeKey() and upsertValueAccessorCompositeKey().
+   *
+   * @param key The key.
+   * @param functor A pointer to a functor, which should provide a call
+   *        operator which takes an aggregation state (of type std::uint8_t *)
+   *        as an argument.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert a new entry in this hash table.
+   **/
+  template <typename FunctorT>
+  inline bool upsertCompositeKey(const std::vector<TypedValue> &key,
+                                 FunctorT *functor,
+                                 const std::size_t index);
+
+  /**
+   * @brief Lookup a composite key against this hash table to find a matching
+   *        entry.
+   *
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param key The key to look up.
+   * @return The value of a matched entry if a matching key is found.
+   *         Otherwise, return NULL.
+   **/
+  inline const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key) const;
+
+  /**
+   * @brief Lookup a composite key against this hash table to find a matching
+   *        entry. Then return the aggregation state component with the
+   *        specified index.
+   *
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param key The key to look up.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return The aggregation state of the specified index if a matching key is
+   *         found. Otherwise, return NULL.
+   **/
+  inline const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key,
+      const std::size_t index) const;
+
+  /**
+   * @brief Apply a functor to each (key, value) pair in this hash table.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each key, value pair in
+   *        this hash table.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEach(FunctorT *functor) const;
+
+  /**
+   * @brief Apply a functor to each (key, aggregation state) pair in this hash
+   *        table, where the aggregation state is retrieved from the value
+   *        that maps to the corresponding key with the specified index.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each (key, aggregation state)
+   *        pair in this hash table.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEach(FunctorT *functor, const int index) const;
+
+  /**
+   * @brief Apply a functor to each key, value pair in this hash table.
+   *        Composite key version.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each key, value pair in
+   *        this hash table.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEachCompositeKey(FunctorT *functor) const;
+
+  /**
+   * @brief Apply a functor to each (key, aggregation state) pair in this hash
+   *        table, where the aggregation state is retrieved from the value
+   *        that maps to the corresponding key with the specified index.
+   *        Composite key version.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each (key, aggregation state)
+   *        pair in this hash table.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEachCompositeKey(FunctorT *functor,
+                                         const std::size_t index) const;
+
+ private:
+  void resize(const std::size_t extra_buckets,
+              const std::size_t extra_variable_storage,
+              const std::size_t retry_num = 0);
+
+  inline std::size_t calculateVariableLengthCompositeKeyCopySize(
+      const std::vector<TypedValue> &key) const {
+    std::size_t total = 0;
+    for (std::vector<TypedValue>::size_type idx = 0; idx < key.size(); ++idx) {
+      if (!(*key_inline_)[idx]) {
+        total += key[idx].getDataSize();
+      }
+    }
+    return total;
+  }
+
+  inline bool getNextEntry(TypedValue *key,
+                           const std::uint8_t **value,
+                           std::size_t *entry_num) const;
+
+  inline bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
+                                       const std::uint8_t **value,
+                                       std::size_t *entry_num) const;
+
+  inline std::uint8_t* upsertCompositeKeyInternal(
+      const std::vector<TypedValue> &key,
+      const std::size_t variable_key_size);
+
+  template <bool use_two_accessors>
+  inline bool upsertValueAccessorCompositeKeyInternal(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      ValueAccessor *base_accessor,
+      ColumnVectorsValueAccessor *derived_accessor);
+
+  // Generate a hash for a composite key by hashing each component of 'key' and
+  // mixing their bits with CombineHashes().
+  inline std::size_t hashCompositeKey(const std::vector<TypedValue> &key) const;
+
+  // Set information about which key components are stored inline. This usually
+  // comes from a HashTableKeyManager, and is set by the constructor of a
+  // subclass of HashTable.
+  inline void setKeyInline(const std::vector<bool> *key_inline) {
+    scalar_key_inline_ = key_inline->front();
+    key_inline_ = key_inline;
+  }
+
+  inline static std::size_t ComputeTotalPayloadSize(
+      const std::vector<AggregationHandle *> &handles) {
+    std::size_t total_payload_size = sizeof(SpinMutex);
+    for (const auto *handle : handles) {
+      total_payload_size += handle->getPayloadSize();
+    }
+    return total_payload_size;
+  }
+
+  // Assign '*key_vector' with the attribute values specified by 'key_ids' at
+  // the current position of 'accessor'. If 'check_for_null_keys' is true, stops
+  // and returns true if any of the values is null, otherwise returns false.
+  template <bool use_two_accessors,
+            bool check_for_null_keys,
+            typename ValueAccessorT>
+  inline static bool GetCompositeKeyFromValueAccessor(
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorT *accessor,
+      const ColumnVectorsValueAccessor *derived_accessor,
+      std::vector<TypedValue> *key_vector) {
+    for (std::size_t key_idx = 0; key_idx < key_ids.size(); ++key_idx) {
+      const MultiSourceAttributeId &key_id = key_ids[key_idx];
+      if (use_two_accessors && key_id.source == ValueAccessorSource::kDerived) {
+        (*key_vector)[key_idx] = derived_accessor->getTypedValue(key_id.attr_id);
+      } else {
+        (*key_vector)[key_idx] = accessor->getTypedValue(key_id.attr_id);
+      }
+      if (check_for_null_keys && (*key_vector)[key_idx].isNull()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  struct Header {
+    std::size_t num_slots;
+    std::size_t num_buckets;
+    alignas(kCacheLineBytes) std::atomic<std::size_t> buckets_allocated;
+    alignas(kCacheLineBytes)
+        std::atomic<std::size_t> variable_length_bytes_allocated;
+  };
+
+  // Type(s) of keys.
+  const std::vector<const Type *> key_types_;
+
+  // Information about whether key components are stored inline or in a
+  // separate variable-length storage region. This is usually determined by a
+  // HashTableKeyManager and set by calling setKeyInline().
+  bool scalar_key_inline_;
+  const std::vector<bool> *key_inline_;
+
+  const std::size_t num_handles_;
+  const std::vector<AggregationHandle *> handles_;
+
+  std::size_t total_payload_size_;
+  std::vector<std::size_t> payload_offsets_;
+  std::uint8_t *init_payload_;
+
+  StorageManager *storage_manager_;
+  MutableBlobReference blob_;
+
+  // Locked in shared mode for most operations, exclusive mode during resize.
+  // Not locked at all for non-resizable HashTables.
+  alignas(kCacheLineBytes) SpinSharedMutex<true> resize_shared_mutex_;
+
+  std::size_t kBucketAlignment;
+
+  // Value's offset in a bucket is the first alignof(ValueT) boundary after the
+  // next pointer and hash code.
+  std::size_t kValueOffset;
+
+  // Round bucket size up to a multiple of kBucketAlignment.
+  inline std::size_t ComputeBucketSize(const std::size_t fixed_key_size) {
+    return (((kValueOffset + this->total_payload_size_ + fixed_key_size - 1) /
+             kBucketAlignment) +
+            1) *
+           kBucketAlignment;
+  }
+
+  // Attempt to find an empty bucket to insert 'hash_code' into, starting after
+  // '*bucket' in the chain (or, if '*bucket' is NULL, starting from the slot
+  // array). Returns true and stores SIZE_T_MAX in '*pending_chain_ptr' if an
+  // empty bucket is found. Returns false if 'allow_duplicate_keys' is false
+  // and a hash collision is found (caller should then check whether there is a
+  // genuine key collision or the hash collision is spurious). Returns false
+  // and sets '*bucket' to NULL if there are no more empty buckets in the hash
+  // table. If 'variable_key_allocation_required' is nonzero, this method will
+  // attempt to allocate storage for a variable-length key BEFORE allocating a
+  // bucket, so that no bucket number below 'header_->num_buckets' is ever
+  // deallocated after being allocated.
+  inline bool locateBucketForInsertion(
+      const std::size_t hash_code,
+      const std::size_t variable_key_allocation_required,
+      void **bucket,
+      std::atomic<std::size_t> **pending_chain_ptr,
+      std::size_t *pending_chain_ptr_finish_value);
+
+  // Write a scalar 'key' and its 'hash_code' into the '*bucket', which was
+  // found by locateBucketForInsertion(). Assumes that storage for a
+  // variable-length key copy (if any) was already allocated by a successful
+  // call to allocateVariableLengthKeyStorage().
+  inline void writeScalarKeyToBucket(
+      const TypedValue &key,
+      const std::size_t hash_code,
+      void *bucket);
+
+  // Write a composite 'key' and its 'hash_code' into the '*bucket', which was
+  // found by locateBucketForInsertion(). Assumes that storage for
+  // variable-length key copies (if any) was already allocated by a successful
+  // call to allocateVariableLengthKeyStorage().
+  inline void writeCompositeKeyToBucket(
+      const std::vector<TypedValue> &key,
+      const std::size_t hash_code,
+      void *bucket);
+
+  // Determine whether it is actually necessary to resize this hash table.
+  // Checks that there is at least one unallocated bucket, and that there is
+  // at least 'extra_variable_storage' bytes of variable-length storage free.
+  inline bool isFull(const std::size_t extra_variable_storage) const;
+
+  // Helper object to manage key storage.
+  HashTableKeyManager<false, true> key_manager_;
+
+  // In-memory structure is as follows:
+  //   - SeparateChainingHashTable::Header
+  //   - Array of slots, interpreted as follows:
+  //       - 0 = Points to nothing (empty)
+  //       - SIZE_T_MAX = Pending (some thread is starting a chain from this
+  //         slot and will overwrite it soon)
+  //       - Anything else = The number of the first bucket in the chain for
+  //         this slot PLUS ONE (i.e. subtract one to get the actual bucket
+  //         number).
+  //   - Array of buckets, each of which is:
+  //       - atomic size_t "next" pointer, interpreted the same as slots above.
+  //       - size_t hash value
+  //       - possibly some unused bytes as needed so that ValueT's alignment
+  //         requirement is met
+  //       - ValueT value slot
+  //       - fixed-length key storage (which may include pointers to external
+  //         memory or offsets of variable length keys stored within this hash
+  //         table)
+  //       - possibly some additional unused bytes so that bucket size is a
+  //         multiple of both alignof(std::atomic<std::size_t>) and
+  //         alignof(ValueT)
+  //   - Variable-length key storage region (referenced by offsets stored in
+  //     fixed-length keys).
+  Header *header_;
+
+  std::atomic<std::size_t> *slots_;
+  void *buckets_;
+  const std::size_t bucket_size_;
+
+  DISALLOW_COPY_AND_ASSIGN(PackedPayloadHashTable);
+};
+
+/** @} */
+
+// ----------------------------------------------------------------------------
+// Implementations of template class methods follow.
+
+class HashTableMerger {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param handle The Aggregation handle being used.
+   * @param destination_hash_table The destination hash table to which other
+   *        hash tables will be merged.
+   **/
+  explicit HashTableMerger(PackedPayloadHashTable *destination_hash_table)
+      : destination_hash_table_(destination_hash_table) {}
+
+  /**
+   * @brief The operator for the functor.
+   *
+   * @param group_by_key The group by key being merged.
+   * @param source_state The aggregation state for the given key in the source
+   *        aggregation hash table.
+   **/
+  inline void operator()(const std::vector<TypedValue> &group_by_key,
+                         const std::uint8_t *source_state) {
+    destination_hash_table_->upsertCompositeKey(group_by_key, source_state);
+  }
+
+ private:
+  PackedPayloadHashTable *destination_hash_table_;
+
+  DISALLOW_COPY_AND_ASSIGN(HashTableMerger);
+};
+
+inline std::size_t PackedPayloadHashTable::hashCompositeKey(
+    const std::vector<TypedValue> &key) const {
+  DEBUG_ASSERT(!key.empty());
+  DEBUG_ASSERT(key.size() == key_types_.size());
+  std::size_t hash = key.front().getHash();
+  for (std::vector<TypedValue>::const_iterator key_it = key.begin() + 1;
+       key_it != key.end();
+       ++key_it) {
+    hash = CombineHashes(hash, key_it->getHash());
+  }
+  return hash;
+}
+
+inline bool PackedPayloadHashTable::getNextEntry(TypedValue *key,
+                                                 const std::uint8_t **value,
+                                                 std::size_t *entry_num) const {
+  if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (*entry_num) * bucket_size_;
+    *key = key_manager_.getKeyComponentTyped(bucket, 0);
+    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
+    ++(*entry_num);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+inline bool PackedPayloadHashTable::getNextEntryCompositeKey(
+    std::vector<TypedValue> *key,
+    const std::uint8_t **value,
+    std::size_t *entry_num) const {
+  if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (*entry_num) * bucket_size_;
+    for (std::vector<const Type *>::size_type key_idx = 0;
+         key_idx < this->key_types_.size();
+         ++key_idx) {
+      key->emplace_back(key_manager_.getKeyComponentTyped(bucket, key_idx));
+    }
+    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
+    ++(*entry_num);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+inline bool PackedPayloadHashTable::locateBucketForInsertion(
+    const std::size_t hash_code,
+    const std::size_t variable_key_allocation_required,
+    void **bucket,
+    std::atomic<std::size_t> **pending_chain_ptr,
+    std::size_t *pending_chain_ptr_finish_value) {
+  if (*bucket == nullptr) {
+    *pending_chain_ptr = &(slots_[hash_code % header_->num_slots]);
+  } else {
+    *pending_chain_ptr = static_cast<std::atomic<std::size_t> *>(*bucket);
+  }
+  for (;;) {
+    std::size_t existing_chain_ptr = 0;
+    if ((*pending_chain_ptr)
+            ->compare_exchange_strong(existing_chain_ptr,
+                                      std::numeric_limits<std::size_t>::max(),
+                                      std::memory_order_acq_rel)) {
+      // Got to the end of the chain. Allocate a new bucket.
+
+      // First, allocate variable-length key storage, if needed (i.e. if this
+      // is an upsert and we didn't allocate up-front).
+      if (!key_manager_.allocateVariableLengthKeyStorage(
+              variable_key_allocation_required)) {
+        // Ran out of variable-length storage.
+        (*pending_chain_ptr)->store(0, std::memory_order_release);
+        *bucket = nullptr;
+        return false;
+      }
+
+      const std::size_t allocated_bucket_num =
+          header_->buckets_allocated.fetch_add(1, std::memory_order_relaxed);
+      if (allocated_bucket_num >= header_->num_buckets) {
+        // Ran out of buckets.
+        header_->buckets_allocated.fetch_sub(1, std::memory_order_relaxed);
+        (*pending_chain_ptr)->store(0, std::memory_order_release);
+        *bucket = nullptr;
+        return false;
+      } else {
+        *bucket =
+            static_cast<char *>(buckets_) + allocated_bucket_num * bucket_size_;
+        *pending_chain_ptr_finish_value = allocated_bucket_num + 1;
+        return true;
+      }
+    }
+    // Spin until the real "next" pointer is available.
+    while (existing_chain_ptr == std::numeric_limits<std::size_t>::max()) {
+      existing_chain_ptr =
+          (*pending_chain_ptr)->load(std::memory_order_acquire);
+    }
+    if (existing_chain_ptr == 0) {
+      // Other thread had to roll back, so try again.
+      continue;
+    }
+    // Chase the next pointer.
+    *bucket =
+        static_cast<char *>(buckets_) + (existing_chain_ptr - 1) * bucket_size_;
+    *pending_chain_ptr = static_cast<std::atomic<std::size_t> *>(*bucket);
+    const std::size_t hash_in_bucket = *reinterpret_cast<const std::size_t *>(
+        static_cast<const char *>(*bucket) +
+        sizeof(std::atomic<std::size_t>));
+    if (hash_in_bucket == hash_code) {
+      return false;
+    }
+  }
+}
+
+inline const std::uint8_t* PackedPayloadHashTable::getSingleCompositeKey(
+    const std::vector<TypedValue> &key) const {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
+    }
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
+  }
+
+  // Reached the end of the chain and didn't find a match.
+  return nullptr;
+}
+
+inline const std::uint8_t* PackedPayloadHashTable::getSingleCompositeKey(
+    const std::vector<TypedValue> &key,
+    const std::size_t index) const {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset) +
+             this->payload_offsets_[index];
+    }
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
+  }
+
+  // Reached the end of the chain and didn't find a match.
+  return nullptr;
+}
+
+inline bool PackedPayloadHashTable::upsertCompositeKey(
+    const std::vector<TypedValue> &key,
+    const std::uint8_t *source_state) {
+  const std::size_t variable_size =
+      calculateVariableLengthCompositeKeyCopySize(key);
+  for (;;) {
+    {
+      SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
+      std::uint8_t *value =
+          upsertCompositeKeyInternal(key, variable_size);
+      if (value != nullptr) {
+        SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
+        for (unsigned int k = 0; k < num_handles_; ++k) {
+          handles_[k]->mergeStates(source_state + payload_offsets_[k],
+                                   value + payload_offsets_[k]);
+        }
+        return true;
+      }
+    }
+    resize(0, variable_size);
+  }
+}
+
+template <typename FunctorT>
+inline bool PackedPayloadHashTable::upsertCompositeKey(
+    const std::vector<TypedValue> &key,
+    FunctorT *functor,
+    const std::size_t index) {
+  const std::size_t variable_size =
+      calculateVariableLengthCompositeKeyCopySize(key);
+  for (;;) {
+    {
+      SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
+      std::uint8_t *value =
+          upsertCompositeKeyInternal(key, variable_size);
+      if (value != nullptr) {
+        (*functor)(value + payload_offsets_[index]);
+        return true;
+      }
+    }
+    resize(0, variable_size);
+  }
+}
+
+
+inline std::uint8_t* PackedPayloadHashTable::upsertCompositeKeyInternal(
+    const std::vector<TypedValue> &key,
+    const std::size_t variable_key_size) {
+  if (variable_key_size > 0) {
+    // Don't allocate yet, since the key may already be present. However, we
+    // do check if either the allocated variable storage space OR the free
+    // space is big enough to hold the key (at least one must be true: either
+    // the key is already present and allocated, or we need to be able to
+    // allocate enough space for it).
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(
+        std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size) &&
+        (allocated_bytes + variable_key_size >
+         key_manager_.getVariableLengthKeyStorageSize())) {
+      return nullptr;
+    }
+  }
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 variable_key_size,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets or variable-key space.
+      return nullptr;
+    } else if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Found an already-existing entry for this key.
+      return reinterpret_cast<std::uint8_t *>(static_cast<char *>(bucket) +
+                                              kValueOffset);
+    }
+  }
+
+  // We are now writing to an empty bucket.
+  // Write the key and hash.
+  writeCompositeKeyToBucket(key, hash_code, bucket);
+
+  std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
+  std::memcpy(value, init_payload_, this->total_payload_size_);
+
+  // Update the previous chaing pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value,
+                           std::memory_order_release);
+
+  // Return the value.
+  return value;
+}
+
+template <bool use_two_accessors>
+inline bool PackedPayloadHashTable::upsertValueAccessorCompositeKeyInternal(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_ids,
+    ValueAccessor *base_accessor,
+    ColumnVectorsValueAccessor *derived_accessor) {
+  std::size_t variable_size;
+  std::vector<TypedValue> key_vector;
+  key_vector.resize(key_ids.size());
+
+  return InvokeOnAnyValueAccessor(
+      base_accessor,
+      [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
+    bool continuing = true;
+    while (continuing) {
+      {
+        continuing = false;
+        SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
+        while (accessor->next()) {
+          if (use_two_accessors) {
+            derived_accessor->next();
+          }
+          if (this->GetCompositeKeyFromValueAccessor<use_two_accessors, true>(
+                  key_ids,
+                  accessor,
+                  derived_accessor,
+                  &key_vector)) {
+            continue;
+          }
+          variable_size = this->calculateVariableLengthCompositeKeyCopySize(key_vector);
+          std::uint8_t *value = this->upsertCompositeKeyInternal(
+              key_vector, variable_size);
+          if (value == nullptr) {
+            continuing = true;
+            break;
+          } else {
+            SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
+            for (unsigned int k = 0; k < num_handles_; ++k) {
+              const auto &ids = argument_ids[k];
+              if (ids.empty()) {
+                handles_[k]->updateStateNullary(value + payload_offsets_[k]);
+              } else {
+                const MultiSourceAttributeId &arg_id = ids.front();
+                if (use_two_accessors && arg_id.source == ValueAccessorSource::kDerived) {
+                  DCHECK_NE(arg_id.attr_id, kInvalidAttributeID);
+                  handles_[k]->updateStateUnary(derived_accessor->getTypedValue(arg_id.attr_id),
+                                                value + payload_offsets_[k]);
+                } else {
+                  handles_[k]->updateStateUnary(accessor->getTypedValue(arg_id.attr_id),
+                                                value + payload_offsets_[k]);
+                }
+              }
+            }
+          }
+        }
+      }
+      if (continuing) {
+        this->resize(0, variable_size);
+        accessor->previous();
+        if (use_two_accessors) {
+          derived_accessor->previous();
+        }
+      }
+    }
+    return true;
+  });
+}
+
+inline void PackedPayloadHashTable::writeScalarKeyToBucket(
+    const TypedValue &key,
+    const std::size_t hash_code,
+    void *bucket) {
+  *reinterpret_cast<std::size_t *>(static_cast<char *>(bucket) +
+                                   sizeof(std::atomic<std::size_t>)) =
+      hash_code;
+  key_manager_.writeKeyComponentToBucket(key, 0, bucket, nullptr);
+}
+
+inline void PackedPayloadHashTable::writeCompositeKeyToBucket(
+    const std::vector<TypedValue> &key,
+    const std::size_t hash_code,
+    void *bucket) {
+  DEBUG_ASSERT(key.size() == this->key_types_.size());
+  *reinterpret_cast<std::size_t *>(static_cast<char *>(bucket) +
+                                   sizeof(std::atomic<std::size_t>)) =
+      hash_code;
+  for (std::size_t idx = 0; idx < this->key_types_.size(); ++idx) {
+    key_manager_.writeKeyComponentToBucket(key[idx], idx, bucket, nullptr);
+  }
+}
+
+inline bool PackedPayloadHashTable::isFull(
+    const std::size_t extra_variable_storage) const {
+  if (header_->buckets_allocated.load(std::memory_order_relaxed) >=
+      header_->num_buckets) {
+    // All buckets are allocated.
+    return true;
+  }
+
+  if (extra_variable_storage > 0) {
+    if (extra_variable_storage +
+            header_->variable_length_bytes_allocated.load(
+                std::memory_order_relaxed) >
+        key_manager_.getVariableLengthKeyStorageSize()) {
+      // Not enough variable-length key storage space.
+      return true;
+    }
+  }
+
+  return false;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadHashTable::forEach(FunctorT *functor) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  TypedValue key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntry(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr);
+  }
+  return entries_visited;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadHashTable::forEach(
+    FunctorT *functor, const int index) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  TypedValue key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntry(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr + payload_offsets_[index]);
+    key.clear();
+  }
+  return entries_visited;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadHashTable::forEachCompositeKey(
+    FunctorT *functor) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  std::vector<TypedValue> key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr);
+    key.clear();
+  }
+  return entries_visited;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadHashTable::forEachCompositeKey(
+    FunctorT *functor,
+    const std::size_t index) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  std::vector<TypedValue> key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr + payload_offsets_[index]);
+    key.clear();
+  }
+  return entries_visited;
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_PACKED_PAYLOAD_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/PartitionedHashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/PartitionedHashTablePool.hpp b/storage/PartitionedHashTablePool.hpp
index 95d1810..0e62511 100644
--- a/storage/PartitionedHashTablePool.hpp
+++ b/storage/PartitionedHashTablePool.hpp
@@ -21,22 +21,19 @@
 #define QUICKSTEP_STORAGE_PARTITIONED_HASH_TABLE_POOL_HPP_
 
 #include <algorithm>
-#include <chrono>
+#include <cstddef>
 #include <memory>
-#include <utility>
 #include <vector>
 
-#include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/HashTableFactory.hpp"
 #include "utility/Macros.hpp"
-#include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
+class AggregationHandle;
 class StorageManager;
 class Type;
 
@@ -54,33 +51,6 @@ class PartitionedHashTablePool {
   /**
    * @brief Constructor.
    *
-   * @param estimated_num_entries The maximum number of entries in a hash table.
-   * @param num_partitions The number of partitions (i.e. number of HashTables)
-   * @param hash_table_impl_type The type of hash table implementation.
-   * @param group_by_types A vector of pointer of types which form the group by
-   *        key.
-   * @param agg_handle The aggregation handle.
-   * @param storage_manager A pointer to the storage manager.
-   **/
-  PartitionedHashTablePool(const std::size_t estimated_num_entries,
-                           const std::size_t num_partitions,
-                           const HashTableImplType hash_table_impl_type,
-                           const std::vector<const Type *> &group_by_types,
-                           AggregationHandle *agg_handle,
-                           StorageManager *storage_manager)
-      : estimated_num_entries_(
-            setHashTableSize(estimated_num_entries, num_partitions)),
-        num_partitions_(num_partitions),
-        hash_table_impl_type_(hash_table_impl_type),
-        group_by_types_(group_by_types),
-        agg_handle_(DCHECK_NOTNULL(agg_handle)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)) {
-    initializeAllHashTables();
-  }
-
-  /**
-   * @brief Constructor.
-   *
    * @note This constructor is relevant for the HashTable specialized for
    *       aggregation.
    *
@@ -89,8 +59,6 @@ class PartitionedHashTablePool {
    * @param hash_table_impl_type The type of hash table implementation.
    * @param group_by_types A vector of pointer of types which form the group by
    *        key.
-   * @param payload_sizes The sizes of the payload elements (i.e.
-   *        AggregationStates).
    * @param handles The aggregation handles.
    * @param storage_manager A pointer to the storage manager.
    **/
@@ -98,7 +66,6 @@ class PartitionedHashTablePool {
                            const std::size_t num_partitions,
                            const HashTableImplType hash_table_impl_type,
                            const std::vector<const Type *> &group_by_types,
-                           const std::vector<std::size_t> &payload_sizes,
                            const std::vector<AggregationHandle *> &handles,
                            StorageManager *storage_manager)
       : estimated_num_entries_(
@@ -106,7 +73,6 @@ class PartitionedHashTablePool {
         num_partitions_(num_partitions),
         hash_table_impl_type_(hash_table_impl_type),
         group_by_types_(group_by_types),
-        payload_sizes_(payload_sizes),
         handles_(handles),
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {
     initializeAllHashTables();
@@ -150,25 +116,17 @@ class PartitionedHashTablePool {
  private:
   void initializeAllHashTables() {
     for (std::size_t part_num = 0; part_num < num_partitions_; ++part_num) {
-      AggregationStateHashTableBase *part_hash_table = createNewHashTableFast();
+      AggregationStateHashTableBase *part_hash_table = createNewHashTable();
       hash_tables_.push_back(
           std::unique_ptr<AggregationStateHashTableBase>(part_hash_table));
     }
   }
 
   AggregationStateHashTableBase* createNewHashTable() {
-    return agg_handle_->createGroupByHashTable(hash_table_impl_type_,
-                                               group_by_types_,
-                                               estimated_num_entries_,
-                                               storage_manager_);
-  }
-
-  AggregationStateHashTableBase* createNewHashTableFast() {
-    return AggregationStateFastHashTableFactory::CreateResizable(
+    return AggregationStateHashTableFactory::CreateResizable(
                 hash_table_impl_type_,
                 group_by_types_,
                 estimated_num_entries_,
-                payload_sizes_,
                 handles_,
                 storage_manager_);
   }
@@ -189,10 +147,6 @@ class PartitionedHashTablePool {
   const HashTableImplType hash_table_impl_type_;
 
   const std::vector<const Type *> group_by_types_;
-
-  std::vector<std::size_t> payload_sizes_;
-
-  AggregationHandle *agg_handle_;
   const std::vector<AggregationHandle *> handles_;
   StorageManager *storage_manager_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index de2d25b..0cc7735 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -19,8 +19,8 @@
 
 #include "storage/StorageBlock.hpp"
 
-#include <climits>
 #include <memory>
+#include <random>
 #include <type_traits>
 #include <unordered_map>
 #include <utility>
@@ -28,7 +28,6 @@
 
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/BasicColumnStoreTupleStorageSubBlock.hpp"
@@ -37,7 +36,6 @@
 #include "storage/CompressedColumnStoreTupleStorageSubBlock.hpp"
 #include "storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp"
 #include "storage/CountedReference.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
 #include "storage/SMAIndexSubBlock.hpp"
@@ -396,166 +394,6 @@ void StorageBlock::selectSimple(const std::vector<attribute_id> &selection,
                                                       accessor.get());
 }
 
-AggregationState* StorageBlock::aggregate(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const std::vector<attribute_id> *arguments_as_attributes,
-    const TupleIdSequence *filter) const {
-#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.
-  if ((arguments_as_attributes != nullptr) && (!arguments_as_attributes->empty())) {
-    DCHECK_EQ(arguments.size(), arguments_as_attributes->size())
-        << "Mismatch between number of arguments and number of attribute_ids";
-    return aggregateHelperValueAccessor(handle, *arguments_as_attributes, filter);
-  }
-  // TODO(shoban): We may want to optimize for ScalarLiteral here.
-#endif
-
-  // Call aggregateHelperColumnVector() to materialize each argument as a
-  // ColumnVector, then aggregate over those.
-  return aggregateHelperColumnVector(handle, arguments, filter);
-}
-
-void StorageBlock::aggregateGroupBy(
-    const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const TupleIdSequence *filter,
-    AggregationStateHashTableBase *hash_table,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK_GT(group_by.size(), 0u)
-      << "Called aggregateGroupBy() with zero GROUP BY expressions";
-
-  SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                    indices_,
-                                    indices_consistent_);
-
-  // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-  std::vector<attribute_id> argument_ids;
-
-  // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
-  std::vector<attribute_id> key_ids;
-
-  // An intermediate ValueAccessor that stores the materialized 'arguments' for
-  // this aggregate, as well as the GROUP BY expression values.
-  ColumnVectorsValueAccessor temp_result;
-  {
-    std::unique_ptr<ValueAccessor> accessor(tuple_store_->createValueAccessor(filter));
-    attribute_id attr_id = 0;
-
-    // First, put GROUP BY keys into 'temp_result'.
-    if (reuse_group_by_vectors->empty()) {
-      // Compute GROUP BY values from group_by Scalars, and store them in
-      // reuse_group_by_vectors for reuse by other aggregates on this same
-      // block.
-      reuse_group_by_vectors->reserve(group_by.size());
-      for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
-        reuse_group_by_vectors->emplace_back(
-            group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
-        temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
-        key_ids.push_back(attr_id++);
-      }
-    } else {
-      // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
-      DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
-          << "Wrong number of reuse_group_by_vectors";
-      for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
-        temp_result.addColumn(reuse_cv.get(), false);
-        key_ids.push_back(attr_id++);
-      }
-    }
-
-    // Compute argument vectors and add them to 'temp_result'.
-    for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
-        for (const std::unique_ptr<const Scalar> &args : argument) {
-          temp_result.addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
-          argument_ids.push_back(attr_id++);
-        }
-        if (argument.empty()) {
-          argument_ids.push_back(kInvalidAttributeID);
-        }
-     }
-  }
-
-  hash_table->upsertValueAccessorCompositeKeyFast(argument_ids,
-                                                  &temp_result,
-                                                  key_ids,
-                                                  true);
-}
-
-
-void StorageBlock::aggregateDistinct(
-    const AggregationHandle &handle,
-    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 TupleIdSequence *filter,
-    AggregationStateHashTableBase *distinctify_hash_table,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK_GT(arguments.size(), 0u)
-      << "Called aggregateDistinct() with zero argument expressions";
-  DCHECK((group_by.size() == 0 || reuse_group_by_vectors != nullptr));
-
-  std::vector<attribute_id> key_ids;
-
-  // An intermediate ValueAccessor that stores the materialized 'arguments' for
-  // this aggregate, as well as the GROUP BY expression values.
-  ColumnVectorsValueAccessor temp_result;
-  {
-    std::unique_ptr<ValueAccessor> accessor(tuple_store_->createValueAccessor(filter));
-
-#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.
-    if ((arguments_as_attributes != nullptr) && (!arguments_as_attributes->empty())) {
-      DCHECK_EQ(arguments.size(), arguments_as_attributes->size())
-          << "Mismatch between number of arguments and number of attribute_ids";
-      DCHECK_EQ(group_by.size(), 0u);
-      handle.insertValueAccessorIntoDistinctifyHashTable(
-          accessor.get(), *arguments_as_attributes, distinctify_hash_table);
-      return;
-    }
-#endif
-
-    SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                      indices_,
-                                      indices_consistent_);
-    attribute_id attr_id = 0;
-
-    if (!group_by.empty()) {
-      // Put GROUP BY keys into 'temp_result'.
-      if (reuse_group_by_vectors->empty()) {
-        // Compute GROUP BY values from group_by Scalars, and store them in
-        // reuse_group_by_vectors for reuse by other aggregates on this same
-        // block.
-        reuse_group_by_vectors->reserve(group_by.size());
-        for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
-          reuse_group_by_vectors->emplace_back(
-              group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
-          temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
-          key_ids.push_back(attr_id++);
-        }
-      } else {
-        // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
-        DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
-            << "Wrong number of reuse_group_by_vectors";
-        for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
-          temp_result.addColumn(reuse_cv.get(), false);
-          key_ids.push_back(attr_id++);
-        }
-      }
-    }
-    // Compute argument vectors and add them to 'temp_result'.
-    for (const std::unique_ptr<const Scalar> &argument : arguments) {
-      temp_result.addColumn(argument->getAllValues(accessor.get(), &sub_blocks_ref));
-      key_ids.push_back(attr_id++);
-    }
-  }
-
-  handle.insertValueAccessorIntoDistinctifyHashTable(
-      &temp_result, key_ids, distinctify_hash_table);
-}
-
 // TODO(chasseur): Vectorization for updates.
 StorageBlock::UpdateResult StorageBlock::update(
     const unordered_map<attribute_id, unique_ptr<const Scalar>> &assignments,
@@ -1262,61 +1100,6 @@ std::unordered_map<attribute_id, TypedValue>* StorageBlock::generateUpdatedValue
   return update_map;
 }
 
-AggregationState* StorageBlock::aggregateHelperColumnVector(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const TupleIdSequence *matches) const {
-  if (arguments.empty()) {
-    // Special case. This is a nullary aggregate (i.e. COUNT(*)).
-    return handle.accumulateNullary(matches == nullptr ? tuple_store_->numTuples()
-                                                       : matches->size());
-  } else {
-    // Set up a ValueAccessor that will be used when materializing argument
-    // values below (possibly filtered based on the '*matches' to a filter
-    // predicate).
-    std::unique_ptr<ValueAccessor> accessor;
-    if (matches == nullptr) {
-      accessor.reset(tuple_store_->createValueAccessor());
-    } else {
-      accessor.reset(tuple_store_->createValueAccessor(matches));
-    }
-
-    SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                      indices_,
-                                      indices_consistent_);
-
-    // Materialize each argument's values for this block as a ColumnVector.
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    for (const std::unique_ptr<const Scalar> &argument : arguments) {
-      column_vectors.emplace_back(argument->getAllValues(accessor.get(), &sub_blocks_ref));
-    }
-
-    // Have the AggregationHandle actually do the aggregation.
-    return handle.accumulateColumnVectors(column_vectors);
-  }
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* StorageBlock::aggregateHelperValueAccessor(
-    const AggregationHandle &handle,
-    const std::vector<attribute_id> &argument_ids,
-    const TupleIdSequence *matches) const {
-  // Set up a ValueAccessor to aggregate over (possibly filtered based on the
-  // '*matches' to a filter predicate).
-  std::unique_ptr<ValueAccessor> accessor;
-  if (matches == nullptr) {
-    accessor.reset(tuple_store_->createValueAccessor());
-  } else {
-    accessor.reset(tuple_store_->createValueAccessor(matches));
-  }
-
-  // Have the AggregationHandle actually do the aggregation.
-  return handle.accumulateValueAccessor(
-      accessor.get(),
-      argument_ids);
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
 void StorageBlock::updateHeader() {
   DEBUG_ASSERT(*static_cast<const int*>(block_memory_) == block_header_.ByteSize());
 
@@ -1346,59 +1129,4 @@ const std::size_t StorageBlock::getNumTuples() const {
   return tuple_store_->numTuples();
 }
 
-void StorageBlock::aggregateGroupByPartitioned(
-    const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const TupleIdSequence *filter,
-    const std::size_t num_partitions,
-    ColumnVectorsValueAccessor *temp_result,
-    std::vector<attribute_id> *argument_ids,
-    std::vector<attribute_id> *key_ids,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK(!group_by.empty())
-      << "Called aggregateGroupByPartitioned() with zero GROUP BY expressions";
-
-  SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                    indices_,
-                                    indices_consistent_);
-
-  std::unique_ptr<ValueAccessor> accessor(
-      tuple_store_->createValueAccessor(filter));
-
-  attribute_id attr_id = 0;
-
-  // First, put GROUP BY keys into 'temp_result'.
-  if (reuse_group_by_vectors->empty()) {
-    // Compute GROUP BY values from group_by Scalars, and store them in
-    // reuse_group_by_vectors for reuse by other aggregates on this same
-    // block.
-    reuse_group_by_vectors->reserve(group_by.size());
-    for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
-      reuse_group_by_vectors->emplace_back(
-          group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
-      temp_result->addColumn(reuse_group_by_vectors->back().get(), false);
-      key_ids->push_back(attr_id++);
-    }
-  } else {
-    // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
-    DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
-        << "Wrong number of reuse_group_by_vectors";
-    for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
-      temp_result->addColumn(reuse_cv.get(), false);
-      key_ids->push_back(attr_id++);
-    }
-  }
-
-  // Compute argument vectors and add them to 'temp_result'.
-  for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
-    for (const std::unique_ptr<const Scalar> &args : argument) {
-      temp_result->addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
-      argument_ids->push_back(attr_id++);
-    }
-    if (argument.empty()) {
-      argument_ids->push_back(kInvalidAttributeID);
-    }
-  }
-}
-
 }  // namespace quickstep


[47/50] [abbrv] incubator-quickstep git commit: Removed an incorrect TODO.

Posted by ji...@apache.org.
Removed an incorrect TODO.


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

Branch: refs/heads/LIP-time-decomposition
Commit: a8e561881389eae8899360af141f5b50c66b3692
Parents: 960eb35
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 9 21:31:31 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 21:31:47 2017 -0800

----------------------------------------------------------------------
 cli/QuickstepCli.cpp | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a8e56188/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 02ec4ec..eddee8c 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -266,7 +266,6 @@ int main(int argc, char* argv[]) {
     worker_client_ids.push_back(workers.back().getBusClientID());
   }
 
-  // TODO(zuyu): Move WorkerDirectory within Shiftboss once the latter is added.
   WorkerDirectory worker_directory(worker_cpu_affinities.size(),
                                    worker_client_ids,
                                    worker_numa_nodes);


[35/50] [abbrv] incubator-quickstep git commit: Style fixes for TextScanOperator.

Posted by ji...@apache.org.
Style fixes for TextScanOperator.


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

Branch: refs/heads/LIP-time-decomposition
Commit: a9fe07d5b0885d08b4aac328aa7deea81d94bda7
Parents: aa7f6fe
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 01:57:23 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 01:57:23 2017 -0800

----------------------------------------------------------------------
 relational_operators/TextScanOperator.cpp | 74 +++++++++++++-------------
 relational_operators/TextScanOperator.hpp |  6 +--
 2 files changed, 39 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a9fe07d5/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 6650319..0a83a85 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -110,8 +110,6 @@ bool TextScanOperator::getAllWorkOrders(
 
   if (blocking_dependencies_met_ && !work_generated_) {
     for (const std::string &file : files) {
-      // Use standard C libary to retrieve the file size.
-
 #ifdef QUICKSTEP_HAVE_UNISTD
       // Check file permissions before trying to open it.
       const int access_result = access(file.c_str(), R_OK);
@@ -255,11 +253,11 @@ void TextScanWorkOrder::execute() {
     } else {
       vector_tuple_returned = parseRow(&row_ptr, relation, &is_faulty);
       if (is_faulty) {
-          // Skip faulty rows
-          LOG(INFO) << "Faulty row found. Hence switching to next row.";
+        // Skip faulty rows
+        LOG(INFO) << "Faulty row found. Hence switching to next row.";
       } else {
-            // Convert vector returned to tuple only when a valid row is encountered.
-            tuples.emplace_back(Tuple(std::move(vector_tuple_returned)));
+        // Convert vector returned to tuple only when a valid row is encountered.
+        tuples.emplace_back(Tuple(std::move(vector_tuple_returned)));
       }
     }
   }
@@ -297,11 +295,11 @@ void TextScanWorkOrder::execute() {
 
     vector_tuple_returned = parseRow(&row_ptr, relation, &is_faulty);
     if (is_faulty) {
-        // Skip the faulty row.
-        LOG(INFO) << "Faulty row found. Hence switching to next row.";
+      // Skip the faulty row.
+      LOG(INFO) << "Faulty row found. Hence switching to next row.";
     } else {
-        // Convert vector returned to tuple only when a valid row is encountered.
-        tuples.emplace_back(Tuple(std::move(vector_tuple_returned)));
+      // Convert vector returned to tuple only when a valid row is encountered.
+      tuples.emplace_back(Tuple(std::move(vector_tuple_returned)));
     }
   }
 
@@ -346,11 +344,11 @@ std::vector<TypedValue> TextScanWorkOrder::parseRow(const char **row_ptr,
   std::string value_str;
   for (const auto &attr : relation) {
     if (has_reached_end_of_line) {
-        // Do not abort if one of the row is faulty.
-        // Set is_faulty to true and SKIP the current row.
-        *is_faulty = true;
-        LOG(INFO) << "Row has too few fields.";
-        return attribute_values;
+      // Do not abort if one of the row is faulty.
+      // Set is_faulty to true and SKIP the current row.
+      *is_faulty = true;
+      LOG(INFO) << "Row has too few fields.";
+      return attribute_values;
     }
 
     value_str.clear();
@@ -363,46 +361,46 @@ std::vector<TypedValue> TextScanWorkOrder::parseRow(const char **row_ptr,
     if (is_null_literal) {
       // NULL literal.
       if (!attr.getType().isNullable()) {
-          *is_faulty = true;
-          LOG(INFO) << "NULL literal '\\N' was specified for a column with a "
-                     "non-nullable Type.";
-          skipFaultyRow(row_ptr);
-          return attribute_values;
+        *is_faulty = true;
+        LOG(INFO) << "NULL literal '\\N' was specified for a column with a "
+                   "non-nullable Type.";
+        skipFaultyRow(row_ptr);
+        return attribute_values;
       }
       attribute_values.emplace_back(attr.getType().makeNullValue());
     } else {
       attribute_values.emplace_back();
       if (!attr.getType().parseValueFromString(value_str, &(attribute_values.back()))) {
-          // Do not abort if one of the row is faulty.
-          *is_faulty = true;
-          LOG(INFO) << "Failed to parse value.";
-          skipFaultyRow(row_ptr);
-          return attribute_values;
+        // Do not abort if one of the row is faulty.
+        *is_faulty = true;
+        LOG(INFO) << "Failed to parse value.";
+        skipFaultyRow(row_ptr);
+        return attribute_values;
       }
     }
   }
 
   if (!has_reached_end_of_line) {
-      // Do not abort if one of the row is faulty.
-      // Set is_faulty to true and SKIP the current row.
-      *is_faulty = true;
-      LOG(INFO) << "Row has too many fields.";
-      skipFaultyRow(row_ptr);
+    // Do not abort if one of the row is faulty.
+    // Set is_faulty to true and SKIP the current row.
+    *is_faulty = true;
+    LOG(INFO) << "Row has too many fields.";
+    skipFaultyRow(row_ptr);
   }
 
   return attribute_values;
 }
 
 void TextScanWorkOrder::skipFaultyRow(const char **field_ptr) const {
-    const char *cur_ptr = *field_ptr;
-    // Move row pointer to the end of faulty row.
-    for (;; ++cur_ptr) {
-        const char c = *cur_ptr;
-        if (c == '\n') {
-            break;
-        }
+  const char *cur_ptr = *field_ptr;
+  // Move row pointer to the end of faulty row.
+  for (;; ++cur_ptr) {
+    const char c = *cur_ptr;
+    if (c == '\n') {
+        break;
     }
-    *field_ptr = cur_ptr + 1;
+  }
+  *field_ptr = cur_ptr + 1;
 }
 
 void TextScanWorkOrder::extractFieldString(const char **field_ptr,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a9fe07d5/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 65863b3..eada190 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -264,9 +264,9 @@ class TextScanWorkOrder : public WorkOrder {
    * @param is_faulty OUTPUT parameter. Set to true if the row is faulty,
    * @return The tuple parsed from the char stream.
    */
-std::vector<TypedValue> parseRow(const char **row_ptr,
-                 const CatalogRelationSchema &relation,
-                 bool *is_faulty) const;
+  std::vector<TypedValue> parseRow(const char **row_ptr,
+                                   const CatalogRelationSchema &relation,
+                                   bool *is_faulty) const;
 
   /**
    * @brief Parse up to three octal digits (0-7) starting at \p *literal_ptr as


[31/50] [abbrv] incubator-quickstep git commit: Avoid crash due to an error in deleting a file in HDFS.

Posted by ji...@apache.org.
Avoid crash due to an error in deleting a file in HDFS.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 9a95c23bdfebe877288f325b3844b3a01c77c08a
Parents: f46ae15
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Feb 7 23:23:50 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Feb 7 23:23:50 2017 -0800

----------------------------------------------------------------------
 storage/FileManagerHdfs.cpp | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9a95c23b/storage/FileManagerHdfs.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.cpp b/storage/FileManagerHdfs.cpp
index 937750a..3a5167c 100644
--- a/storage/FileManagerHdfs.cpp
+++ b/storage/FileManagerHdfs.cpp
@@ -117,12 +117,17 @@ size_t FileManagerHdfs::numSlots(const block_id block) const {
 bool FileManagerHdfs::deleteBlockOrBlob(const block_id block) {
   const string filename(blockFilename(block));
 
-  if ((hdfsDelete(hdfs_, filename.c_str(), 0) == 0) || (errno == ENOENT)) {
-    return true;
-  } else {
-    LOG(ERROR) << "Failed to delete file " << filename << " with error: " << strerror(errno);
-    return false;
+  if (hdfsDelete(hdfs_, filename.c_str(), 0)) {
+    switch (errno) {
+      case EINPROGRESS:
+      case ENOENT:
+        break;
+      default:
+        LOG(ERROR) << "Failed to delete file " << filename << " with error: " << strerror(errno);
+    }
   }
+
+  return true;
 }
 
 bool FileManagerHdfs::readBlockOrBlob(const block_id block,


[12/50] [abbrv] incubator-quickstep git commit: Fixed the linking issue for the distributed cli.

Posted by ji...@apache.org.
Fixed the linking issue for the distributed cli.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 7727e7735630a064c1feff9985143463b61002d6
Parents: 6ec9e9d
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 2 19:13:09 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 2 19:13:09 2017 -0800

----------------------------------------------------------------------
 CMakeLists.txt                 | 23 +++++++++++++++++++----
 cli/distributed/CMakeLists.txt | 20 --------------------
 2 files changed, 19 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7727e773/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index ccb23a3..85210ef 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -808,17 +808,32 @@ target_link_libraries(quickstep_cli_shell ${LIBS})
 
 if (ENABLE_DISTRIBUTED)
   # Build the quickstep_distributed_cli_shell executable.
-  add_executable (quickstep_distributed_cli_shell cli/distributed/QuickstepDistributedCli.cpp)
+  add_executable (quickstep_distributed_cli_shell
+                  cli/distributed/Cli.hpp
+                  cli/distributed/Cli.cpp
+                  cli/distributed/QuickstepDistributedCli.cpp)
   # Link against direct deps (will transitively pull in everything needed).
-  # NOTE(zuyu): Link quickstep_cli_LineReader on behalf of quickstep_cli_distributed_Cli,
-  # as a workaround for bypassing conditionally built target checks in validate_cmakelists.py.
   target_link_libraries(quickstep_distributed_cli_shell
                         glog
+                        quickstep_catalog_CatalogRelation
+                        quickstep_cli_Flags
                         quickstep_cli_LineReader
-                        quickstep_cli_distributed_Cli
+                        quickstep_cli_PrintToScreen
                         quickstep_cli_distributed_Conductor
                         quickstep_cli_distributed_Executor
+                        quickstep_cli_distributed_Role
+                        quickstep_parser_ParseStatement
+                        quickstep_parser_SqlParserWrapper
+                        quickstep_queryexecution_BlockLocatorUtil
+                        quickstep_queryexecution_QueryExecutionMessages_proto
+                        quickstep_queryexecution_QueryExecutionTypedefs
+                        quickstep_queryexecution_QueryExecutionUtil
+                        quickstep_storage_DataExchangerAsync
+                        quickstep_storage_StorageBlockInfo
+                        quickstep_storage_StorageManager
+                        quickstep_utility_Macros
                         quickstep_utility_StringUtil
+                        tmb
                         ${GFLAGS_LIB_NAME}
                         ${GRPCPLUSPLUS_LIBRARIES})
 endif(ENABLE_DISTRIBUTED)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7727e773/cli/distributed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/distributed/CMakeLists.txt b/cli/distributed/CMakeLists.txt
index a00ffda..b46082f 100644
--- a/cli/distributed/CMakeLists.txt
+++ b/cli/distributed/CMakeLists.txt
@@ -18,30 +18,11 @@
 set_gflags_lib_name ()
 
 # Declare micro-libs and link dependencies:
-add_library(quickstep_cli_distributed_Cli Cli.cpp Cli.hpp)
 add_library(quickstep_cli_distributed_Conductor Conductor.cpp Conductor.hpp)
 add_library(quickstep_cli_distributed_Executor Executor.cpp Executor.hpp)
 add_library(quickstep_cli_distributed_Role Role.cpp Role.hpp)
 
 # Link dependencies:
-target_link_libraries(quickstep_cli_distributed_Cli
-                      glog
-                      quickstep_catalog_CatalogRelation
-                      quickstep_cli_Flags
-                      quickstep_cli_PrintToScreen
-                      quickstep_cli_distributed_Role
-                      quickstep_parser_ParseStatement
-                      quickstep_parser_SqlParserWrapper
-                      quickstep_queryexecution_BlockLocatorUtil
-                      quickstep_queryexecution_QueryExecutionMessages_proto
-                      quickstep_queryexecution_QueryExecutionTypedefs
-                      quickstep_queryexecution_QueryExecutionUtil
-                      quickstep_storage_DataExchangerAsync
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageManager
-                      quickstep_utility_Macros
-                      quickstep_utility_StringUtil
-                      tmb)
 target_link_libraries(quickstep_cli_distributed_Conductor
                       glog
                       quickstep_cli_DefaultsConfigurator
@@ -83,7 +64,6 @@ target_link_libraries(quickstep_cli_distributed_Role
 add_library(quickstep_cli_distributed ../../empty_src.cpp CliDistributedModule.hpp)
 
 target_link_libraries(quickstep_cli_distributed
-                      quickstep_cli_distributed_Cli
                       quickstep_cli_distributed_Conductor
                       quickstep_cli_distributed_Executor
                       quickstep_cli_distributed_Role)


[07/50] [abbrv] incubator-quickstep git commit: Enabled some checks for the distributed version in the release build.

Posted by ji...@apache.org.
Enabled some checks for the distributed version in the release build.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 5ffdaaf9f9d42cb25ffcbaf59cfafc049dcaca27
Parents: dff4a14
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 31 14:45:27 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 31 14:45:27 2017 -0800

----------------------------------------------------------------------
 cli/distributed/Cli.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ffdaaf9/cli/distributed/Cli.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Cli.cpp b/cli/distributed/Cli.cpp
index 01f824d..5af70e6 100644
--- a/cli/distributed/Cli.cpp
+++ b/cli/distributed/Cli.cpp
@@ -95,13 +95,13 @@ void Cli::init() {
   tmb::MessageStyle style;
 
   TaggedMessage cli_reg_message(kDistributedCliRegistrationMessage);
-  DCHECK(tmb::MessageBus::SendStatus::kOK ==
+  CHECK(tmb::MessageBus::SendStatus::kOK ==
       bus_.Send(cli_id_, all_addresses, style, move(cli_reg_message)));
 
   // Wait for Conductor to response.
   const AnnotatedMessage cli_reg_response_message(bus_.Receive(cli_id_, 0, true));
-  DCHECK_EQ(kDistributedCliRegistrationResponseMessage,
-            cli_reg_response_message.tagged_message.message_type());
+  CHECK_EQ(kDistributedCliRegistrationResponseMessage,
+           cli_reg_response_message.tagged_message.message_type());
   conductor_client_id_ = cli_reg_response_message.sender;
 
   DLOG(INFO) << "DistributedCli received typed '" << kDistributedCliRegistrationResponseMessage


[48/50] [abbrv] incubator-quickstep git commit: Minor improved the scheduling algorithm in the distributed version.

Posted by ji...@apache.org.
Minor improved the scheduling algorithm in 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/ab46d78d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/ab46d78d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/ab46d78d

Branch: refs/heads/LIP-time-decomposition
Commit: ab46d78de70b7490dc1ce25aad51a5165848ae95
Parents: fead6f8
Author: Zuyu Zhang <zu...@apache.org>
Authored: Fri Feb 10 20:37:23 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Feb 10 20:37:23 2017 -0800

----------------------------------------------------------------------
 query_execution/ForemanDistributed.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab46d78d/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index 8c20e65..389d6ab 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -378,7 +378,8 @@ void ForemanDistributed::dispatchWorkOrderMessages(const vector<unique_ptr<S::Wo
     sendWorkOrderMessage(shiftboss_index_for_particular_work_order_type, proto);
     shiftboss_directory_.incrementNumQueuedWorkOrders(shiftboss_index_for_particular_work_order_type);
 
-    if (shiftboss_index == shiftboss_index_for_particular_work_order_type) {
+    if (shiftboss_index == shiftboss_index_for_particular_work_order_type &&
+        shiftboss_directory_.hasReachedCapacity(shiftboss_index)) {
       shiftboss_index = (shiftboss_index + 1) % shiftboss_directory_.size();
     } else {
       // NOTE(zuyu): This is not the exact round-robin scheduling, as in this case,


[45/50] [abbrv] incubator-quickstep git commit: Fixed the lint issue in QueryManagerDistributed.

Posted by ji...@apache.org.
Fixed the lint issue in QueryManagerDistributed.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 960eb3570b4a5d3ddac300c8c083686b35eb46a0
Parents: 167ee87
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 9 21:14:46 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 21:14:46 2017 -0800

----------------------------------------------------------------------
 query_execution/QueryManagerDistributed.cpp | 2 +-
 query_execution/QueryManagerDistributed.hpp | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/960eb357/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 174c490..6c6f895 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -53,7 +53,7 @@ namespace quickstep {
 QueryManagerDistributed::QueryManagerDistributed(QueryHandle *query_handle,
                                                  const tmb::client_id foreman_client_id,
                                                  const std::size_t num_shiftbosses,
-                                                 tmb::Address &&shiftboss_addresses,
+                                                 tmb::Address &&shiftboss_addresses,  // NOLINT(whitespace/operators)
                                                  tmb::MessageBus *bus)
     : QueryManagerBase(query_handle),
       foreman_client_id_(foreman_client_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/960eb357/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 14401a6..3ebc434 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -65,7 +65,7 @@ class QueryManagerDistributed final : public QueryManagerBase {
   QueryManagerDistributed(QueryHandle *query_handle,
                           const tmb::client_id foreman_client_id,
                           const std::size_t num_shiftbosses,
-                          tmb::Address &&shiftboss_addresses,
+                          tmb::Address &&shiftboss_addresses,  // NOLINT(whitespace/operators)
                           tmb::MessageBus *bus);
 
   ~QueryManagerDistributed() override {}


[27/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
index 6e6d188..798ba76 100644
--- a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
@@ -32,8 +32,9 @@
 #include "expressions/aggregation/AggregationHandleMin.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
 #include "types/DatetimeLit.hpp"
@@ -50,10 +51,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
@@ -222,34 +220,6 @@ class AggregationHandleMinTest : public ::testing::Test {
   }
 
   template <typename GenericType>
-  void checkAggregationMinGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
-            .isNull());
-
-    typename GenericType::cpptype min;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType>(type, &min));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_min_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMinValue<typename GenericType::cpptype>(
-        min, *aggregation_handle_min_, *cv_state);
-
-    aggregation_handle_min_->mergeStates(*cv_state,
-                                         aggregation_handle_min_state_.get());
-    CheckMinValue<typename GenericType::cpptype>(
-        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType>
   void checkAggregationMinGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
@@ -265,7 +235,8 @@ class AggregationHandleMinTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_min_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -277,7 +248,6 @@ class AggregationHandleMinTest : public ::testing::Test {
     CheckMinValue<typename GenericType::cpptype>(
         min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   template <typename StringType>
   void checkAggregationMinString() {
@@ -382,33 +352,6 @@ class AggregationHandleMinTest : public ::testing::Test {
   }
 
   template <typename StringType, typename ColumnVectorType>
-  void checkAggregationMinStringColumnVector() {
-    const StringType &type = StringType::Instance(10, true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
-            .isNull());
-
-    std::string min;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorString<ColumnVectorType>(type, &min));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_min_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMinString(min, *aggregation_handle_min_, *cv_state);
-
-    aggregation_handle_min_->mergeStates(*cv_state,
-                                         aggregation_handle_min_state_.get());
-    CheckMinString(
-        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename StringType, typename ColumnVectorType>
   void checkAggregationMinStringValueAccessor() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
@@ -423,7 +366,8 @@ class AggregationHandleMinTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_min_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -434,7 +378,6 @@ class AggregationHandleMinTest : public ::testing::Test {
     CheckMinString(
         min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_min_;
   std::unique_ptr<AggregationState> aggregation_handle_min_state_;
@@ -511,43 +454,6 @@ TEST_F(AggregationHandleMinTest, VarCharTypeTest) {
   checkAggregationMinString<VarCharType>();
 }
 
-TEST_F(AggregationHandleMinTest, IntTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<IntType>();
-}
-
-TEST_F(AggregationHandleMinTest, LongTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<LongType>();
-}
-
-TEST_F(AggregationHandleMinTest, FloatTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<FloatType>();
-}
-
-TEST_F(AggregationHandleMinTest, DoubleTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<DoubleType>();
-}
-
-TEST_F(AggregationHandleMinTest, DatetimeTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<DatetimeType>();
-}
-
-TEST_F(AggregationHandleMinTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleMinTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<YearMonthIntervalType>();
-}
-
-TEST_F(AggregationHandleMinTest, CharTypeColumnVectorTest) {
-  checkAggregationMinStringColumnVector<CharType, NativeColumnVector>();
-}
-
-TEST_F(AggregationHandleMinTest, VarCharTypeColumnVectorTest) {
-  checkAggregationMinStringColumnVector<VarCharType, IndirectColumnVector>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleMinTest, IntTypeValueAccessorTest) {
   checkAggregationMinGenericValueAccessor<IntType>();
 }
@@ -583,7 +489,6 @@ TEST_F(AggregationHandleMinTest, CharTypeValueAccessorTest) {
 TEST_F(AggregationHandleMinTest, VarCharTypeValueAccessorTest) {
   checkAggregationMinStringValueAccessor<VarCharType, IndirectColumnVector>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleMinDeathTest, WrongTypeTest) {
@@ -685,28 +590,25 @@ TEST_F(AggregationHandleMinTest, GroupByTableMergeTest) {
   initializeHandle(int_non_null_type);
   storage_manager_.reset(new StorageManager("./test_min_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_min_.get()->getPayloadSize()},
           {aggregation_handle_min_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_min_.get()->getPayloadSize()},
           {aggregation_handle_min_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleMin *aggregation_handle_min_derived =
       static_cast<AggregationHandleMin *>(aggregation_handle_min_.get());
@@ -776,47 +678,47 @@ TEST_F(AggregationHandleMinTest, GroupByTableMergeTest) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckMinValue<int>(
       common_key_source_min_val.getLiteral<int>(),
-      aggregation_handle_min_derived->finalizeHashTableEntryFast(
+      aggregation_handle_min_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
-  CheckMinValue<int>(exclusive_key_destination_min_val.getLiteral<int>(),
-                     aggregation_handle_min_derived->finalizeHashTableEntryFast(
-                         destination_hash_table_derived->getSingleCompositeKey(
-                             exclusive_destination_key) +
-                         1));
-  CheckMinValue<int>(exclusive_key_source_min_val.getLiteral<int>(),
-                     aggregation_handle_min_derived->finalizeHashTableEntryFast(
-                         source_hash_table_derived->getSingleCompositeKey(
-                             exclusive_source_key) +
-                         1));
+  CheckMinValue<int>(
+      exclusive_key_destination_min_val.getLiteral<int>(),
+      aggregation_handle_min_derived->finalizeHashTableEntry(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) + 1));
+  CheckMinValue<int>(
+      exclusive_key_source_min_val.getLiteral<int>(),
+      aggregation_handle_min_derived->finalizeHashTableEntry(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
index 1d1c084..31a35a0 100644
--- a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
@@ -29,8 +29,9 @@
 #include "expressions/aggregation/AggregationHandleSum.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
 #include "types/DoubleType.hpp"
@@ -45,10 +46,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "gtest/gtest.h"
 
@@ -186,36 +184,6 @@ class AggregationHandleSumTest : public ::testing::Test {
   }
 
   template <typename GenericType, typename PrecisionType>
-  void checkAggregationSumGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_)
-            .isNull());
-
-    typename PrecisionType::cpptype sum;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType, typename PrecisionType::cpptype>(
-            type, &sum));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_sum_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckSumValue<typename PrecisionType::cpptype>(
-        sum, *aggregation_handle_sum_, *cv_state);
-
-    aggregation_handle_sum_->mergeStates(*cv_state,
-                                         aggregation_handle_sum_state_.get());
-    CheckSumValue<typename PrecisionType::cpptype>(
-        sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType, typename PrecisionType>
   void checkAggregationSumGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
 
@@ -233,7 +201,8 @@ class AggregationHandleSumTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_sum_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -245,7 +214,6 @@ class AggregationHandleSumTest : public ::testing::Test {
     CheckSumValue<typename PrecisionType::cpptype>(
         sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_sum_;
   std::unique_ptr<AggregationState> aggregation_handle_sum_state_;
@@ -306,33 +274,6 @@ TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeTest) {
   checkAggregationSumGeneric<YearMonthIntervalType, YearMonthIntervalType>();
 }
 
-TEST_F(AggregationHandleSumTest, IntTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<IntType, LongType>();
-}
-
-TEST_F(AggregationHandleSumTest, LongTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<LongType, LongType>();
-}
-
-TEST_F(AggregationHandleSumTest, FloatTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<FloatType, DoubleType>();
-}
-
-TEST_F(AggregationHandleSumTest, DoubleTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<DoubleType, DoubleType>();
-}
-
-TEST_F(AggregationHandleSumTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<DatetimeIntervalType,
-                                         DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<YearMonthIntervalType,
-                                         YearMonthIntervalType>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleSumTest, IntTypeValueAccessorTest) {
   checkAggregationSumGenericValueAccessor<IntType, LongType>();
 }
@@ -358,7 +299,6 @@ TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeValueAccessorTest) {
   checkAggregationSumGenericValueAccessor<YearMonthIntervalType,
                                           YearMonthIntervalType>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleSumDeathTest, CharTypeTest) {
@@ -464,28 +404,25 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   initializeHandle(long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_sum_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_sum_.get()->getPayloadSize()},
           {aggregation_handle_sum_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_sum_.get()->getPayloadSize()},
           {aggregation_handle_sum_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleSum *aggregation_handle_sum_derived =
       static_cast<AggregationHandleSum *>(aggregation_handle_sum_.get());
@@ -563,49 +500,47 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckSumValue<std::int64_t>(
       common_key_merged_val.getLiteral<std::int64_t>(),
-      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+      aggregation_handle_sum_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
   CheckSumValue<std::int64_t>(
       exclusive_key_destination_sum_val.getLiteral<std::int64_t>(),
-      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+      aggregation_handle_sum_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(
-              exclusive_destination_key) +
-          1));
+              exclusive_destination_key) + 1));
   CheckSumValue<std::int64_t>(
       exclusive_key_source_sum_val.getLiteral<std::int64_t>(),
-      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+      aggregation_handle_sum_derived->finalizeHashTableEntry(
           source_hash_table_derived->getSingleCompositeKey(
-              exclusive_source_key) +
-          1));
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 895c2ea..ed0f99c 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -200,20 +200,6 @@ class QueryContext {
   }
 
   /**
-   * @brief Destroy the payloads from the aggregation hash tables.
-   *
-   * @warning After calling these methods, the hash table will be in an invalid
-   *          state. No other operation should be performed on them.
-   *
-   * @param id The ID of the AggregationOperationState.
-   **/
-  inline void destroyAggregationHashTablePayload(const aggregation_state_id id) {
-    DCHECK_LT(id, aggregation_states_.size());
-    DCHECK(aggregation_states_[id]);
-    aggregation_states_[id]->destroyAggregationHashTablePayload();
-  }
-
-  /**
    * @brief Whether the given GeneratorFunctionHandle id is valid.
    *
    * @param id The GeneratorFunctionHandle id.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 7f90e11..8b8fa3c 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -64,6 +64,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_expressions_Expressions_proto
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunction_proto
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_expressions_scalar_ScalarAttribute
@@ -125,6 +126,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_InitializeAggregationOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RelationalOperator
@@ -145,10 +147,12 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_storage_StorageBlockLayout_proto
                       quickstep_storage_SubBlockTypeRegistry
                       quickstep_types_Type
+                      quickstep_types_TypeID
                       quickstep_types_Type_proto
                       quickstep_types_TypedValue
                       quickstep_types_TypedValue_proto
                       quickstep_types_containers_Tuple_proto
+                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros
                       quickstep_utility_SqlError)
 if (ENABLE_DISTRIBUTED)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 6918313..1b50caa 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -49,6 +49,7 @@
 #include "expressions/Expressions.pb.h"
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunction.pb.h"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "expressions/scalar/ScalarAttribute.hpp"
@@ -105,6 +106,7 @@
 #include "relational_operators/DropTableOperator.hpp"
 #include "relational_operators/FinalizeAggregationOperator.hpp"
 #include "relational_operators/HashJoinOperator.hpp"
+#include "relational_operators/InitializeAggregationOperator.hpp"
 #include "relational_operators/InsertOperator.hpp"
 #include "relational_operators/NestedLoopsJoinOperator.hpp"
 #include "relational_operators/RelationalOperator.hpp"
@@ -126,9 +128,11 @@
 #include "storage/SubBlockTypeRegistry.hpp"
 #include "types/Type.hpp"
 #include "types/Type.pb.h"
+#include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "types/TypedValue.pb.h"
 #include "types/containers/Tuple.pb.h"
+#include "utility/EqualsAnyConstant.hpp"
 #include "utility/SqlError.hpp"
 
 #include "gflags/gflags.h"
@@ -159,6 +163,10 @@ static const volatile bool aggregate_hashtable_type_dummy
 
 DEFINE_bool(parallelize_load, true, "Parallelize loading data files.");
 
+DEFINE_int64(collision_free_vector_table_max_size, 1000000000,
+              "The maximum allowed key range (number of entries) for using a "
+              "CollisionFreeVectorTable.");
+
 namespace E = ::quickstep::optimizer::expressions;
 namespace P = ::quickstep::optimizer::physical;
 namespace S = ::quickstep::serialization;
@@ -371,6 +379,105 @@ void ExecutionGenerator::dropAllTemporaryRelations() {
   }
 }
 
+bool ExecutionGenerator::canUseCollisionFreeAggregation(
+    const P::AggregatePtr &aggregate,
+    const std::size_t estimated_num_groups,
+    std::size_t *max_num_groups) const {
+#ifdef QUICKSTEP_DISTRIBUTED
+  // Currently we cannot do this fast path with the distributed setting. See
+  // the TODOs at InitializeAggregationOperator::getAllWorkOrderProtos() and
+  // FinalizeAggregationOperator::getAllWorkOrderProtos().
+  return false;
+#endif
+
+  // Supports only single group-by key.
+  if (aggregate->grouping_expressions().size() != 1) {
+    return false;
+  }
+
+  // We need to know the exact min/max stats of the group-by key.
+  // So it must be a CatalogAttribute (but not an expression).
+  E::AttributeReferencePtr group_by_key_attr;
+  const E::ExpressionPtr agg_expr = aggregate->grouping_expressions().front();
+  if (!E::SomeAttributeReference::MatchesWithConditionalCast(agg_expr, &group_by_key_attr)) {
+    return false;
+  }
+
+  bool min_value_stat_is_exact;
+  bool max_value_stat_is_exact;
+  const TypedValue min_value =
+      cost_model_for_aggregation_->findMinValueStat(
+          aggregate, group_by_key_attr, &min_value_stat_is_exact);
+  const TypedValue max_value =
+      cost_model_for_aggregation_->findMaxValueStat(
+          aggregate, group_by_key_attr, &max_value_stat_is_exact);
+  if (min_value.isNull() || max_value.isNull() ||
+      (!min_value_stat_is_exact) || (!max_value_stat_is_exact)) {
+    return false;
+  }
+
+  std::int64_t min_cpp_value;
+  std::int64_t max_cpp_value;
+  switch (group_by_key_attr->getValueType().getTypeID()) {
+    case TypeID::kInt: {
+      min_cpp_value = min_value.getLiteral<int>();
+      max_cpp_value = max_value.getLiteral<int>();
+      break;
+    }
+    case TypeID::kLong: {
+      min_cpp_value = min_value.getLiteral<std::int64_t>();
+      max_cpp_value = max_value.getLiteral<std::int64_t>();
+      break;
+    }
+    default:
+      return false;
+  }
+
+  // TODO(jianqiao):
+  // 1. Handle the case where min_cpp_value is below 0 or far greater than 0.
+  // 2. Reason about the table size bound (e.g. by checking memory size) instead
+  //    of hardcoding it as a gflag.
+  if (min_cpp_value < 0 ||
+      max_cpp_value >= FLAGS_collision_free_vector_table_max_size ||
+      max_cpp_value / static_cast<double>(estimated_num_groups) > 256.0) {
+    return false;
+  }
+
+  for (const auto &agg_expr : aggregate->aggregate_expressions()) {
+    const E::AggregateFunctionPtr agg_func =
+        std::static_pointer_cast<const E::AggregateFunction>(agg_expr->expression());
+
+    if (agg_func->is_distinct()) {
+      return false;
+    }
+
+    // TODO(jianqiao): Support AggregationID::AVG.
+    if (!QUICKSTEP_EQUALS_ANY_CONSTANT(agg_func->getAggregate().getAggregationID(),
+                                       AggregationID::kCount,
+                                       AggregationID::kSum)) {
+      return false;
+    }
+
+    const auto &arguments = agg_func->getArguments();
+    if (arguments.size() > 1u) {
+      return false;
+    }
+
+    if (arguments.size() == 1u) {
+      if (!QUICKSTEP_EQUALS_ANY_CONSTANT(arguments.front()->getValueType().getTypeID(),
+                                         TypeID::kInt,
+                                         TypeID::kLong,
+                                         TypeID::kFloat,
+                                         TypeID::kDouble)) {
+        return false;
+      }
+    }
+  }
+
+  *max_num_groups = static_cast<std::size_t>(max_cpp_value) + 1;
+  return true;
+}
+
 void ExecutionGenerator::convertNamedExpressions(
     const std::vector<E::NamedExpressionPtr> &named_expressions,
     S::QueryContext::ScalarGroup *scalar_group_proto) {
@@ -1475,6 +1582,8 @@ void ExecutionGenerator::convertAggregate(
       findRelationInfoOutputByPhysical(physical_plan->input());
   aggr_state_proto->set_relation_id(input_relation_info->relation->getID());
 
+  bool use_parallel_initialization = false;
+
   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;
@@ -1495,9 +1604,25 @@ void ExecutionGenerator::convertAggregate(
   }
 
   if (!group_by_types.empty()) {
-    // Right now, only SeparateChaining is supported.
-    aggr_state_proto->set_hash_table_impl_type(
-        serialization::HashTableImplType::SEPARATE_CHAINING);
+    const std::size_t estimated_num_groups =
+        cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
+
+    std::size_t max_num_groups;
+    if (canUseCollisionFreeAggregation(physical_plan,
+                                       estimated_num_groups,
+                                       &max_num_groups)) {
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::COLLISION_FREE_VECTOR);
+      aggr_state_proto->set_estimated_num_entries(max_num_groups);
+      use_parallel_initialization = true;
+    } else {
+      // Otherwise, use SeparateChaining.
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::SEPARATE_CHAINING);
+      aggr_state_proto->set_estimated_num_entries(std::max(16uL, estimated_num_groups));
+    }
+  } else {
+    aggr_state_proto->set_estimated_num_entries(1uL);
   }
 
   for (const E::AliasPtr &named_aggregate_expression : physical_plan->aggregate_expressions()) {
@@ -1535,10 +1660,6 @@ void ExecutionGenerator::convertAggregate(
     aggr_state_proto->mutable_predicate()->CopyFrom(predicate->getProto());
   }
 
-  const std::size_t estimated_num_groups =
-      cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
-  aggr_state_proto->set_estimated_num_entries(std::max(16uL, estimated_num_groups));
-
   const QueryPlan::DAGNodeIndex aggregation_operator_index =
       execution_plan_->addRelationalOperator(
           new AggregationOperator(
@@ -1553,6 +1674,18 @@ void ExecutionGenerator::convertAggregate(
                                          false /* is_pipeline_breaker */);
   }
 
+  if (use_parallel_initialization) {
+    const QueryPlan::DAGNodeIndex initialize_aggregation_operator_index =
+        execution_plan_->addRelationalOperator(
+            new InitializeAggregationOperator(
+                query_handle_->query_id(),
+                aggr_state_index));
+
+    execution_plan_->addDirectDependency(aggregation_operator_index,
+                                         initialize_aggregation_operator_index,
+                                         true /* is_pipeline_breaker */);
+  }
+
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;
   const QueryContext::insert_destination_id insert_destination_index =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index eba6eee..987f11a 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXECUTION_GENERATOR_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXECUTION_GENERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <unordered_map>
@@ -37,6 +38,7 @@
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#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"
@@ -204,6 +206,22 @@ class ExecutionGenerator {
   std::string getNewRelationName();
 
   /**
+   * @brief Checks whether an aggregate node can be efficiently evaluated with
+   *        the collision-free aggregation fast path.
+   *
+   * @param aggregate The physical aggregate node to be checked.
+   * @param estimated_num_groups The estimated number of groups for the aggregate.
+   * @param exact_num_groups If collision-free aggregation is applicable, the
+   *        pointed content of this pointer will be set as the maximum possible
+   *        number of groups that the collision-free hash table need to hold.
+   * @return A bool value indicating whether collision-free aggregation can be
+   *         used to evaluate \p aggregate.
+   */
+  bool canUseCollisionFreeAggregation(const physical::AggregatePtr &aggregate,
+                                      const std::size_t estimated_num_groups,
+                                      std::size_t *max_num_groups) const;
+
+  /**
    * @brief Sets up the info of the CatalogRelation represented by TableReference.
    *        TableReference is not converted to any operator.
    *
@@ -427,7 +445,7 @@ class ExecutionGenerator {
   /**
    * @brief The cost model to use for estimating aggregation hash table size.
    */
-  std::unique_ptr<cost::CostModel> cost_model_for_aggregation_;
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_for_aggregation_;
 
   /**
    * @brief The cost model to use for estimating join hash table size.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c18dc77..df4114d 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -47,6 +47,9 @@ add_library(quickstep_relationaloperators_FinalizeAggregationOperator
             FinalizeAggregationOperator.cpp
             FinalizeAggregationOperator.hpp)
 add_library(quickstep_relationaloperators_HashJoinOperator HashJoinOperator.cpp HashJoinOperator.hpp)
+add_library(quickstep_relationaloperators_InitializeAggregationOperator
+            InitializeAggregationOperator.cpp
+            InitializeAggregationOperator.hpp)
 add_library(quickstep_relationaloperators_InsertOperator InsertOperator.cpp InsertOperator.hpp)
 add_library(quickstep_relationaloperators_NestedLoopsJoinOperator
             NestedLoopsJoinOperator.cpp
@@ -254,6 +257,17 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_InitializeAggregationOperator
+                      glog
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_storage_AggregationOperationState
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_InsertOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -548,6 +562,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_InitializeAggregationOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RebuildWorkOrder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/DestroyAggregationStateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyAggregationStateOperator.cpp b/relational_operators/DestroyAggregationStateOperator.cpp
index 49be43d..62ca9e7 100644
--- a/relational_operators/DestroyAggregationStateOperator.cpp
+++ b/relational_operators/DestroyAggregationStateOperator.cpp
@@ -58,13 +58,6 @@ bool DestroyAggregationStateOperator::getAllWorkOrderProtos(WorkOrderProtosConta
 }
 
 void DestroyAggregationStateWorkOrder::execute() {
-  // NOTE(harshad) : The destroyAggregationHashTablePayload call is separate
-  // from the destroyAggregationState call. The reason is that the aggregation
-  // hash tables don't own the AggregationHandle objects. However the hash table
-  // class requires the handles for destroying the payload (see the
-  // destroyPayload methods in AggregationHandle classes). Therefore, we first
-  // destroy the payloads in the hash table and then destroy the hash table.
-  query_context_->destroyAggregationHashTablePayload(aggr_state_index_);
   query_context_->destroyAggregationState(aggr_state_index_);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 0cbf635..77b4b97 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -44,15 +44,15 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
     AggregationOperationState *agg_state =
         query_context->getAggregationState(aggr_state_index_);
     DCHECK(agg_state != nullptr);
-    for (int part_id = 0;
-         part_id < static_cast<int>(agg_state->getNumPartitions());
+    for (std::size_t part_id = 0;
+         part_id < agg_state->getNumFinalizationPartitions();
          ++part_id) {
       container->addNormalWorkOrder(
           new FinalizeAggregationWorkOrder(
               query_id_,
+              part_id,
               agg_state,
-              query_context->getInsertDestination(output_destination_index_),
-              part_id),
+              query_context->getInsertDestination(output_destination_index_)),
           op_index_);
     }
   }
@@ -61,7 +61,7 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
 
 // TODO(quickstep-team) : Think about how the number of partitions could be
 // accessed in this function. Until then, we can't use partitioned aggregation
-// with the distributed version.
+// finalization with the distributed version.
 bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (blocking_dependencies_met_ && !started_) {
     started_ = true;
@@ -80,11 +80,7 @@ bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer
 }
 
 void FinalizeAggregationWorkOrder::execute() {
-  if (state_->isAggregatePartitioned()) {
-    state_->finalizeAggregatePartitioned(part_id_, output_destination_);
-  } else {
-    state_->finalizeAggregate(output_destination_);
-  }
+  state_->finalizeAggregate(partition_id_, output_destination_);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index ae7127a..3c209b1 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -116,29 +116,29 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
    * @note InsertWorkOrder takes ownership of \c state.
    *
    * @param query_id The ID of the query to which this operator belongs.
+   * @param partition_id The partition ID for which the Finalize aggregation
+   *        work order is issued.
    * @param state The AggregationState to use.
    * @param output_destination The InsertDestination to insert aggregation
    *        results.
-   * @param part_id The partition ID for which the Finalize aggregation work
-   *        order is issued. Ignore if aggregation is not partitioned.
    */
   FinalizeAggregationWorkOrder(const std::size_t query_id,
+                               const std::size_t partition_id,
                                AggregationOperationState *state,
-                               InsertDestination *output_destination,
-                               const int part_id = -1)
+                               InsertDestination *output_destination)
       : WorkOrder(query_id),
+        partition_id_(partition_id),
         state_(DCHECK_NOTNULL(state)),
-        output_destination_(DCHECK_NOTNULL(output_destination)),
-        part_id_(part_id) {}
+        output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~FinalizeAggregationWorkOrder() override {}
 
   void execute() override;
 
  private:
+  const std::size_t partition_id_;
   AggregationOperationState *state_;
   InsertDestination *output_destination_;
-  const int part_id_;
 
   DISALLOW_COPY_AND_ASSIGN(FinalizeAggregationWorkOrder);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/InitializeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.cpp b/relational_operators/InitializeAggregationOperator.cpp
new file mode 100644
index 0000000..b1063ad
--- /dev/null
+++ b/relational_operators/InitializeAggregationOperator.cpp
@@ -0,0 +1,72 @@
+/**
+ * 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 "relational_operators/InitializeAggregationOperator.hpp"
+
+#include <cstddef>
+
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
+#include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
+#include "storage/AggregationOperationState.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool InitializeAggregationOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  if (!started_) {
+    AggregationOperationState *agg_state =
+        query_context->getAggregationState(aggr_state_index_);
+    DCHECK(agg_state != nullptr);
+
+    for (std::size_t part_id = 0;
+         part_id < agg_state->getNumInitializationPartitions();
+         ++part_id) {
+      container->addNormalWorkOrder(
+          new InitializeAggregationWorkOrder(query_id_,
+                                             part_id,
+                                             agg_state),
+          op_index_);
+    }
+    started_ = true;
+  }
+  return true;
+}
+
+// TODO(quickstep-team) : Think about how the number of partitions could be
+// accessed in this function. Until then, we can't use partitioned aggregation
+// initialization with the distributed version.
+bool InitializeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  LOG(FATAL) << "Not supported";
+}
+
+void InitializeAggregationWorkOrder::execute() {
+  state_->initialize(partition_id_);
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/InitializeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.hpp b/relational_operators/InitializeAggregationOperator.hpp
new file mode 100644
index 0000000..58d848b
--- /dev/null
+++ b/relational_operators/InitializeAggregationOperator.hpp
@@ -0,0 +1,122 @@
+/**
+ * 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_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_OPERATOR_HPP_
+
+#include <string>
+
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class AggregationOperationState;
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+namespace serialization { class WorkOrder; }
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+/**
+ * @brief An operator which initializes an AggregationOperationState.
+ **/
+class InitializeAggregationOperator : public RelationalOperator {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of this query.
+   * @param aggr_state_index The index of the AggregationOperationState in QueryContext.
+   **/
+  InitializeAggregationOperator(const std::size_t query_id,
+                                const QueryContext::aggregation_state_id aggr_state_index)
+      : RelationalOperator(query_id),
+        aggr_state_index_(aggr_state_index),
+        started_(false) {}
+
+  ~InitializeAggregationOperator() override {}
+
+  std::string getName() const override {
+    return "InitializeAggregationOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+ private:
+  const QueryContext::aggregation_state_id aggr_state_index_;
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeAggregationOperator);
+};
+
+/**
+ * @brief A WorkOrder produced by InitializeAggregationOperator.
+ **/
+class InitializeAggregationWorkOrder : public WorkOrder {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this operator belongs.
+   * @param partition_id The partition ID for which the work order is issued.
+   * @param state The AggregationOperationState to be initialized.
+   */
+  InitializeAggregationWorkOrder(const std::size_t query_id,
+                                 const std::size_t partition_id,
+                                 AggregationOperationState *state)
+      : WorkOrder(query_id),
+        partition_id_(partition_id),
+        state_(DCHECK_NOTNULL(state)) {}
+
+  ~InitializeAggregationWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  const std::size_t partition_id_;
+
+  AggregationOperationState *state_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeAggregationWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 5e8d03d..bd2a0f8 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -184,8 +184,11 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     }
     case serialization::FINALIZE_AGGREGATION: {
       LOG(INFO) << "Creating FinalizeAggregationWorkOrder in Shiftboss " << shiftboss_index;
+      // TODO(quickstep-team): Handle inner-table partitioning in the distributed
+      // setting.
       return new FinalizeAggregationWorkOrder(
           proto.query_id(),
+          0uL /* partition_id */,
           query_context->getAggregationState(proto.GetExtension(
               serialization::FinalizeAggregationWorkOrder::aggr_state_index)),
           query_context->getInsertDestination(


[13/50] [abbrv] incubator-quickstep git commit: Simplified the SelectOperator w/ partitions.

Posted by ji...@apache.org.
Simplified the SelectOperator w/ partitions.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 929e5f1dd178538725eac1644712828e9dc5843d
Parents: 7727e77
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Jan 30 00:35:12 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 2 19:13:30 2017 -0800

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp  | 30 +++++++---
 relational_operators/SelectOperator.cpp | 84 +++++++++++-----------------
 relational_operators/SelectOperator.hpp | 67 ++++++++++------------
 3 files changed, 84 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/929e5f1d/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index ce1452e..b73de39 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -551,6 +551,13 @@ void ExecutionGenerator::convertSelection(
   const CatalogRelationInfo *input_relation_info =
       findRelationInfoOutputByPhysical(physical_selection->input());
   DCHECK(input_relation_info != nullptr);
+  const CatalogRelation &input_relation = *input_relation_info->relation;
+  const PartitionScheme *input_partition_scheme = input_relation.getPartitionScheme();
+
+  const std::size_t num_partitions =
+      input_partition_scheme
+          ? input_partition_scheme->getPartitionSchemeHeader().getNumPartitions()
+          : 1u;
 
   // Use the "simple" form of the selection operator (a pure projection that
   // doesn't require any expression evaluation or intermediate copies) if
@@ -559,19 +566,21 @@ void ExecutionGenerator::convertSelection(
   SelectOperator *op =
       convertSimpleProjection(project_expressions_group_index, &attributes)
           ? new SelectOperator(query_handle_->query_id(),
-                               *input_relation_info->relation,
+                               input_relation,
                                *output_relation,
                                insert_destination_index,
                                execution_predicate_index,
                                move(attributes),
-                               input_relation_info->isStoredRelation())
+                               input_relation_info->isStoredRelation(),
+                               num_partitions)
           : new SelectOperator(query_handle_->query_id(),
-                               *input_relation_info->relation,
+                               input_relation,
                                *output_relation,
                                insert_destination_index,
                                execution_predicate_index,
                                project_expressions_group_index,
-                               input_relation_info->isStoredRelation());
+                               input_relation_info->isStoredRelation(),
+                               num_partitions);
 
   const QueryPlan::DAGNodeIndex select_index =
       execution_plan_->addRelationalOperator(op);
@@ -1310,7 +1319,13 @@ void ExecutionGenerator::convertInsertSelection(
 
   const CatalogRelationInfo *selection_relation_info =
       findRelationInfoOutputByPhysical(physical_plan->selection());
-  const CatalogRelation *selection_relation = selection_relation_info->relation;
+  const CatalogRelation &selection_relation = *selection_relation_info->relation;
+  const PartitionScheme *selection_partition_scheme = selection_relation.getPartitionScheme();
+
+  const std::size_t num_partitions =
+      selection_partition_scheme
+          ? selection_partition_scheme->getPartitionSchemeHeader().getNumPartitions()
+          : 1u;
 
   // Prepare the attributes, which are output columns of the selection relation.
   std::vector<attribute_id> attributes;
@@ -1331,12 +1346,13 @@ void ExecutionGenerator::convertInsertSelection(
   // physical plan by modifying class Physical.
   SelectOperator *insert_selection_op =
       new SelectOperator(query_handle_->query_id(),
-                         *selection_relation,
+                         selection_relation,
                          destination_relation,
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
                          move(attributes),
-                         selection_relation_info->isStoredRelation());
+                         selection_relation_info->isStoredRelation(),
+                         num_partitions);
 
   const QueryPlan::DAGNodeIndex insert_selection_index =
       execution_plan_->addRelationalOperator(insert_selection_op);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/929e5f1d/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 5419cf8..b63f0be 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -66,64 +66,40 @@ bool SelectOperator::getAllWorkOrders(
       return true;
     }
 
-    if (input_relation_.hasPartitionScheme()) {
-      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
-        for (const block_id input_block_id : input_relation_block_ids_in_partition_[part_id]) {
-          numa_node_id numa_node = 0;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (const block_id input_block_id : input_relation_block_ids_[part_id]) {
+        numa_node_id numa_node = 0;
 #ifdef QUICKSTEP_HAVE_LIBNUMA
-          if (input_relation_.hasNUMAPlacementScheme()) {
-            numa_node = placement_scheme_->getNUMANodeForBlock(input_block_id);
-          }
-#endif  // QUICKSTEP_HAVE_LIBNUMA
-          container->addNormalWorkOrder(
-              new SelectWorkOrder(query_id_, input_relation_, input_block_id, predicate, simple_projection_,
-                                  simple_selection_, selection, output_destination, storage_manager,
-                                  CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context), numa_node),
-              op_index_);
+        if (input_relation_.hasNUMAPlacementScheme()) {
+          numa_node = placement_scheme_->getNUMANodeForBlock(input_block_id);
         }
-      }
-    } else {
-      for (const block_id input_block_id : input_relation_block_ids_) {
+#endif  // QUICKSTEP_HAVE_LIBNUMA
         container->addNormalWorkOrder(
             new SelectWorkOrder(query_id_, input_relation_, input_block_id, predicate, simple_projection_,
                                 simple_selection_, selection, output_destination, storage_manager,
-                                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+                                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context), numa_node),
             op_index_);
       }
     }
     started_ = true;
     return true;
   } else {
-    if (input_relation_.hasPartitionScheme()) {
-      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
-        while (num_workorders_generated_in_partition_[part_id] <
-               input_relation_block_ids_in_partition_[part_id].size()) {
-          const block_id block_in_partition
-              = input_relation_block_ids_in_partition_[part_id][num_workorders_generated_in_partition_[part_id]];
-
-          numa_node_id numa_node = 0;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      while (num_workorders_generated_[part_id] < input_relation_block_ids_[part_id].size()) {
+        const block_id block = input_relation_block_ids_[part_id][num_workorders_generated_[part_id]];
+
+        numa_node_id numa_node = 0;
 #ifdef QUICKSTEP_HAVE_LIBNUMA
-          if (input_relation_.hasNUMAPlacementScheme()) {
-            numa_node = placement_scheme_->getNUMANodeForBlock(block_in_partition);
-          }
-#endif  // QUICKSTEP_HAVE_LIBNUMA
-          container->addNormalWorkOrder(
-              new SelectWorkOrder(query_id_, input_relation_, block_in_partition, predicate, simple_projection_,
-                                  simple_selection_, selection, output_destination, storage_manager,
-                                  CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context), numa_node),
-              op_index_);
-          ++num_workorders_generated_in_partition_[part_id];
+        if (input_relation_.hasNUMAPlacementScheme()) {
+          numa_node = placement_scheme_->getNUMANodeForBlock(block);
         }
-      }
-    } else {
-      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+#endif  // QUICKSTEP_HAVE_LIBNUMA
         container->addNormalWorkOrder(
-            new SelectWorkOrder(query_id_, input_relation_, input_relation_block_ids_[num_workorders_generated_],
-                                predicate, simple_projection_, simple_selection_, selection, output_destination,
-                                storage_manager,
-                                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+            new SelectWorkOrder(query_id_, input_relation_, block, predicate, simple_projection_,
+                                simple_selection_, selection, output_destination, storage_manager,
+                                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context), numa_node),
             op_index_);
-        ++num_workorders_generated_;
+        ++num_workorders_generated_[part_id];
       }
     }
     return done_feeding_input_relation_;
@@ -132,19 +108,25 @@ bool SelectOperator::getAllWorkOrders(
 
 bool SelectOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (input_relation_is_stored_) {
-    if (!started_) {
-      for (const block_id input_block_id : input_relation_block_ids_) {
+    if (started_) {
+      return true;
+    }
+
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (const block_id input_block_id : input_relation_block_ids_[part_id]) {
         container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
       }
-      started_ = true;
     }
+    started_ = true;
     return true;
   } else {
-    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-      container->addWorkOrderProto(
-          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
-          op_index_);
-      ++num_workorders_generated_;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      while (num_workorders_generated_[part_id] < input_relation_block_ids_[part_id].size()) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(input_relation_block_ids_[part_id][num_workorders_generated_[part_id]]),
+            op_index_);
+        ++num_workorders_generated_[part_id];
+      }
     }
     return done_feeding_input_relation_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/929e5f1d/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 79ab37f..b9a4d49 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -85,6 +85,8 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
+   * @param num_partitions The number of partitions in 'input_relation'.
+   *        If no partitions, it is one.
    **/
   SelectOperator(
       const std::size_t query_id,
@@ -93,36 +95,33 @@ class SelectOperator : public RelationalOperator {
       const QueryContext::insert_destination_id output_destination_index,
       const QueryContext::predicate_id predicate_index,
       const QueryContext::scalar_group_id selection_index,
-      const bool input_relation_is_stored)
+      const bool input_relation_is_stored,
+      const std::size_t num_partitions)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
         selection_index_(selection_index),
-        num_workorders_generated_(0),
+        num_partitions_(num_partitions),
+        input_relation_block_ids_(num_partitions),
+        num_workorders_generated_(num_partitions),
         simple_projection_(false),
         input_relation_is_stored_(input_relation_is_stored),
         started_(false) {
 #ifdef QUICKSTEP_HAVE_LIBNUMA
     placement_scheme_ = input_relation.getNUMAPlacementSchemePtr();
 #endif
-    if (input_relation.hasPartitionScheme()) {
-      const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
-      num_partitions_ = part_scheme.getPartitionSchemeHeader().getNumPartitions();
+    if (input_relation_is_stored) {
+      if (input_relation.hasPartitionScheme()) {
+        const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
 
-      num_workorders_generated_in_partition_.resize(num_partitions_);
-
-      if (input_relation_is_stored) {
         for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
-          input_relation_block_ids_in_partition_.push_back(
-              part_scheme.getBlocksInPartition(part_id));
+          input_relation_block_ids_[part_id] = part_scheme.getBlocksInPartition(part_id);
         }
       } else {
-        input_relation_block_ids_in_partition_.resize(num_partitions_);
+        input_relation_block_ids_[0] = input_relation.getBlocksSnapshot();
       }
-    } else if (input_relation_is_stored) {
-      input_relation_block_ids_ = input_relation.getBlocksSnapshot();
     }
   }
 
@@ -144,6 +143,8 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
+   * @param num_partitions The number of partitions in 'input_relation'.
+   *        If no partitions, it is one.
    **/
   SelectOperator(
       const std::size_t query_id,
@@ -152,7 +153,8 @@ class SelectOperator : public RelationalOperator {
       const QueryContext::insert_destination_id output_destination_index,
       const QueryContext::predicate_id predicate_index,
       std::vector<attribute_id> &&selection,
-      const bool input_relation_is_stored)
+      const bool input_relation_is_stored,
+      const std::size_t num_partitions)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -160,29 +162,25 @@ class SelectOperator : public RelationalOperator {
         predicate_index_(predicate_index),
         selection_index_(QueryContext::kInvalidScalarGroupId),
         simple_selection_(std::move(selection)),
-        num_workorders_generated_(0),
+        num_partitions_(num_partitions),
+        input_relation_block_ids_(num_partitions),
+        num_workorders_generated_(num_partitions),
         simple_projection_(true),
         input_relation_is_stored_(input_relation_is_stored),
         started_(false) {
 #ifdef QUICKSTEP_HAVE_LIBNUMA
     placement_scheme_ = input_relation.getNUMAPlacementSchemePtr();
 #endif
-    if (input_relation.hasPartitionScheme()) {
-      const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
-      num_partitions_ = part_scheme.getPartitionSchemeHeader().getNumPartitions();
-
-      num_workorders_generated_in_partition_.resize(num_partitions_);
+    if (input_relation_is_stored) {
+      if (input_relation.hasPartitionScheme()) {
+        const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
 
-      if (input_relation_is_stored) {
         for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
-          input_relation_block_ids_in_partition_.push_back(
-              part_scheme.getBlocksInPartition(part_id));
+          input_relation_block_ids_[part_id] = part_scheme.getBlocksInPartition(part_id);
         }
       } else {
-        input_relation_block_ids_in_partition_.resize(num_partitions_);
+        input_relation_block_ids_[0] = input_relation.getBlocksSnapshot();
       }
-    } else if (input_relation_is_stored) {
-      input_relation_block_ids_ = input_relation.getBlocksSnapshot();
     }
   }
 
@@ -206,11 +204,7 @@ class SelectOperator : public RelationalOperator {
 
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
                       const partition_id part_id) override {
-    if (input_relation_.hasPartitionScheme()) {
-      input_relation_block_ids_in_partition_[part_id].push_back(input_block_id);
-    } else {
-      input_relation_block_ids_.push_back(input_block_id);
-    }
+    input_relation_block_ids_[part_id].push_back(input_block_id);
   }
 
   QueryContext::insert_destination_id getInsertDestinationID() const override {
@@ -237,17 +231,12 @@ class SelectOperator : public RelationalOperator {
   const QueryContext::scalar_group_id selection_index_;
   const std::vector<attribute_id> simple_selection_;
 
-  std::vector<block_id> input_relation_block_ids_;
-  // A single workorder is generated for each block of input relation.
-  std::vector<block_id>::size_type num_workorders_generated_;
-
-  // Used for the partition case only.
+  const std::size_t num_partitions_;
   // A vector of vectors V where V[i] indicates the list of block IDs of the
   // input relation that belong to the partition i.
-  std::vector<std::vector<block_id>> input_relation_block_ids_in_partition_;
+  std::vector<std::vector<block_id>> input_relation_block_ids_;
   // A single workorder is generated for each block in each partition of input relation.
-  std::vector<std::size_t> num_workorders_generated_in_partition_;
-  std::size_t num_partitions_;
+  std::vector<std::size_t> num_workorders_generated_;
 
   const bool simple_projection_;
   const bool input_relation_is_stored_;


[05/50] [abbrv] incubator-quickstep git commit: Push down low cost disjunctive predicates to filter the stored relations early

Posted by ji...@apache.org.
Push down low cost disjunctive predicates to filter the stored relations early


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

Branch: refs/heads/LIP-time-decomposition
Commit: 259cd5e731ead6e38f546c66211aceb3c20f6f4d
Parents: 6d83b46
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 30 01:02:19 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Tue Jan 31 10:59:08 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/PhysicalGenerator.cpp           |  15 ++
 query_optimizer/rules/CMakeLists.txt            |  24 ++
 .../PushDownLowCostDisjunctivePredicate.cpp     | 225 +++++++++++++++++++
 .../PushDownLowCostDisjunctivePredicate.hpp     | 116 ++++++++++
 5 files changed, 381 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index e8bc21c..0ca971d 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index e12f8be..bd05267 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
+#include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
 #include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
@@ -108,12 +109,22 @@ P::PhysicalPtr PhysicalGenerator::generateInitialPlan(
 P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
   rules.emplace_back(new PruneColumns());
+
+  // TODO(jianqiao): It is possible for PushDownLowCostDisjunctivePredicate to
+  // generate two chaining Selection nodes that can actually be fused into one.
+  // Note that currently it is okay to have the two Selections because they are
+  // applied to a small cardinality stored relation, which is very light-weight.
+  // However it is better to have a FuseSelection optimization (or even a more
+  // general FusePhysical optimization) in the future.
+  rules.emplace_back(new PushDownLowCostDisjunctivePredicate());
+
   if (FLAGS_reorder_hash_joins) {
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
     rules.emplace_back(new PruneColumns());
   } else {
     rules.emplace_back(new SwapProbeBuild());
   }
+
   if (FLAGS_reorder_columns) {
     // NOTE(jianqiao): This optimization relies on the fact that the intermediate
     // relations all have SPLIT_ROW_STORE layouts. If this fact gets changed, the
@@ -121,6 +132,10 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     // should be re-evaluated.
     rules.emplace_back(new ReorderColumns());
   }
+
+  // NOTE(jianqiao): Adding rules after AttachLIPFilters requires extra handling
+  // of LIPFilterConfiguration for transformed nodes. So currently it is suggested
+  // that all the new rules be placed before this point.
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index fe2fd17..86d1ef7 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -24,6 +24,9 @@ add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp C
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
+add_library(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+            PushDownLowCostDisjunctivePredicate.cpp
+            PushDownLowCostDisjunctivePredicate.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
 add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
@@ -111,6 +114,26 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+                      ${GFLAGS_LIB_NAME}
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_LogicalAnd
+                      quickstep_queryoptimizer_expressions_LogicalOr
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_NestedLoopsJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExpressionUtil
@@ -225,6 +248,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_GenerateJoins
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
+                      quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
new file mode 100644
index 0000000..e39f155
--- /dev/null
+++ b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
@@ -0,0 +1,225 @@
+/**
+ * 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 "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
+
+#include <cstddef>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/LogicalAnd.hpp"
+#include "query_optimizer/expressions/LogicalOr.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+
+#include "gflags/gflags.h"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+DEFINE_uint64(push_down_disjunctive_predicate_cardinality_threshold, 100u,
+              "The cardinality threshold for a stored relation for the "
+              "PushDownLowCostDisjunctivePredicate optimization rule to push "
+              "down a disjunctive predicate to pre-filter that relation.");
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr PushDownLowCostDisjunctivePredicate::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  const P::TopLevelPlanPtr top_level_plan =
+     std::static_pointer_cast<const P::TopLevelPlan>(input);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(
+          top_level_plan->shared_subplans()));
+
+  collectApplicablePredicates(input);
+
+  if (!applicable_predicates_.empty()) {
+    // Apply the selected predicates to stored relations.
+    return attachPredicates(input);
+  } else {
+    return input;
+  }
+}
+
+void PushDownLowCostDisjunctivePredicate::collectApplicablePredicates(
+    const physical::PhysicalPtr &input) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(input, &table_reference)) {
+    // Consider only stored relations with small cardinality as targets.
+    if (cost_model_->estimateCardinality(input) <=
+            FLAGS_push_down_disjunctive_predicate_cardinality_threshold) {
+      applicable_nodes_.emplace_back(input, &table_reference->attribute_list());
+    }
+    return;
+  }
+
+  for (const auto &child : input->children()) {
+    collectApplicablePredicates(child);
+  }
+
+  E::PredicatePtr filter_predicate = nullptr;
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      filter_predicate =
+          std::static_pointer_cast<const P::Aggregate>(input)->filter_predicate();
+      break;
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr hash_join =
+          std::static_pointer_cast<const P::HashJoin>(input);
+      if (hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
+        filter_predicate = hash_join->residual_predicate();
+      }
+      break;
+    }
+    case P::PhysicalType::kNestedLoopsJoin: {
+      filter_predicate =
+          std::static_pointer_cast<const P::NestedLoopsJoin>(input)->join_predicate();
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      filter_predicate =
+          std::static_pointer_cast<const P::Selection>(input)->filter_predicate();
+      break;
+    }
+    default:
+      break;
+  }
+
+  E::LogicalOrPtr disjunctive_predicate;
+  if (filter_predicate == nullptr ||
+      !E::SomeLogicalOr::MatchesWithConditionalCast(filter_predicate, &disjunctive_predicate)) {
+    return;
+  }
+
+  // Consider only disjunctive normal form, i.e. disjunction of conjunctions.
+  // Divide the disjunctive components into groups.
+  std::vector<std::vector<E::PredicatePtr>> candidate_predicates;
+  std::vector<std::vector<std::vector<E::AttributeReferencePtr>>> candidate_attributes;
+  for (const auto &conjunctive_predicate : disjunctive_predicate->operands()) {
+    candidate_predicates.emplace_back();
+    candidate_attributes.emplace_back();
+    E::LogicalAndPtr logical_and;
+    if (E::SomeLogicalAnd::MatchesWithConditionalCast(conjunctive_predicate, &logical_and)) {
+      for (const auto &predicate : logical_and->operands()) {
+        candidate_predicates.back().emplace_back(predicate);
+        candidate_attributes.back().emplace_back(
+            predicate->getReferencedAttributes());
+      }
+    } else {
+      candidate_predicates.back().emplace_back(conjunctive_predicate);
+      candidate_attributes.back().emplace_back(
+          conjunctive_predicate->getReferencedAttributes());
+    }
+  }
+
+  // Check whether the conditions are met for pushing down part of the predicates
+  // to each small-cardinality stored relation.
+  for (const auto &node_pair : applicable_nodes_) {
+    const std::vector<E::AttributeReferencePtr> &target_attributes = *node_pair.second;
+    std::vector<E::PredicatePtr> selected_disj_preds;
+    for (std::size_t i = 0; i < candidate_predicates.size(); ++i) {
+      const auto &cand_preds = candidate_predicates[i];
+      const auto &cand_attrs = candidate_attributes[i];
+
+      std::vector<E::PredicatePtr> selected_conj_preds;
+      for (std::size_t j = 0; j < cand_preds.size(); ++j) {
+        if (E::SubsetOfExpressions(cand_attrs[j], target_attributes)) {
+          selected_conj_preds.emplace_back(cand_preds[j]);
+        }
+      }
+      if (selected_conj_preds.empty()) {
+        // Not every disjunctive component contains a predicate that can be applied
+        // to the table reference node -- condition failed, exit.
+        selected_disj_preds.clear();
+        break;
+      } else {
+        selected_disj_preds.emplace_back(
+            CreateConjunctive(selected_conj_preds));
+      }
+    }
+    if (!selected_disj_preds.empty()) {
+      applicable_predicates_[node_pair.first].add(
+          CreateDisjunctive(selected_disj_preds));
+    }
+  }
+}
+
+P::PhysicalPtr PushDownLowCostDisjunctivePredicate::attachPredicates(
+    const P::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = attachPredicates(child);
+    new_children.push_back(new_child);
+  }
+
+  const P::PhysicalPtr output =
+      new_children == input->children() ? input
+                                        : input->copyWithNewChildren(new_children);
+
+  const auto &node_it = applicable_predicates_.find(input);
+  if (node_it != applicable_predicates_.end()) {
+    const E::PredicatePtr filter_predicate =
+        CreateConjunctive(node_it->second.predicates);
+    return P::Selection::Create(output,
+                                E::ToNamedExpressions(output->getOutputAttributes()),
+                                filter_predicate);
+  }
+
+  return output;
+}
+
+E::PredicatePtr PushDownLowCostDisjunctivePredicate::CreateConjunctive(
+    const std::vector<E::PredicatePtr> predicates) {
+  DCHECK_GE(predicates.size(), 1u);
+  if (predicates.size() == 1) {
+    return predicates.front();
+  } else {
+    return E::LogicalAnd::Create(predicates);
+  }
+}
+
+E::PredicatePtr PushDownLowCostDisjunctivePredicate::CreateDisjunctive(
+    const std::vector<E::PredicatePtr> predicates) {
+  DCHECK_GE(predicates.size(), 1u);
+  if (predicates.size() == 1) {
+    return predicates.front();
+  } else {
+    return E::LogicalOr::Create(predicates);
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
new file mode 100644
index 0000000..3e4b602
--- /dev/null
+++ b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_
+
+#include <cstddef>
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to push down low-cost disjunctive
+ *        predicate when proper conditions are met.
+ *
+ * Here we elaborate the conditions.
+ *
+ * Let
+ *   P = p_{1,1} AND ... AND p_{1, m_1} OR ... OR p_{n,1} AND ... AND p_{n, m_n}
+ * be a predicate in disjunctive normal form.
+ *
+ * Now consider each small-cardinality relation R, if for each i in 1..n, there
+ * exists at least one predicate p_{i, k_i} that is applicable to R. Then we can
+ * construct a new predicate
+ *   P' = p_{1, k_1} OR ... OR p_{n, k_n}
+ * and push down P' to be applied to R.
+ *
+ * Also, if any conjunctive component in P contains more than one predicate that
+ * is applicable to R, then we can combine all these applicable predicates as a
+ * conjunctive component in P'.
+ *
+ * Finally, note that if there exists a conjunctive component that contains no
+ * predicate applicable to R. Then the condition fails and we cannot do a push
+ * down for R.
+ */
+class PushDownLowCostDisjunctivePredicate : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  PushDownLowCostDisjunctivePredicate() {}
+
+  ~PushDownLowCostDisjunctivePredicate() override {}
+
+  std::string getName() const override {
+    return "PushDownLowCostDisjunctivePredicate";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct PredicateInfo {
+    PredicateInfo() {}
+    inline void add(expressions::PredicatePtr predicate) {
+      predicates.emplace_back(predicate);
+    }
+    std::vector<expressions::PredicatePtr> predicates;
+  };
+
+  void collectApplicablePredicates(const physical::PhysicalPtr &input);
+
+  physical::PhysicalPtr attachPredicates(const physical::PhysicalPtr &input) const;
+
+  static expressions::PredicatePtr CreateConjunctive(
+      const std::vector<expressions::PredicatePtr> predicates);
+
+  static expressions::PredicatePtr CreateDisjunctive(
+      const std::vector<expressions::PredicatePtr> predicates);
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  std::vector<std::pair<physical::PhysicalPtr,
+                        const std::vector<expressions::AttributeReferencePtr> *>> applicable_nodes_;
+  std::map<physical::PhysicalPtr, PredicateInfo> applicable_predicates_;
+
+  DISALLOW_COPY_AND_ASSIGN(PushDownLowCostDisjunctivePredicate);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_


[19/50] [abbrv] incubator-quickstep git commit: Fix the bug with SingleIdentityHashFilter when filter_cardinality is 0.

Posted by ji...@apache.org.
Fix the bug with SingleIdentityHashFilter when filter_cardinality is 0.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 669a74c71fde158106e72fba936de2104998aa61
Parents: aef1c35
Author: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Authored: Sun Feb 5 15:46:48 2017 -0600
Committer: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Committed: Tue Feb 7 10:16:01 2017 -0600

----------------------------------------------------------------------
 query_optimizer/rules/AttachLIPFilters.cpp      | 3 ++-
 utility/lip_filter/SingleIdentityHashFilter.hpp | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/669a74c7/query_optimizer/rules/AttachLIPFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachLIPFilters.cpp b/query_optimizer/rules/AttachLIPFilters.cpp
index 48b68bc..4b6ac59 100644
--- a/query_optimizer/rules/AttachLIPFilters.cpp
+++ b/query_optimizer/rules/AttachLIPFilters.cpp
@@ -19,6 +19,7 @@
 
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 
+#include <algorithm>
 #include <map>
 #include <set>
 #include <unordered_set>
@@ -128,7 +129,7 @@ void AttachLIPFilters::attachLIPFilters(
           lip_filter_configuration_->addBuildInfo(
               P::SingleIdentityHashFilterBuildInfo::Create(
                   pair.second->source_attribute,
-                  pair.second->estimated_cardinality * 8),
+                  std::max(64uL, pair.second->estimated_cardinality * 8u)),
               pair.second->source);
           lip_filter_configuration_->addProbeInfo(
               P::LIPFilterProbeInfo::Create(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/669a74c7/utility/lip_filter/SingleIdentityHashFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/SingleIdentityHashFilter.hpp b/utility/lip_filter/SingleIdentityHashFilter.hpp
index 2823818..5c0e8a2 100644
--- a/utility/lip_filter/SingleIdentityHashFilter.hpp
+++ b/utility/lip_filter/SingleIdentityHashFilter.hpp
@@ -66,7 +66,7 @@ class SingleIdentityHashFilter : public LIPFilter {
       : LIPFilter(LIPFilterType::kSingleIdentityHashFilter),
         filter_cardinality_(filter_cardinality),
         bit_array_(GetByteSize(filter_cardinality)) {
-    DCHECK_GE(filter_cardinality, 0u);
+    DCHECK_GE(filter_cardinality, 1u);
     std::memset(bit_array_.data(),
                 0x0,
                 sizeof(std::atomic<std::uint8_t>) * GetByteSize(filter_cardinality));


[10/50] [abbrv] incubator-quickstep git commit: Add KEYS file for release

Posted by ji...@apache.org.
Add KEYS file for release


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

Branch: refs/heads/LIP-time-decomposition
Commit: 50f3ba65fbd19d2dc07f704a36c4320dd7fdba23
Parents: 4ba819c
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Thu Feb 2 18:12:04 2017 -0600
Committer: Jignesh Patel <jm...@hotmail.com>
Committed: Thu Feb 2 18:12:04 2017 -0600

----------------------------------------------------------------------
 KEYS | 51 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/50f3ba65/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
new file mode 100644
index 0000000..ce3964f
--- /dev/null
+++ b/KEYS
@@ -0,0 +1,51 @@
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v1
+
+mQINBFiTyOUBEAC6iCMXSfoQPf29eall0puYj13vMESHG2lo8/wQXJPxarv0zUd0
+yiRQDwjzU9lqLaft8nIx38QrLzGxAr/HNPVMr2Ng7NIVcgadIjtumNvIt1ML1dMO
++c/k41xk629Kz+HQ+OiWg0t76EDv2Kf1EZ5Ai5Uay7wxOKjRU3QXl5QghvUMwgAo
+UlqShqNEyHyPOaZKa7EzBk3kBPjppuHs/3mWFXTISPHFKdNux0yroW/43tS7LmuC
+YNaM5+NZlzzNOqDgtcDR7klpKPvnFyuDqlZr3x3MBUDBOgBgovhVJqYCCihCkofK
+C1Dw5RhJE0sswhEcl9eoC+ANeSrQPUTabyb+Duud/UJ7cCAcH1HG26QQUpMQj7Fl
+NzyyZONbOuFQ2Nxc1lmua2/D5aSyQ20wrnxQ//2P1Da4PU/EbW0AR+VAUJM4c1do
+5b7Sy6kSK9SVhEJAavsNeOuldmUi9OPrxGD9PUD4OUmMxn/y+LDmEXvSTOB7+Wg+
+vSZOViwHVrAqrykZnLDKIvv7q+kzXaxQxFFtE6D5qjrdcdzw+2Cf+RZafSticIWi
+pPUJxn2ua9h94O9aYLOBp/bGkeb2f4eFW87RS6rBIWofIFnLIR3Kq64H1cKHVcXe
+gJcYB6usczpNDPgeoQVFAOYWpLQ6x3WRA4T5i8TLWohoqa2j5gjERgMOtQARAQAB
+tCJKaWduZXNoIFBhdGVsIDxqaWduZXNoQGFwYWNoZS5vcmc+iQI3BBMBCgAhBQJY
+k8jlAhsDBQsJCAcDBRUKCQgLBRYCAwEAAh4BAheAAAoJENt3gMCjwp4SVoMQAJM6
+z0TIdH7dNp7aF1EtksuekwzFO1dX2zz7qxF7O1a5ckCJppmFc0H1OjhHm0fG8Who
+tDiGue5NdyjcdXstmZoi+Xk7rMVIXSu+CYvoSDr3LKrKLJzSqWFmsd3CtwHl5vWD
+8MQTgmmdYJh+2lq5sAPJq4EUzAotdMgSLcv6+u/alR71WtzpKwuhOwfGLBe24w5A
+1pDbqso8rgX78Ga9q6Kqah/BM7YqPmxfgtazecl781KsaE6ODPW20XxrvvMq99+3
+LQ3NWpibZ6ed7T7ak2CEDRcnYayNQhvHWrygjO/JOE8xQ0BsBQ1hNmSBLkAVJa+L
+BQ2g1bR3bT7dQNr9V+fQPtZoA/ZNxttk+Nu+NZ1ZQ9jEyrkzpZmZlnBephQCNGeK
+5cfHNLJYwBmPDRZ6/UYwLj+OsfJAAAJHNbnV2d9pj5tp49EJ9UIJ4p71SlNHhM9H
+O1JXKJOD9jpomNeRlDTg7jh5h3LtBR3rtFX+kCo5Ktzoe4dTQ5Ei7qMpQ/ygrT7+
+QqUFZ3jvr1o5OzLUuDg0qBR2C0wPAJN+y3aqdU6Aw0o40UKsme687DhUPHGrZJ26
+YXYJ71iUg0G7VeEh75QylNkqbpUPMvMm7rxe1icfbxZu4sBdQlJj5G1DntMLeACB
+6UVgRwE5kn+V/59wL3GNg/CwT6cbjPZq4+tH0QkxuQINBFiTyOUBEAC6AaqV3ijL
+N9HJUobrkJZ6/MBEYkh37+XFF8swzW4RkGz9lTnim/FN341xfwmLQEjTfvX9Bd0q
+y2QyCSnP96C5i8cV8DNOK57JXipaWijFBBGOHOwDTJx2uxJJIWCtVJowievRBPW3
+T7r2e6WHepYtlDMjkJltwycRP8MsfMx1lcO9IACYakbZZAy0cqWgxdtTkucitcHO
+8/nJxSrIzlBnU0ktCH5/tvu/XN+TIEOHKgPMEldByeQ/m53CTU9X94YnUJ0DoqqG
+pCqO3LHV3iMONlfnBhpQfOD/Mzv2B+SKu4+LK1quUtfA0U089WrcrjBbF86gTsTj
+GLhzqX+/xb8Vf+/Jbx72uPNW0+TGqO0TzAujGL/wMZOPiBvVC17HI9qd8GC0pXe4
+y+pMiaCFcW0kDBLf69oaICrRVXOR75NyCyR9s0Dac+/DwiU3R3nafM6E9j7PXuNz
+PXmwlYoKLBwmOYOYN8j5fbaZHsZeDIoGkLMPpG1mLG82w9ONMwAt8BZPVwURa1SU
+Q0sv0QoO/QOYuk0QY7k8t6+q82oKIAcyqaLWx3wfCXdwhAF9jGVdShH3pVD2BiYC
+vRteQRQjy5XdSwjkASCNPMnIEpWWI7XADG0vBimdvHwLHXgp/eK6XtsqpAFmU0JJ
+WWy3AcWznUZnxKV1UgfmJkHmbD8jRMZndwARAQABiQIfBBgBCgAJBQJYk8jlAhsM
+AAoJENt3gMCjwp4SC/UP/1FAmrdH9hSAyTkJgU23iK+tZ7n4+C61B7pYS7J2r2Un
+tAJwAD/NjGUawfH6D32P97AMm/7Zls9K8PiMrHcLVRYbzR9327By/xTuq4voPg88
+/9u6gt57KbYrvQLTmGjsdpEkdAdGrKOuppuoOW5gge20pulSAO23JooNnyo6hQR4
+lnN02oGrGlRR6A6bVHfeXDtztr+NMjZ/32w8ZjeYppi96lZhJzrJhozchb7rBPEi
+I1Dkgb1IApfCHxvSXNlF3kyiDANwksCTE5otREE2/4hLu+KSs00OPYhJi/2/e69A
+CO01C5iVRpCs9/ofa0Rm+BbWVLWBcsZMR3TDJ/zHlf791FnsQJxFsf3uRmA+BzwK
+KSbcqUt0yc9/nuC4O1I/Ew00Gz5Vrk1Gaw54XXDBsJJvhnanw2E8qc1l9Cu5c/Ly
+AX5WxwPs31QMFNVvLVZ8qyqS5H6Z5kq1gLIjLYSLdnvDW6U6sDn9DOEqtFl72Y+3
+v2kCW3/ARGls2/G2wPPo9iWlYNkoF61o1rquvG1IUN8+mXCdjeAz3NvtDDPotxp9
+iYXZdNoVw3RZC2XRQB+as9wYnz/Ziqbrrw58/E5FdmC8U8+Fa/0lTUh6VsPjpu5u
+E7agqOm2ReVbNLPyHa2oGftKu0Cwyghbys5xNxqbNPQnFR9N9Soi+0n4IGCZ/tj5
+=qv6a
+-----END PGP PUBLIC KEY BLOCK-----


[42/50] [abbrv] incubator-quickstep git commit: Minor refactored QueryManagerDistributed.

Posted by ji...@apache.org.
Minor refactored QueryManagerDistributed.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 8229994ff813b990830a48617d64362aa4c20c63
Parents: 1cfc1c4
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 9 14:21:28 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 14:21:28 2017 -0800

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                |  1 -
 query_execution/PolicyEnforcerDistributed.cpp | 12 ++++++++++--
 query_execution/QueryManagerDistributed.cpp   | 17 ++++++-----------
 query_execution/QueryManagerDistributed.hpp   | 15 ++++++++++-----
 4 files changed, 26 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8229994f/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index e26bde0..5ad6999 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -257,7 +257,6 @@ if (ENABLE_DISTRIBUTED)
                         quickstep_queryexecution_QueryExecutionTypedefs
                         quickstep_queryexecution_QueryExecutionUtil
                         quickstep_queryexecution_QueryManagerBase
-                        quickstep_queryexecution_ShiftbossDirectory
                         quickstep_queryexecution_WorkOrderProtosContainer
                         quickstep_relationaloperators_RelationalOperator
                         quickstep_relationaloperators_WorkOrder_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8229994f/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index ef5abb0..12d2037 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -131,9 +131,17 @@ bool PolicyEnforcerDistributed::admitQuery(QueryHandle *query_handle) {
   // initializes.
   initiateQueryInShiftboss(query_handle);
 
+  const std::size_t num_shiftbosses = shiftboss_directory_->size();
+
+  tmb::Address shiftboss_addresses;
+  for (std::size_t i = 0; i < num_shiftbosses; ++i) {
+    shiftboss_addresses.AddRecipient(shiftboss_directory_->getClientId(i));
+  }
+
   // Query with the same ID not present, ok to admit.
-  admitted_queries_[query_id].reset(
-      new QueryManagerDistributed(query_handle, shiftboss_directory_, foreman_client_id_, bus_));
+  admitted_queries_.emplace(query_id,
+                            std::make_unique<QueryManagerDistributed>(
+                                query_handle, foreman_client_id_, num_shiftbosses, move(shiftboss_addresses), bus_));
   return true;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8229994f/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 6ac96ab..174c490 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -30,7 +30,6 @@
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryExecutionUtil.hpp"
-#include "query_execution/ShiftbossDirectory.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.pb.h"
@@ -52,12 +51,14 @@ using std::vector;
 namespace quickstep {
 
 QueryManagerDistributed::QueryManagerDistributed(QueryHandle *query_handle,
-                                                 const ShiftbossDirectory *shiftboss_directory,
                                                  const tmb::client_id foreman_client_id,
+                                                 const std::size_t num_shiftbosses,
+                                                 tmb::Address &&shiftboss_addresses,
                                                  tmb::MessageBus *bus)
     : QueryManagerBase(query_handle),
-      shiftboss_directory_(shiftboss_directory),
       foreman_client_id_(foreman_client_id),
+      num_shiftbosses_(num_shiftbosses),
+      shiftboss_addresses_(move(shiftboss_addresses)),
       bus_(bus),
       normal_workorder_protos_container_(
           new WorkOrderProtosContainer(num_operators_in_dag_)) {
@@ -142,7 +143,7 @@ void QueryManagerDistributed::processInitiateRebuildResponseMessage(const dag_no
                                                                     const std::size_t shiftboss_index) {
   query_exec_state_->updateRebuildStatus(op_index, num_rebuild_work_orders, shiftboss_index);
 
-  if (!query_exec_state_->hasRebuildFinished(op_index, shiftboss_directory_->size())) {
+  if (!query_exec_state_->hasRebuildFinished(op_index, num_shiftbosses_)) {
     // Wait for the rebuild work orders to finish.
     return;
   }
@@ -181,16 +182,10 @@ bool QueryManagerDistributed::initiateRebuild(const dag_node_index index) {
                            kInitiateRebuildMessage);
   free(proto_bytes);
 
-  // TODO(quickstep-team): Dynamically scale-up/down Shiftbosses.
-  tmb::Address shiftboss_addresses;
-  for (std::size_t i = 0; i < shiftboss_directory_->size(); ++i) {
-    shiftboss_addresses.AddRecipient(shiftboss_directory_->getClientId(i));
-  }
-
   DLOG(INFO) << "ForemanDistributed sent InitiateRebuildMessage (typed '" << kInitiateRebuildMessage
              << "') to all Shiftbosses";
   QueryExecutionUtil::BroadcastMessage(foreman_client_id_,
-                                       shiftboss_addresses,
+                                       shiftboss_addresses_,
                                        move(tagged_msg),
                                        bus_);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8229994f/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 631b15a..759fa70 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -32,6 +32,7 @@
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "utility/Macros.hpp"
 
+#include "tmb/address.h"
 #include "tmb/id_typedefs.h"
 
 namespace tmb { class MessageBus; }
@@ -39,7 +40,6 @@ namespace tmb { class MessageBus; }
 namespace quickstep {
 
 class QueryHandle;
-class ShiftbossDirectory;
 
 namespace serialization { class WorkOrderMessage; }
 
@@ -57,13 +57,15 @@ class QueryManagerDistributed final : public QueryManagerBase {
    * @brief Constructor.
    *
    * @param query_handle The QueryHandle object for this query.
-   * @param shiftboss_directory The ShiftbossDirectory to use.
    * @param foreman_client_id The TMB client ID of the foreman thread.
+   * @param num_shiftbosses The number of Shiftbosses for rebuild.
+   * @param shiftboss_addresses The TMB Address of Shiftbosses for rebuild.
    * @param bus The TMB used for communication.
    **/
   QueryManagerDistributed(QueryHandle *query_handle,
-                          const ShiftbossDirectory *shiftboss_directory,
                           const tmb::client_id foreman_client_id,
+                          const std::size_t num_shiftbosses,
+                          tmb::Address &&shiftboss_addresses,
                           tmb::MessageBus *bus);
 
   ~QueryManagerDistributed() override {}
@@ -153,9 +155,12 @@ class QueryManagerDistributed final : public QueryManagerBase {
            (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
   }
 
-  const ShiftbossDirectory *shiftboss_directory_;
-
   const tmb::client_id foreman_client_id_;
+
+  // TODO(quickstep-team): deal with Shiftboss failure.
+  const std::size_t num_shiftbosses_;
+  const tmb::Address shiftboss_addresses_;
+
   tmb::MessageBus *bus_;
 
   std::unique_ptr<WorkOrderProtosContainer> normal_workorder_protos_container_;


[14/50] [abbrv] incubator-quickstep git commit: Fixed CopyFrom bug in the distributed version.

Posted by ji...@apache.org.
Fixed CopyFrom bug in 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/61391caa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/61391caa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/61391caa

Branch: refs/heads/LIP-time-decomposition
Commit: 61391caac81201383fa7ea03b1321be50194b525
Parents: 929e5f1
Author: Zuyu Zhang <zu...@apache.org>
Authored: Fri Feb 3 00:09:37 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Feb 3 00:09:37 2017 -0800

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61391caa/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index b73de39..6918313 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -988,6 +988,11 @@ void ExecutionGenerator::convertCopyFrom(
   // CopyFrom is converted to a TextScan and a SaveBlocks.
 
   const CatalogRelation *output_relation = physical_plan->catalog_relation();
+  const relation_id output_rel_id = output_relation->getID();
+
+#ifdef QUICKSTEP_DISTRIBUTED
+  referenced_relation_ids_.insert(output_rel_id);
+#endif
 
   // Create InsertDestination proto.
   const QueryContext::insert_destination_id insert_destination_index =
@@ -995,7 +1000,7 @@ void ExecutionGenerator::convertCopyFrom(
   S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
 
   insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
-  insert_destination_proto->set_relation_id(output_relation->getID());
+  insert_destination_proto->set_relation_id(output_rel_id);
   insert_destination_proto->mutable_layout()->MergeFrom(
       output_relation->getDefaultStorageBlockLayout().getDescription());
 
@@ -1016,7 +1021,7 @@ void ExecutionGenerator::convertCopyFrom(
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
   CatalogRelation *mutable_output_relation =
-      catalog_database_->getRelationByIdMutable(output_relation->getID());
+      catalog_database_->getRelationByIdMutable(output_rel_id);
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
       execution_plan_->addRelationalOperator(
           new SaveBlocksOperator(query_handle_->query_id(), mutable_output_relation));


[15/50] [abbrv] incubator-quickstep git commit: Extract getFilesize as a method.

Posted by ji...@apache.org.
Extract getFilesize as a method.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 43a9b3981146ffaa6da125faf7bc0fa2a918834d
Parents: 61391ca
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sat Feb 4 16:08:02 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Sat Feb 4 16:08:02 2017 -0800

----------------------------------------------------------------------
 relational_operators/TextScanOperator.cpp | 25 ++++++++++++++++---------
 1 file changed, 16 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/43a9b398/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index aa734d3..6650319 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -78,6 +78,20 @@ static bool ValidateTextScanTextSegmentSize(const char *flagname,
 static const volatile bool text_scan_text_segment_size_dummy = gflags::RegisterFlagValidator(
     &FLAGS_textscan_text_segment_size, &ValidateTextScanTextSegmentSize);
 
+namespace {
+
+size_t getFileSize(const string &file_name) {
+  // Use standard C libary to retrieve the file size.
+  FILE *fp = std::fopen(file_name.c_str(), "rb");
+  std::fseek(fp, 0, SEEK_END);
+  const std::size_t file_size = std::ftell(fp);
+  std::fclose(fp);
+
+  return file_size;
+}
+
+}  // namespace
+
 bool TextScanOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,
@@ -105,10 +119,7 @@ bool TextScanOperator::getAllWorkOrders(
           << "File " << file << " is not readable due to permission issues.";
 #endif  // QUICKSTEP_HAVE_UNISTD
 
-      FILE *fp = std::fopen(file.c_str(), "rb");
-      std::fseek(fp, 0, SEEK_END);
-      const std::size_t file_size = std::ftell(fp);
-      std::fclose(fp);
+      const std::size_t file_size = getFileSize(file);
 
       std::size_t text_offset = 0;
       for (size_t num_full_segments = file_size / FLAGS_textscan_text_segment_size;
@@ -148,11 +159,7 @@ bool TextScanOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container
   const std::vector<std::string> files = utility::file::GlobExpand(file_pattern_);
   if (blocking_dependencies_met_ && !work_generated_) {
     for (const string &file : files) {
-      // Use standard C libary to retrieve the file size.
-      FILE *fp = std::fopen(file.c_str(), "rb");
-      std::fseek(fp, 0, SEEK_END);
-      const std::size_t file_size = std::ftell(fp);
-      std::fclose(fp);
+      const std::size_t file_size = getFileSize(file);
 
       size_t text_offset = 0;
       for (size_t num_full_segments = file_size / FLAGS_textscan_text_segment_size;


[23/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/FastHashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTableFactory.hpp b/storage/FastHashTableFactory.hpp
deleted file mode 100644
index 682cc2a..0000000
--- a/storage/FastHashTableFactory.hpp
+++ /dev/null
@@ -1,224 +0,0 @@
-/**
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *
- *   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_STORAGE_FAST_HASH_TABLE_FACTORY_HPP_
-#define QUICKSTEP_STORAGE_FAST_HASH_TABLE_FACTORY_HPP_
-
-#include <cstddef>
-#include <string>
-#include <vector>
-
-#include "storage/HashTable.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
-#include "storage/HashTableFactory.hpp"
-#include "storage/HashTable.pb.h"
-#include "storage/LinearOpenAddressingHashTable.hpp"
-#include "storage/SeparateChainingHashTable.hpp"
-#include "storage/FastSeparateChainingHashTable.hpp"
-#include "storage/SimpleScalarSeparateChainingHashTable.hpp"
-#include "storage/TupleReference.hpp"
-#include "types/TypeFactory.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class StorageManager;
-class Type;
-
-/** \addtogroup Storage
- *  @{
- */
-
-/**
- * @brief Templated all-static factory class that makes it easier to
- *        instantiate HashTables with the particular HashTable implementation
- *        chosen at runtime. All template parameters are exactly the same as
- *        those of HashTable.
- **/
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-class FastHashTableFactory {
- public:
-  /**
-   * @brief Create a new resizable HashTable, with the type selected by
-   *        hash_table_type. Other parameters are forwarded to the HashTable's
-   *        constructor.
-   *
-   * @param hash_table_type The specific HashTable implementation that should
-   *        be used.
-   * @param key_types A vector of one or more types (>1 indicates a composite
-   *        key). Forwarded as-is to the HashTable's constructor.
-   * @param num_entries The estimated number of entries the HashTable will
-   *        hold. Forwarded as-is to the HashTable's constructor.
-   * @param payload_sizes The sizes in bytes for the AggregationStates for the
-   *        respective AggregationHandles.
-   * @param handles The AggregationHandles used in this HashTable.
-   * @param storage_manager The StorageManager to use (a StorageBlob will be
-   *        allocated to hold the HashTable's contents). Forwarded as-is to the
-   *        HashTable's constructor.
-   * @return A new resizable HashTable.
-   **/
-  static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
-      CreateResizable(const HashTableImplType hash_table_type,
-                      const std::vector<const Type*> &key_types,
-                      const std::size_t num_entries,
-                      const std::vector<std::size_t> &payload_sizes,
-                      const std::vector<AggregationHandle *> &handles,
-                      StorageManager *storage_manager) {
-    DCHECK(resizable);
-
-    switch (hash_table_type) {
-      case HashTableImplType::kSeparateChaining:
-        return new FastSeparateChainingHashTable<
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
-      default: {
-        LOG(FATAL) << "Unrecognized HashTableImplType in HashTableFactory::createResizable()\n";
-      }
-    }
-  }
-
-  /**
-   * @brief Create a new fixed-sized HashTable, with the type selected by
-   *        hash_table_type. Other parameters are forwarded to the HashTables's
-   *        constructor.
-   *
-   * @param hash_table_type The specific HashTable implementation that should
-   *        be used.
-   * @param key_types A vector of one or more types (>1 indicates a composite
-   *        key). Forwarded as-is to the HashTable's constructor.
-   * @param hash_table_memory A pointer to memory to use for the HashTable.
-   *        Forwarded as-is to the HashTable's constructor.
-   * @param hash_table_memory_size The size of hash_table_memory in bytes.
-   *        Forwarded as-is to the HashTable's constructor.
-   * @param new_hash_table If true, the HashTable is being constructed for the
-   *        first time and hash_table_memory will be cleared. If false, reload
-   *        a pre-existing HashTable. Forwarded as-is to the HashTable's
-   *        constructor.
-   * @param hash_table_memory_zeroed If new_hash_table is true, setting this to
-   *        true means that the HashTable will assume that hash_table_memory
-   *        has already been zeroed-out (any newly-allocated block or blob
-   *        memory from StorageManager is zeroed-out). If false, the HashTable
-   *        will explicitly zero-fill its memory as neccessary. This parameter
-   *        has no effect when new_hash_table is false. Forwarded as-is to the
-   *        HashTable's constructor.
-   * @return A new (or reloaded) fixed-size HashTable.
-   **/
-  static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
-      CreateFixedSize(const HashTableImplType hash_table_type,
-                      const std::vector<const Type*> &key_types,
-                      void *hash_table_memory,
-                      const std::size_t hash_table_memory_size,
-                      const bool new_hash_table,
-                      const bool hash_table_memory_zeroed) {
-    DCHECK(!resizable);
-
-    switch (hash_table_type) {
-      case HashTableImplType::kSeparateChaining:
-        return new SeparateChainingHashTable<
-            int,
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types,
-                                  hash_table_memory,
-                                  hash_table_memory_size,
-                                  new_hash_table,
-                                  hash_table_memory_zeroed);
-      default: {
-        LOG(FATAL) << "Unrecognized HashTableImplType\n";
-      }
-    }
-  }
-
-  /**
-   * @brief Check whether a serialization::HashTable describing a resizable
-   *        HashTable is fully-formed and all parts are valid.
-   *
-   * @param proto A serialized Protocol Buffer description of a HashTable,
-   *        originally generated by the optimizer.
-   * @return Whether proto is fully-formed and valid.
-   **/
-  static bool ProtoIsValid(const serialization::HashTable &proto) {
-    if (!proto.IsInitialized() ||
-        !serialization::HashTableImplType_IsValid(
-            proto.hash_table_impl_type())) {
-      return false;
-    }
-
-    for (int i = 0; i < proto.key_types_size(); ++i) {
-      if (!TypeFactory::ProtoIsValid(proto.key_types(i))) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  /**
-   * @brief Create a new resizable HashTable according to a protobuf
-   *        description.
-   *
-   * @param proto A protobuf description of a resizable HashTable.
-   * @param storage_manager The StorageManager to use (a StorageBlob will be
-   *        allocated to hold the HashTable's contents).
-   * @return A new resizable HashTable with parameters specified by proto.
-   **/
-  static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
-      CreateResizableFromProto(const serialization::HashTable &proto,
-                               StorageManager *storage_manager) {
-    DCHECK(ProtoIsValid(proto))
-        << "Attempted to create HashTable from invalid proto description:\n"
-        << proto.DebugString();
-
-    std::vector<const Type*> key_types;
-    for (int i = 0; i < proto.key_types_size(); ++i) {
-      key_types.emplace_back(&TypeFactory::ReconstructFromProto(proto.key_types(i)));
-    }
-
-    auto hash_table = CreateResizable(HashTableImplTypeFromProto(proto.hash_table_impl_type()),
-                                      key_types,
-                                      proto.estimated_num_entries(),
-                                      storage_manager);
-    return hash_table;
-  }
-
- private:
-  // Class is all-static and should not be instantiated.
-  FastHashTableFactory();
-
-  DISALLOW_COPY_AND_ASSIGN(FastHashTableFactory);
-};
-
-/**
- * @brief Convenient alias that provides a HashTableFactory whose only template
- *        parameter is the aggregate state type.
- **/
-using AggregationStateFastHashTableFactory
-    = FastHashTableFactory<true, false, true, false>;
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_HASH_TABLE_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
deleted file mode 100644
index 2435d45..0000000
--- a/storage/FastSeparateChainingHashTable.hpp
+++ /dev/null
@@ -1,1551 +0,0 @@
-/**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *
- *   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_STORAGE_FAST_SEPARATE_CHAINING_HASH_TABLE_HPP_
-#define QUICKSTEP_STORAGE_FAST_SEPARATE_CHAINING_HASH_TABLE_HPP_
-
-#include <algorithm>
-#include <atomic>
-#include <cstddef>
-#include <cstring>
-#include <limits>
-#include <memory>
-#include <utility>
-#include <vector>
-
-#include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableBase.hpp"
-#include "storage/HashTableKeyManager.hpp"
-#include "storage/StorageBlob.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageConstants.hpp"
-#include "storage/StorageManager.hpp"
-#include "threading/SpinSharedMutex.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/Alignment.hpp"
-#include "utility/Macros.hpp"
-#include "utility/PrimeNumber.hpp"
-
-namespace quickstep {
-
-/** \addtogroup Storage
- *  @{
- */
-
-/**
- * @brief A hash table implementation which uses separate chaining for buckets.
- **/
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-class FastSeparateChainingHashTable
-    : public FastHashTable<resizable,
-                           serializable,
-                           force_key_copy,
-                           allow_duplicate_keys> {
- public:
-  FastSeparateChainingHashTable(const std::vector<const Type *> &key_types,
-                                const std::size_t num_entries,
-                                const std::vector<std::size_t> &payload_sizes,
-                                const std::vector<AggregationHandle *> &handles,
-                                StorageManager *storage_manager);
-
-  ~FastSeparateChainingHashTable() override {
-    std::free(init_payload_);
-  }
-
-  void clear() override;
-
-  std::size_t numEntries() const override {
-    return header_->buckets_allocated.load(std::memory_order_relaxed);
-  }
-
-  const std::uint8_t* getSingle(const TypedValue &key) const override;
-  const std::uint8_t* getSingleCompositeKey(
-      const std::vector<TypedValue> &key) const override;
-  const std::uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key,
-                                            int index) const override;
-
-  void getAll(const TypedValue &key,
-              std::vector<const std::uint8_t *> *values) const override;
-  void getAllCompositeKey(
-      const std::vector<TypedValue> &key,
-      std::vector<const std::uint8_t *> *values) const override;
-
- protected:
-  HashTablePutResult putInternal(
-      const TypedValue &key,
-      const std::size_t variable_key_size,
-      const std::uint8_t &value,
-      HashTablePreallocationState *prealloc_state) override;
-
-  HashTablePutResult putCompositeKeyInternalFast(
-      const std::vector<TypedValue> &key,
-      const std::size_t variable_key_size,
-      const std::uint8_t *init_value_ptr,
-      HashTablePreallocationState *prealloc_state) override;
-
-  std::uint8_t* upsertInternalFast(const TypedValue &key,
-                                   const std::size_t variable_key_size,
-                                   const std::uint8_t *init_value_ptr) override;
-
-  std::uint8_t* upsertCompositeKeyInternalFast(
-      const std::vector<TypedValue> &key,
-      const std::uint8_t *init_value_ptr,
-      const std::size_t variable_key_size) override;
-
-  bool getNextEntry(TypedValue *key,
-                    const std::uint8_t **value,
-                    std::size_t *entry_num) const override;
-  bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                                const std::uint8_t **value,
-                                std::size_t *entry_num) const override;
-
-  bool getNextEntryForKey(const TypedValue &key,
-                          const std::size_t hash_code,
-                          const std::uint8_t **value,
-                          std::size_t *entry_num) const override;
-  bool getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
-                                   const std::size_t hash_code,
-                                   const std::uint8_t **value,
-                                   std::size_t *entry_num) const override;
-
-  bool hasKey(const TypedValue &key) const override;
-  bool hasCompositeKey(const std::vector<TypedValue> &key) const override;
-
-  void resize(const std::size_t extra_buckets,
-              const std::size_t extra_variable_storage,
-              const std::size_t retry_num = 0) override;
-
-  bool preallocateForBulkInsert(
-      const std::size_t total_entries,
-      const std::size_t total_variable_key_size,
-      HashTablePreallocationState *prealloc_state) override;
-
-  void destroyPayload() override {
-    const std::size_t num_buckets =
-        header_->buckets_allocated.load(std::memory_order_relaxed);
-    void *bucket_ptr = static_cast<char *>(buckets_) + kValueOffset;
-    for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
-      for (std::size_t handle_id = 0; handle_id < num_handles_; ++handle_id) {
-        void *value_internal_ptr =
-            static_cast<char *>(bucket_ptr) + this->payload_offsets_[handle_id];
-        handles_[handle_id]->destroyPayload(static_cast<std::uint8_t *>(value_internal_ptr));
-      }
-      bucket_ptr = static_cast<char *>(bucket_ptr) + bucket_size_;
-    }
-  }
-
- private:
-  struct Header {
-    std::size_t num_slots;
-    std::size_t num_buckets;
-    alignas(kCacheLineBytes) std::atomic<std::size_t> buckets_allocated;
-    alignas(kCacheLineBytes)
-        std::atomic<std::size_t> variable_length_bytes_allocated;
-  };
-
-  std::uint8_t *init_payload_;
-  std::size_t kBucketAlignment;
-
-  // Value's offset in a bucket is the first alignof(ValueT) boundary after the
-  // next pointer and hash code.
-  std::size_t kValueOffset;
-
-  // Round bucket size up to a multiple of kBucketAlignment.
-  constexpr std::size_t ComputeBucketSize(const std::size_t fixed_key_size) {
-    return (((kValueOffset + this->total_payload_size_ + fixed_key_size - 1) /
-             kBucketAlignment) +
-            1) *
-           kBucketAlignment;
-  }
-
-  // Attempt to find an empty bucket to insert 'hash_code' into, starting after
-  // '*bucket' in the chain (or, if '*bucket' is NULL, starting from the slot
-  // array). Returns true and stores SIZE_T_MAX in '*pending_chain_ptr' if an
-  // empty bucket is found. Returns false if 'allow_duplicate_keys' is false
-  // and a hash collision is found (caller should then check whether there is a
-  // genuine key collision or the hash collision is spurious). Returns false
-  // and sets '*bucket' to NULL if there are no more empty buckets in the hash
-  // table. If 'variable_key_allocation_required' is nonzero, this method will
-  // attempt to allocate storage for a variable-length key BEFORE allocating a
-  // bucket, so that no bucket number below 'header_->num_buckets' is ever
-  // deallocated after being allocated.
-  inline bool locateBucketForInsertion(
-      const std::size_t hash_code,
-      const std::size_t variable_key_allocation_required,
-      void **bucket,
-      std::atomic<std::size_t> **pending_chain_ptr,
-      std::size_t *pending_chain_ptr_finish_value,
-      HashTablePreallocationState *prealloc_state);
-
-  // Write a scalar 'key' and its 'hash_code' into the '*bucket', which was
-  // found by locateBucketForInsertion(). Assumes that storage for a
-  // variable-length key copy (if any) was already allocated by a successful
-  // call to allocateVariableLengthKeyStorage().
-  inline void writeScalarKeyToBucket(
-      const TypedValue &key,
-      const std::size_t hash_code,
-      void *bucket,
-      HashTablePreallocationState *prealloc_state);
-
-  // Write a composite 'key' and its 'hash_code' into the '*bucket', which was
-  // found by locateBucketForInsertion(). Assumes that storage for
-  // variable-length key copies (if any) was already allocated by a successful
-  // call to allocateVariableLengthKeyStorage().
-  inline void writeCompositeKeyToBucket(
-      const std::vector<TypedValue> &key,
-      const std::size_t hash_code,
-      void *bucket,
-      HashTablePreallocationState *prealloc_state);
-
-  // Determine whether it is actually necessary to resize this hash table.
-  // Checks that there is at least one unallocated bucket, and that there is
-  // at least 'extra_variable_storage' bytes of variable-length storage free.
-  bool isFull(const std::size_t extra_variable_storage) const;
-
-  const std::vector<AggregationHandle *> &handles_;
-  const std::size_t num_handles_;
-
-  // Helper object to manage key storage.
-  HashTableKeyManager<serializable, force_key_copy> key_manager_;
-
-  // In-memory structure is as follows:
-  //   - SeparateChainingHashTable::Header
-  //   - Array of slots, interpreted as follows:
-  //       - 0 = Points to nothing (empty)
-  //       - SIZE_T_MAX = Pending (some thread is starting a chain from this
-  //         slot and will overwrite it soon)
-  //       - Anything else = The number of the first bucket in the chain for
-  //         this slot PLUS ONE (i.e. subtract one to get the actual bucket
-  //         number).
-  //   - Array of buckets, each of which is:
-  //       - atomic size_t "next" pointer, interpreted the same as slots above.
-  //       - size_t hash value
-  //       - possibly some unused bytes as needed so that ValueT's alignment
-  //         requirement is met
-  //       - ValueT value slot
-  //       - fixed-length key storage (which may include pointers to external
-  //         memory or offsets of variable length keys stored within this hash
-  //         table)
-  //       - possibly some additional unused bytes so that bucket size is a
-  //         multiple of both alignof(std::atomic<std::size_t>) and
-  //         alignof(ValueT)
-  //   - Variable-length key storage region (referenced by offsets stored in
-  //     fixed-length keys).
-  Header *header_;
-
-  std::atomic<std::size_t> *slots_;
-  void *buckets_;
-  const std::size_t bucket_size_;
-
-  DISALLOW_COPY_AND_ASSIGN(FastSeparateChainingHashTable);
-};
-
-/** @} */
-
-// ----------------------------------------------------------------------------
-// Implementations of template class methods follow.
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-FastSeparateChainingHashTable<resizable,
-                              serializable,
-                              force_key_copy,
-                              allow_duplicate_keys>::
-    FastSeparateChainingHashTable(
-        const std::vector<const Type *> &key_types,
-        const std::size_t num_entries,
-        const std::vector<std::size_t> &payload_sizes,
-        const std::vector<AggregationHandle *> &handles,
-        StorageManager *storage_manager)
-    : FastHashTable<resizable,
-                    serializable,
-                    force_key_copy,
-                    allow_duplicate_keys>(key_types,
-                                          num_entries,
-                                          handles,
-                                          payload_sizes,
-                                          storage_manager,
-                                          false,
-                                          false,
-                                          true),
-      kBucketAlignment(alignof(std::atomic<std::size_t>)),
-      kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
-      handles_(handles),
-      num_handles_(handles.size()),
-      key_manager_(this->key_types_, kValueOffset + this->total_payload_size_),
-      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
-  init_payload_ =
-      static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
-  DCHECK(init_payload_ != nullptr);
-  int k = 0;
-  for (auto handle : this->handles_) {
-    handle->initPayload(init_payload_ + this->payload_offsets_[k]);
-    k++;
-  }
-  // Bucket size always rounds up to the alignment requirement of the atomic
-  // size_t "next" pointer at the front or a ValueT, whichever is larger.
-  //
-  // Give base HashTable information about what key components are stored
-  // inline from 'key_manager_'.
-  this->setKeyInline(key_manager_.getKeyInline());
-
-  // Pick out a prime number of slots and calculate storage requirements.
-  std::size_t num_slots_tmp =
-      get_next_prime_number(num_entries * kHashTableLoadFactor);
-  std::size_t required_memory =
-      sizeof(Header) + num_slots_tmp * sizeof(std::atomic<std::size_t>) +
-      (num_slots_tmp / kHashTableLoadFactor) *
-          (bucket_size_ + key_manager_.getEstimatedVariableKeySize());
-  std::size_t num_storage_slots =
-      this->storage_manager_->SlotsNeededForBytes(required_memory);
-  if (num_storage_slots == 0) {
-    FATAL_ERROR(
-        "Storage requirement for SeparateChainingHashTable "
-        "exceeds maximum allocation size.");
-  }
-
-  // Get a StorageBlob to hold the hash table.
-  const block_id blob_id =
-      this->storage_manager_->createBlob(num_storage_slots);
-  this->blob_ = this->storage_manager_->getBlobMutable(blob_id);
-
-  void *aligned_memory_start = this->blob_->getMemoryMutable();
-  std::size_t available_memory = num_storage_slots * kSlotSizeBytes;
-  if (align(alignof(Header),
-            sizeof(Header),
-            aligned_memory_start,
-            available_memory) == nullptr) {
-    // With current values from StorageConstants.hpp, this should be
-    // impossible. A blob is at least 1 MB, while a Header has alignment
-    // requirement of just kCacheLineBytes (64 bytes).
-    FATAL_ERROR(
-        "StorageBlob used to hold resizable "
-        "SeparateChainingHashTable is too small to meet alignment "
-        "requirements of SeparateChainingHashTable::Header.");
-  } else if (aligned_memory_start != this->blob_->getMemoryMutable()) {
-    // This should also be impossible, since the StorageManager allocates slots
-    // aligned to kCacheLineBytes.
-    DEV_WARNING("StorageBlob memory adjusted by "
-                << (num_storage_slots * kSlotSizeBytes - available_memory)
-                << " bytes to meet alignment requirement for "
-                << "SeparateChainingHashTable::Header.");
-  }
-
-  // Locate the header.
-  header_ = static_cast<Header *>(aligned_memory_start);
-  aligned_memory_start =
-      static_cast<char *>(aligned_memory_start) + sizeof(Header);
-  available_memory -= sizeof(Header);
-
-  // Recompute the number of slots & buckets using the actual available memory.
-  // Most likely, we got some extra free bucket space due to "rounding up" to
-  // the storage blob's size. It's also possible (though very unlikely) that we
-  // will wind up with fewer buckets than we initially wanted because of screwy
-  // alignment requirements for ValueT.
-  std::size_t num_buckets_tmp =
-      available_memory /
-      (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) + bucket_size_ +
-       key_manager_.getEstimatedVariableKeySize());
-  num_slots_tmp =
-      get_previous_prime_number(num_buckets_tmp * kHashTableLoadFactor);
-  num_buckets_tmp = num_slots_tmp / kHashTableLoadFactor;
-  DEBUG_ASSERT(num_slots_tmp > 0);
-  DEBUG_ASSERT(num_buckets_tmp > 0);
-
-  // Locate the slot array.
-  slots_ = static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
-  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
-                         sizeof(std::atomic<std::size_t>) * num_slots_tmp;
-  available_memory -= sizeof(std::atomic<std::size_t>) * num_slots_tmp;
-
-  // Locate the buckets.
-  buckets_ = aligned_memory_start;
-  // Extra-paranoid: If ValueT has an alignment requirement greater than that
-  // of std::atomic<std::size_t>, we may need to adjust the start of the bucket
-  // array.
-  if (align(kBucketAlignment, bucket_size_, buckets_, available_memory) ==
-      nullptr) {
-    FATAL_ERROR(
-        "StorageBlob used to hold resizable "
-        "SeparateChainingHashTable is too small to meet "
-        "alignment requirements of buckets.");
-  } else if (buckets_ != aligned_memory_start) {
-    DEV_WARNING(
-        "Bucket array start position adjusted to meet alignment "
-        "requirement for SeparateChainingHashTable's value type.");
-    if (num_buckets_tmp * bucket_size_ > available_memory) {
-      --num_buckets_tmp;
-    }
-  }
-
-  // Fill in the header.
-  header_->num_slots = num_slots_tmp;
-  header_->num_buckets = num_buckets_tmp;
-  header_->buckets_allocated.store(0, std::memory_order_relaxed);
-  header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
-  available_memory -= bucket_size_ * (header_->num_buckets);
-
-  // Locate variable-length key storage region, and give it all the remaining
-  // bytes in the blob.
-  key_manager_.setVariableLengthStorageInfo(
-      static_cast<char *>(buckets_) + header_->num_buckets * bucket_size_,
-      available_memory,
-      &(header_->variable_length_bytes_allocated));
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable,
-                                   serializable,
-                                   force_key_copy,
-                                   allow_duplicate_keys>::clear() {
-  const std::size_t used_buckets =
-      header_->buckets_allocated.load(std::memory_order_relaxed);
-  // Destroy existing values, if necessary.
-  destroyPayload();
-
-  // Zero-out slot array.
-  std::memset(
-      slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
-
-  // Zero-out used buckets.
-  std::memset(buckets_, 0x0, used_buckets * bucket_size_);
-
-  header_->buckets_allocated.store(0, std::memory_order_relaxed);
-  header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
-  key_manager_.zeroNextVariableLengthKeyOffset();
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-const std::uint8_t* FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::getSingle(const TypedValue &key) const {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(
-      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  const std::size_t hash_code = key.getHash();
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-
-  // Reached the end of the chain and didn't find a match.
-  return nullptr;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-const std::uint8_t* FastSeparateChainingHashTable<resizable,
-                                                  serializable,
-                                                  force_key_copy,
-                                                  allow_duplicate_keys>::
-    getSingleCompositeKey(const std::vector<TypedValue> &key) const {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-
-  // Reached the end of the chain and didn't find a match.
-  return nullptr;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-const std::uint8_t* FastSeparateChainingHashTable<resizable,
-                                                  serializable,
-                                                  force_key_copy,
-                                                  allow_duplicate_keys>::
-    getSingleCompositeKey(const std::vector<TypedValue> &key, int index) const {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset) +
-             this->payload_offsets_[index];
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-
-  // Reached the end of the chain and didn't find a match.
-  return nullptr;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::getAll(const TypedValue &key,
-                                  std::vector<const std::uint8_t *> *values)
-    const {
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(
-      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  const std::size_t hash_code = key.getHash();
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      values->push_back(
-          reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset));
-      if (!allow_duplicate_keys) {
-        return;
-      }
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable,
-                                   serializable,
-                                   force_key_copy,
-                                   allow_duplicate_keys>::
-    getAllCompositeKey(const std::vector<TypedValue> &key,
-                       std::vector<const std::uint8_t *> *values) const {
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      values->push_back(
-          reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset));
-      if (!allow_duplicate_keys) {
-        return;
-      }
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-HashTablePutResult FastSeparateChainingHashTable<resizable,
-                                                 serializable,
-                                                 force_key_copy,
-                                                 allow_duplicate_keys>::
-    putInternal(const TypedValue &key,
-                const std::size_t variable_key_size,
-                const std::uint8_t &value,
-                HashTablePreallocationState *prealloc_state) {
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(
-      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  if (prealloc_state == nullptr) {
-    // Early check for a free bucket.
-    if (header_->buckets_allocated.load(std::memory_order_relaxed) >=
-        header_->num_buckets) {
-      return HashTablePutResult::kOutOfSpace;
-    }
-
-    // TODO(chasseur): If allow_duplicate_keys is true, avoid storing more than
-    // one copy of the same variable-length key.
-    if (!key_manager_.allocateVariableLengthKeyStorage(variable_key_size)) {
-      // Ran out of variable-length key storage space.
-      return HashTablePutResult::kOutOfSpace;
-    }
-  }
-
-  const std::size_t hash_code = key.getHash();
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 0,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 prealloc_state)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets. Deallocate any variable space that we were unable
-      // to use.
-      DEBUG_ASSERT(prealloc_state == nullptr);
-      key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
-      return HashTablePutResult::kOutOfSpace;
-    } else {
-      // Hash collision found, and duplicates aren't allowed.
-      DEBUG_ASSERT(!allow_duplicate_keys);
-      DEBUG_ASSERT(prealloc_state == nullptr);
-      if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-        // Duplicate key. Deallocate any variable storage space and return.
-        key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
-        return HashTablePutResult::kDuplicateKey;
-      }
-    }
-  }
-
-  // Write the key and hash.
-  writeScalarKeyToBucket(key, hash_code, bucket, prealloc_state);
-
-  // Store the value by using placement new with ValueT's copy constructor.
-  new (static_cast<char *>(bucket) + kValueOffset) std::uint8_t(value);
-
-  // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value,
-                           std::memory_order_release);
-
-  // We're all done.
-  return HashTablePutResult::kOK;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-HashTablePutResult FastSeparateChainingHashTable<resizable,
-                                                 serializable,
-                                                 force_key_copy,
-                                                 allow_duplicate_keys>::
-    putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                const std::size_t variable_key_size,
-                                const std::uint8_t *init_value_ptr,
-                                HashTablePreallocationState *prealloc_state) {
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  if (prealloc_state == nullptr) {
-    // Early check for a free bucket.
-    if (header_->buckets_allocated.load(std::memory_order_relaxed) >=
-        header_->num_buckets) {
-      return HashTablePutResult::kOutOfSpace;
-    }
-
-    // TODO(chasseur): If allow_duplicate_keys is true, avoid storing more than
-    // one copy of the same variable-length key.
-    if (!key_manager_.allocateVariableLengthKeyStorage(variable_key_size)) {
-      // Ran out of variable-length key storage space.
-      return HashTablePutResult::kOutOfSpace;
-    }
-  }
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 0,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 prealloc_state)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets. Deallocate any variable space that we were unable
-      // to use.
-      DEBUG_ASSERT(prealloc_state == nullptr);
-      key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
-      return HashTablePutResult::kOutOfSpace;
-    } else {
-      // Hash collision found, and duplicates aren't allowed.
-      DEBUG_ASSERT(!allow_duplicate_keys);
-      DEBUG_ASSERT(prealloc_state == nullptr);
-      if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-        // Duplicate key. Deallocate any variable storage space and return.
-        key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
-        return HashTablePutResult::kDuplicateKey;
-      }
-    }
-  }
-
-  // Write the key and hash.
-  writeCompositeKeyToBucket(key, hash_code, bucket, prealloc_state);
-
-  std::uint8_t *value = static_cast<std::uint8_t *>(bucket) + kValueOffset;
-  memcpy(value, init_value_ptr, this->total_payload_size_);
-  // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value,
-                           std::memory_order_release);
-
-  // We're all done.
-  return HashTablePutResult::kOK;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-std::uint8_t* FastSeparateChainingHashTable<resizable,
-                                            serializable,
-                                            force_key_copy,
-                                            allow_duplicate_keys>::
-    upsertInternalFast(const TypedValue &key,
-                       const std::size_t variable_key_size,
-                       const std::uint8_t *init_value_ptr) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(
-      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  if (variable_key_size > 0) {
-    // Don't allocate yet, since the key may already be present. However, we
-    // do check if either the allocated variable storage space OR the free
-    // space is big enough to hold the key (at least one must be true: either
-    // the key is already present and allocated, or we need to be able to
-    // allocate enough space for it).
-    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(
-        std::memory_order_relaxed);
-    if ((allocated_bytes < variable_key_size) &&
-        (allocated_bytes + variable_key_size >
-         key_manager_.getVariableLengthKeyStorageSize())) {
-      return nullptr;
-    }
-  }
-
-  const std::size_t hash_code = key.getHash();
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 variable_key_size,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 nullptr)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets or variable-key space.
-      return nullptr;
-    } else if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-      // Found an already-existing entry for this key.
-      return reinterpret_cast<std::uint8_t *>(static_cast<char *>(bucket) +
-                                              kValueOffset);
-    }
-  }
-
-  // We are now writing to an empty bucket.
-  // Write the key and hash.
-  writeScalarKeyToBucket(key, hash_code, bucket, nullptr);
-
-  // Copy the supplied 'initial_value' into place.
-  std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
-  if (init_value_ptr == nullptr) {
-    memcpy(value, init_payload_, this->total_payload_size_);
-  } else {
-    memcpy(value, init_value_ptr, this->total_payload_size_);
-  }
-
-  // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value,
-                           std::memory_order_release);
-
-  // Return the value.
-  return value;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-std::uint8_t* FastSeparateChainingHashTable<resizable,
-                                            serializable,
-                                            force_key_copy,
-                                            allow_duplicate_keys>::
-    upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                   const std::uint8_t *init_value_ptr,
-                                   const std::size_t variable_key_size) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  if (variable_key_size > 0) {
-    // Don't allocate yet, since the key may already be present. However, we
-    // do check if either the allocated variable storage space OR the free
-    // space is big enough to hold the key (at least one must be true: either
-    // the key is already present and allocated, or we need to be able to
-    // allocate enough space for it).
-    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(
-        std::memory_order_relaxed);
-    if ((allocated_bytes < variable_key_size) &&
-        (allocated_bytes + variable_key_size >
-         key_manager_.getVariableLengthKeyStorageSize())) {
-      return nullptr;
-    }
-  }
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 variable_key_size,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 nullptr)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets or variable-key space.
-      return nullptr;
-    } else if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Found an already-existing entry for this key.
-      return reinterpret_cast<std::uint8_t *>(static_cast<char *>(bucket) +
-                                              kValueOffset);
-    }
-  }
-
-  // We are now writing to an empty bucket.
-  // Write the key and hash.
-  writeCompositeKeyToBucket(key, hash_code, bucket, nullptr);
-
-  std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
-  if (init_value_ptr == nullptr) {
-    memcpy(value, init_payload_, this->total_payload_size_);
-  } else {
-    memcpy(value, init_value_ptr, this->total_payload_size_);
-  }
-
-  // Update the previous chaing pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value,
-                           std::memory_order_release);
-
-  // Return the value.
-  return value;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::getNextEntry(TypedValue *key,
-                                        const std::uint8_t **value,
-                                        std::size_t *entry_num) const {
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (*entry_num) * bucket_size_;
-    *key = key_manager_.getKeyComponentTyped(bucket, 0);
-    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
-    ++(*entry_num);
-    return true;
-  } else {
-    return false;
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable,
-                                   serializable,
-                                   force_key_copy,
-                                   allow_duplicate_keys>::
-    getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                             const std::uint8_t **value,
-                             std::size_t *entry_num) const {
-  if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (*entry_num) * bucket_size_;
-    for (std::vector<const Type *>::size_type key_idx = 0;
-         key_idx < this->key_types_.size();
-         ++key_idx) {
-      key->emplace_back(key_manager_.getKeyComponentTyped(bucket, key_idx));
-    }
-    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
-    ++(*entry_num);
-    return true;
-  } else {
-    return false;
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::getNextEntryForKey(const TypedValue &key,
-                                              const std::size_t hash_code,
-                                              const std::uint8_t **value,
-                                              std::size_t *entry_num) const {
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(
-      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  if (*entry_num == 0) {
-    *entry_num =
-        slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  } else if (*entry_num == std::numeric_limits<std::size_t>::max()) {
-    return false;
-  }
-
-  while (*entry_num != 0) {
-    DEBUG_ASSERT(*entry_num != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (*entry_num - 1) * bucket_size_;
-    *entry_num =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
-      if (*entry_num == 0) {
-        // If this is the last bucket in the chain, prevent the next call from
-        // starting over again.
-        *entry_num = std::numeric_limits<std::size_t>::max();
-      }
-      return true;
-    }
-  }
-
-  // Reached the end of the chain.
-  return false;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable,
-                                   serializable,
-                                   force_key_copy,
-                                   allow_duplicate_keys>::
-    getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
-                                const std::size_t hash_code,
-                                const std::uint8_t **value,
-                                std::size_t *entry_num) const {
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  if (*entry_num == 0) {
-    *entry_num =
-        slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  } else if (*entry_num == std::numeric_limits<std::size_t>::max()) {
-    return false;
-  }
-
-  while (*entry_num != 0) {
-    DEBUG_ASSERT(*entry_num != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (*entry_num - 1) * bucket_size_;
-    *entry_num =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Match located.
-      *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
-      if (*entry_num == 0) {
-        // If this is the last bucket in the chain, prevent the next call from
-        // starting over again.
-        *entry_num = std::numeric_limits<std::size_t>::max();
-      }
-      return true;
-    }
-  }
-
-  // Reached the end of the chain.
-  return false;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::hasKey(const TypedValue &key) const {
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(
-      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  const std::size_t hash_code = key.getHash();
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-      // Find a match.
-      return true;
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-  return false;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::hasCompositeKey(const std::vector<TypedValue> &key)
-    const {
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref =
-      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
-  while (bucket_ref != 0) {
-    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket =
-        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
-        bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) &&
-        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Find a match.
-      return true;
-    }
-    bucket_ref =
-        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
-            std::memory_order_relaxed);
-  }
-  return false;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::resize(const std::size_t extra_buckets,
-                                  const std::size_t extra_variable_storage,
-                                  const std::size_t retry_num) {
-  DEBUG_ASSERT(resizable);
-
-  // A retry should never be necessary with this implementation of HashTable.
-  // Separate chaining ensures that any resized hash table with more buckets
-  // than the original table will be able to hold more entries than the
-  // original.
-  DEBUG_ASSERT(retry_num == 0);
-
-  SpinSharedMutexExclusiveLock<true> write_lock(this->resize_shared_mutex_);
-
-  // Recheck whether the hash table is still full. Note that multiple threads
-  // might wait to rebuild this hash table simultaneously. Only the first one
-  // should do the rebuild.
-  if (!isFull(extra_variable_storage)) {
-    return;
-  }
-
-  // Approximately double the number of buckets and slots.
-  //
-  // TODO(chasseur): It may be worth it to more than double the number of
-  // buckets here so that we can maintain a good, sparse fill factor for a
-  // longer time as more values are inserted. Such behavior should take into
-  // account kHashTableLoadFactor.
-  std::size_t resized_num_slots = get_next_prime_number(
-      (header_->num_buckets + extra_buckets / 2) * kHashTableLoadFactor * 2);
-  std::size_t variable_storage_required =
-      (resized_num_slots / kHashTableLoadFactor) *
-      key_manager_.getEstimatedVariableKeySize();
-  const std::size_t original_variable_storage_used =
-      header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
-  // If this resize was triggered by a too-large variable-length key, bump up
-  // the variable-length storage requirement.
-  if ((extra_variable_storage > 0) &&
-      (extra_variable_storage + original_variable_storage_used >
-       key_manager_.getVariableLengthKeyStorageSize())) {
-    variable_storage_required += extra_variable_storage;
-  }
-
-  const std::size_t resized_memory_required =
-      sizeof(Header) + resized_num_slots * sizeof(std::atomic<std::size_t>) +
-      (resized_num_slots / kHashTableLoadFactor) * bucket_size_ +
-      variable_storage_required;
-  const std::size_t resized_storage_slots =
-      this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
-  if (resized_storage_slots == 0) {
-    FATAL_ERROR(
-        "Storage requirement for resized SeparateChainingHashTable "
-        "exceeds maximum allocation size.");
-  }
-
-  // Get a new StorageBlob to hold the resized hash table.
-  const block_id resized_blob_id =
-      this->storage_manager_->createBlob(resized_storage_slots);
-  MutableBlobReference resized_blob =
-      this->storage_manager_->getBlobMutable(resized_blob_id);
-
-  // Locate data structures inside the new StorageBlob.
-  void *aligned_memory_start = resized_blob->getMemoryMutable();
-  std::size_t available_memory = resized_storage_slots * kSlotSizeBytes;
-  if (align(alignof(Header),
-            sizeof(Header),
-            aligned_memory_start,
-            available_memory) == nullptr) {
-    // Should be impossible, as noted in constructor.
-    FATAL_ERROR(
-        "StorageBlob used to hold resized SeparateChainingHashTable "
-        "is too small to meet alignment requirements of "
-        "LinearOpenAddressingHashTable::Header.");
-  } else if (aligned_memory_start != resized_blob->getMemoryMutable()) {
-    // Again, should be impossible.
-    DEV_WARNING("In SeparateChainingHashTable::resize(), StorageBlob "
-                << "memory adjusted by "
-                << (resized_num_slots * kSlotSizeBytes - available_memory)
-                << " bytes to meet alignment requirement for "
-                << "LinearOpenAddressingHashTable::Header.");
-  }
-
-  Header *resized_header = static_cast<Header *>(aligned_memory_start);
-  aligned_memory_start =
-      static_cast<char *>(aligned_memory_start) + sizeof(Header);
-  available_memory -= sizeof(Header);
-
-  // As in constructor, recompute the number of slots and buckets using the
-  // actual available memory.
-  std::size_t resized_num_buckets =
-      (available_memory - extra_variable_storage) /
-      (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) + bucket_size_ +
-       key_manager_.getEstimatedVariableKeySize());
-  resized_num_slots =
-      get_previous_prime_number(resized_num_buckets * kHashTableLoadFactor);
-  resized_num_buckets = resized_num_slots / kHashTableLoadFactor;
-
-  // Locate slot array.
-  std::atomic<std::size_t> *resized_slots =
-      static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
-  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
-                         sizeof(std::atomic<std::size_t>) * resized_num_slots;
-  available_memory -= sizeof(std::atomic<std::size_t>) * resized_num_slots;
-
-  // As in constructor, we will be extra paranoid and use align() to locate the
-  // start of the array of buckets, as well.
-  void *resized_buckets = aligned_memory_start;
-  if (align(
-          kBucketAlignment, bucket_size_, resized_buckets, available_memory) ==
-      nullptr) {
-    FATAL_ERROR(
-        "StorageBlob used to hold resized SeparateChainingHashTable "
-        "is too small to meet alignment requirements of buckets.");
-  } else if (resized_buckets != aligned_memory_start) {
-    DEV_WARNING(
-        "Bucket array start position adjusted to meet alignment "
-        "requirement for SeparateChainingHashTable's value type.");
-    if (resized_num_buckets * bucket_size_ + variable_storage_required >
-        available_memory) {
-      --resized_num_buckets;
-    }
-  }
-  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
-                         resized_num_buckets * bucket_size_;
-  available_memory -= resized_num_buckets * bucket_size_;
-
-  void *resized_variable_length_key_storage = aligned_memory_start;
-  const std::size_t resized_variable_length_key_storage_size = available_memory;
-
-  const std::size_t original_buckets_used =
-      header_->buckets_allocated.load(std::memory_order_relaxed);
-
-  // Initialize the header.
-  resized_header->num_slots = resized_num_slots;
-  resized_header->num_buckets = resized_num_buckets;
-  resized_header->buckets_allocated.store(original_buckets_used,
-                                          std::memory_order_relaxed);
-  resized_header->variable_length_bytes_allocated.store(
-      original_variable_storage_used, std::memory_order_relaxed);
-
-  // Bulk-copy buckets. This is safe because:
-  //     1. The "next" pointers will be adjusted when rebuilding chains below.
-  //     2. The hash codes will stay the same.
-  //     3. For key components:
-  //       a. Inline keys will stay exactly the same.
-  //       b. Offsets into variable-length storage will remain valid, because
-  //          we also do a byte-for-byte copy of variable-length storage below.
-  //       c. Absolute external pointers will still point to the same address.
-  //       d. Relative pointers are not used with resizable hash tables.
-  //     4. If values are not trivially copyable, then we invoke ValueT's copy
-  //        or move constructor with placement new.
-  // NOTE(harshad) - Regarding point 4 above, as this is a specialized
-  // hash table implemented for aggregation, the values are trivially copyable,
-  // therefore we don't need to invoke payload values' copy/move constructors.
-  std::memcpy(resized_buckets, buckets_, original_buckets_used * bucket_size_);
-
-  // Copy over variable-length key components, if any.
-  if (original_variable_storage_used > 0) {
-    DEBUG_ASSERT(original_variable_storage_used ==
-                 key_manager_.getNextVariableLengthKeyOffset());
-    DEBUG_ASSERT(original_variable_storage_used <=
-                 resized_variable_length_key_storage_size);
-    std::memcpy(resized_variable_length_key_storage,
-                key_manager_.getVariableLengthKeyStorage(),
-                original_variable_storage_used);
-  }
-
-  destroyPayload();
-
-  // Make resized structures active.
-  std::swap(this->blob_, resized_blob);
-  header_ = resized_header;
-  slots_ = resized_slots;
-  buckets_ = resized_buckets;
-  key_manager_.setVariableLengthStorageInfo(
-      resized_variable_length_key_storage,
-      resized_variable_length_key_storage_size,
-      &(resized_header->variable_length_bytes_allocated));
-
-  // Drop the old blob.
-  const block_id old_blob_id = resized_blob->getID();
-  resized_blob.release();
-  this->storage_manager_->deleteBlockOrBlobFile(old_blob_id);
-
-  // Rebuild chains.
-  void *current_bucket = buckets_;
-  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used;
-       ++bucket_num) {
-    std::atomic<std::size_t> *next_ptr =
-        static_cast<std::atomic<std::size_t> *>(current_bucket);
-    const std::size_t hash_code = *reinterpret_cast<const std::size_t *>(
-        static_cast<const char *>(current_bucket) +
-        sizeof(std::atomic<std::size_t>));
-
-    const std::size_t slot_number = hash_code % header_->num_slots;
-    std::size_t slot_ptr_value = 0;
-    if (slots_[slot_number].compare_exchange_strong(
-            slot_ptr_value, bucket_num + 1, std::memory_order_relaxed)) {
-      // This bucket is the first in the chain for this block, so reset its
-      // next pointer to 0.
-      next_ptr->store(0, std::memory_order_relaxed);
-    } else {
-      // A chain already exists starting from this slot, so put this bucket at
-      // the head.
-      next_ptr->store(slot_ptr_value, std::memory_order_relaxed);
-      slots_[slot_number].store(bucket_num + 1, std::memory_order_relaxed);
-    }
-    current_bucket = static_cast<char *>(current_bucket) + bucket_size_;
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable,
-                                   serializable,
-                                   force_key_copy,
-                                   allow_duplicate_keys>::
-    preallocateForBulkInsert(const std::size_t total_entries,
-                             const std::size_t total_variable_key_size,
-                             HashTablePreallocationState *prealloc_state) {
-  DEBUG_ASSERT(allow_duplicate_keys);
-  if (!key_manager_.allocateVariableLengthKeyStorage(total_variable_key_size)) {
-    return false;
-  }
-
-  // We use load then compare-exchange here instead of simply fetch-add,
-  // because if multiple threads are simultaneously trying to allocate more
-  // than one bucket and exceed 'header_->num_buckets', their respective
-  // rollbacks might happen in such an order that some bucket ranges get
-  // skipped, while others might get double-allocated later.
-  std::size_t original_buckets_allocated =
-      header_->buckets_allocated.load(std::memory_order_relaxed);
-  std::size_t buckets_post_allocation =
-      original_buckets_allocated + total_entries;
-  while ((buckets_post_allocation <= header_->num_buckets) &&
-         !header_->buckets_allocated.compare_exchange_weak(
-             original_buckets_allocated,
-             buckets_post_allocation,
-             std::memory_order_relaxed)) {
-    buckets_post_allocation = original_buckets_allocated + total_entries;
-  }
-
-  if (buckets_post_allocation > header_->num_buckets) {
-    key_manager_.deallocateVariableLengthKeyStorage(total_variable_key_size);
-    return false;
-  }
-
-  prealloc_state->bucket_position = original_buckets_allocated;
-  if (total_variable_key_size != 0) {
-    prealloc_state->variable_length_key_position =
-        key_manager_.incrementNextVariableLengthKeyOffset(
-            total_variable_key_size);
-  }
-  return true;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-inline bool FastSeparateChainingHashTable<resizable,
-                                          serializable,
-                                          force_key_copy,
-                                          allow_duplicate_keys>::
-    locateBucketForInsertion(const std::size_t hash_code,
-                             const std::size_t variable_key_allocation_required,
-                             void **bucket,
-                             std::atomic<std::size_t> **pending_chain_ptr,
-                             std::size_t *pending_chain_ptr_finish_value,
-                             HashTablePreallocationState *prealloc_state) {
-  DEBUG_ASSERT((prealloc_state == nullptr) || allow_duplicate_keys);
-  if (*bucket == nullptr) {
-    *pending_chain_ptr = &(slots_[hash_code % header_->num_slots]);
-  } else {
-    *pending_chain_ptr = static_cast<std::atomic<std::size_t> *>(*bucket);
-  }
-  for (;;) {
-    std::size_t existing_chain_ptr = 0;
-    if ((*pending_chain_ptr)
-            ->compare_exchange_strong(existing_chain_ptr,
-                                      std::numeric_limits<std::size_t>::max(),
-                                      std::memory_order_acq_rel)) {
-      // Got to the end of the chain. Allocate a new bucket.
-
-      // First, allocate variable-length key storage, if needed (i.e. if this
-      // is an upsert and we didn't allocate up-front).
-      if ((prealloc_state == nullptr) &&
-          !key_manager_.allocateVariableLengthKeyStorage(
-              variable_key_allocation_required)) {
-        // Ran out of variable-length storage.
-        (*pending_chain_ptr)->store(0, std::memory_order_release);
-        *bucket = nullptr;
-        return false;
-      }
-
-      const std::size_t allocated_bucket_num =
-          (prealloc_state == nullptr)
-              ? header_->buckets_allocated.fetch_add(1,
-                                                     std::memory_order_relaxed)
-              : (prealloc_state->bucket_position)++;
-      if (allocated_bucket_num >= header_->num_buckets) {
-        // Ran out of buckets.
-        DEBUG_ASSERT(prealloc_state == nullptr);
-        header_->buckets_allocated.fetch_sub(1, std::memory_order_relaxed);
-        (*pending_chain_ptr)->store(0, std::memory_order_release);
-        *bucket = nullptr;
-        return false;
-      } else {
-        *bucket =
-            static_cast<char *>(buckets_) + allocated_bucket_num * bucket_size_;
-        *pending_chain_ptr_finish_value = allocated_bucket_num + 1;
-        return true;
-      }
-    }
-    // Spin until the real "next" pointer is available.
-    while (existing_chain_ptr == std::numeric_limits<std::size_t>::max()) {
-      existing_chain_ptr =
-          (*pending_chain_ptr)->load(std::memory_order_acquire);
-    }
-    if (existing_chain_ptr == 0) {
-      // Other thread had to roll back, so try again.
-      continue;
-    }
-    // Chase the next pointer.
-    *bucket =
-        static_cast<char *>(buckets_) + (existing_chain_ptr - 1) * bucket_size_;
-    *pending_chain_ptr = static_cast<std::atomic<std::size_t> *>(*bucket);
-    if (!allow_duplicate_keys) {
-      const std::size_t hash_in_bucket = *reinterpret_cast<const std::size_t *>(
-          static_cast<const char *>(*bucket) +
-          sizeof(std::atomic<std::size_t>));
-      if (hash_in_bucket == hash_code) {
-        return false;
-      }
-    }
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-inline void FastSeparateChainingHashTable<resizable,
-                                          serializable,
-                                          force_key_copy,
-                                          allow_duplicate_keys>::
-    writeScalarKeyToBucket(const TypedValue &key,
-                           const std::size_t hash_code,
-                           void *bucket,
-                           HashTablePreallocationState *prealloc_state) {
-  *reinterpret_cast<std::size_t *>(static_cast<char *>(bucket) +
-                                   sizeof(std::atomic<std::size_t>)) =
-      hash_code;
-  key_manager_.writeKeyComponentToBucket(key, 0, bucket, prealloc_state);
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-inline void FastSeparateChainingHashTable<resizable,
-                                          serializable,
-                                          force_key_copy,
-                                          allow_duplicate_keys>::
-    writeCompositeKeyToBucket(const std::vector<TypedValue> &key,
-                              const std::size_t hash_code,
-                              void *bucket,
-                              HashTablePreallocationState *prealloc_state) {
-  DEBUG_ASSERT(key.size() == this->key_types_.size());
-  *reinterpret_cast<std::size_t *>(static_cast<char *>(bucket) +
-                                   sizeof(std::atomic<std::size_t>)) =
-      hash_code;
-  for (std::size_t idx = 0; idx < this->key_types_.size(); ++idx) {
-    key_manager_.writeKeyComponentToBucket(
-        key[idx], idx, bucket, prealloc_state);
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::isFull(const std::size_t extra_variable_storage)
-    const {
-  if (header_->buckets_allocated.load(std::memory_order_relaxed) >=
-      header_->num_buckets) {
-    // All buckets are allocated.
-    return true;
-  }
-
-  if (extra_variable_storage > 0) {
-    if (extra_variable_storage +
-            header_->variable_length_bytes_allocated.load(
-                std::memory_order_relaxed) >
-        key_manager_.getVariableLengthKeyStorageSize()) {
-      // Not enough variable-length key storage space.
-      return true;
-    }
-  }
-
-  return false;
-}
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_SEPARATE_CHAINING_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index 1d4ccb0..6839ebc 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -22,9 +22,10 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 enum HashTableImplType {
-  LINEAR_OPEN_ADDRESSING = 0;
-  SEPARATE_CHAINING = 1;
-  SIMPLE_SCALAR_SEPARATE_CHAINING = 2;
+  COLLISION_FREE_VECTOR = 0;
+  LINEAR_OPEN_ADDRESSING = 1;
+  SEPARATE_CHAINING = 2;
+  SIMPLE_SCALAR_SEPARATE_CHAINING = 3;
 }
 
 // NOTE(chasseur): This proto describes the run-time parameters for a resizable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index a3180bb..b4b6918 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -23,11 +23,14 @@
 #include <cstddef>
 #include <vector>
 
-#include "ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+class ColumnVectorsValueAccessor;
+class ValueAccessor;
+
 /** \addtogroup Storage
  *  @{
  */
@@ -38,6 +41,7 @@ namespace quickstep {
  *        HashTableFactory to create a HashTable.
  **/
 enum class HashTableImplType {
+  kCollisionFreeVector,
   kLinearOpenAddressing,
   kSeparateChaining,
   kSimpleScalarSeparateChaining
@@ -74,6 +78,17 @@ class HashTableBase {
  public:
   virtual ~HashTableBase() {}
 
+ protected:
+  HashTableBase() {}
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(HashTableBase);
+};
+
+class AggregationStateHashTableBase {
+ public:
+  virtual ~AggregationStateHashTableBase() {}
+
   /**
    * TODO(harshad) We should get rid of this function from here. We are
    * postponing it because of the amount of work to be done is significant.
@@ -90,30 +105,23 @@ class HashTableBase {
    *
    * Optionally, we can also remove the AggregationStateHashTableBase
    * specialization from this file.
+   *
+   * TODO(jianqiao): Refractor the interface design for aggregation hash table.
    **/
-  virtual bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<attribute_id> &argument,
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) {
-    return false;
-  }
+  virtual bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_attr_ids,
+      const ValueAccessorMultiplexer &accessor_mux) = 0;
 
-  /**
-   * @brief Destroy the payload stored in the hash table.
-   **/
-  virtual void destroyPayload() {
-  }
+  virtual void destroyPayload() = 0;
 
  protected:
-  HashTableBase() {}
+  AggregationStateHashTableBase() {}
 
  private:
-  DISALLOW_COPY_AND_ASSIGN(HashTableBase);
+  DISALLOW_COPY_AND_ASSIGN(AggregationStateHashTableBase);
 };
 
-typedef HashTableBase<true, false, true, false> AggregationStateHashTableBase;
-
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index d690557..9686429 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -24,10 +24,12 @@
 #include <string>
 #include <vector>
 
+#include "storage/CollisionFreeVectorTable.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTable.pb.h"
 #include "storage/LinearOpenAddressingHashTable.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/SeparateChainingHashTable.hpp"
 #include "storage/SimpleScalarSeparateChainingHashTable.hpp"
 #include "storage/TupleReference.hpp"
@@ -113,6 +115,8 @@ serialization::HashTableImplType SimplifyHashTableImplTypeProto(
 inline HashTableImplType HashTableImplTypeFromProto(
     const serialization::HashTableImplType proto_type) {
   switch (proto_type) {
+    case serialization::HashTableImplType::COLLISION_FREE_VECTOR:
+      return HashTableImplType::kCollisionFreeVector;
     case serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING:
       return HashTableImplType::kLinearOpenAddressing;
     case serialization::HashTableImplType::SEPARATE_CHAINING:
@@ -324,19 +328,62 @@ class HashTableFactory {
 };
 
 /**
- * @brief Convenient alias that provides a HashTableFactory whose only template
- *        parameter is the aggregate state type.
- **/
-template <typename ValueT>
-using AggregationStateHashTableFactory
-    = HashTableFactory<ValueT, true, false, true, false>;
-
-/**
  * @brief Convenient alias for a HashTableFactory that makes JoinHashTables.
  **/
 typedef HashTableFactory<TupleReference, true, false, false, true>
     JoinHashTableFactory;
 
+/**
+ * @brief Factory class that makes it easier to instantiate aggregation state
+ *        hash tables.
+ **/
+class AggregationStateHashTableFactory {
+ public:
+  /**
+   * @brief Create a new aggregation state hash table, with the type selected by
+   *        hash_table_type. Other parameters are forwarded to the hash table's
+   *        constructor.
+   *
+   * @param hash_table_type The specific hash table implementation that should
+   *        be used.
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key). Forwarded as-is to the hash table's constructor.
+   * @param num_entries The estimated number of entries the hash table will
+   *        hold. Forwarded as-is to the hash table's constructor.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold the hash table's contents). Forwarded as-is to the
+   *        hash table constructor.
+   * @return A new aggregation state hash table.
+   **/
+
+  static AggregationStateHashTableBase* CreateResizable(
+      const HashTableImplType hash_table_type,
+      const std::vector<const Type*> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager) {
+    switch (hash_table_type) {
+      case HashTableImplType::kSeparateChaining:
+        return new PackedPayloadHashTable(
+            key_types, num_entries, handles, storage_manager);
+      case HashTableImplType::kCollisionFreeVector:
+        DCHECK_EQ(1u, key_types.size());
+        return new CollisionFreeVectorTable(
+            key_types.front(), num_entries, handles, storage_manager);
+      default: {
+        LOG(FATAL) << "Unrecognized HashTableImplType in "
+                   << "AggregationStateHashTableFactory::createResizable()";
+      }
+    }
+  }
+
+ private:
+  // Class is all-static and should not be instantiated.
+  AggregationStateHashTableFactory();
+
+  DISALLOW_COPY_AND_ASSIGN(AggregationStateHashTableFactory);
+};
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index 96cf849..df527b8 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -20,23 +20,21 @@
 #ifndef QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_
 #define QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_
 
-#include <chrono>
+#include <cstddef>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/HashTableFactory.hpp"
 #include "threading/SpinMutex.hpp"
 #include "utility/Macros.hpp"
-#include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
+class AggregationHandle;
 class StorageManager;
 class Type;
 
@@ -56,36 +54,6 @@ class HashTablePool {
   /**
    * @brief Constructor.
    *
-   * @param estimated_num_entries The maximum number of entries in a hash table.
-   * @param hash_table_impl_type The type of hash table implementation.
-   * @param group_by_types A vector of pointer of types which form the group by
-   *        key.
-   * @param agg_handle The aggregation handle.
-   * @param storage_manager A pointer to the storage manager.
-   *
-   * @note The estimate of number of entries is quite inaccurate at this time.
-   *       If we go by the current estimate, each hash table demands much
-   *       larger space than it actually needs, which causes the system to
-   *       either trigger evictions or worse - run out of memory. To fix this
-   *       issue, we divide the estimate by 100. The division will not affect
-   *       correctness, however it may allocate some hash tables smaller space
-   *       than their requirement, causing them to be resized during build
-   *       phase, which has a performance penalty.
-   **/
-  HashTablePool(const std::size_t estimated_num_entries,
-                const HashTableImplType hash_table_impl_type,
-                const std::vector<const Type *> &group_by_types,
-                AggregationHandle *agg_handle,
-                StorageManager *storage_manager)
-      : estimated_num_entries_(reduceEstimatedCardinality(estimated_num_entries)),
-        hash_table_impl_type_(hash_table_impl_type),
-        group_by_types_(group_by_types),
-        agg_handle_(DCHECK_NOTNULL(agg_handle)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
-
-  /**
-   * @brief Constructor.
-   *
    * @note This constructor is relevant for HashTables specialized for
    *       aggregation.
    *
@@ -93,52 +61,29 @@ class HashTablePool {
    * @param hash_table_impl_type The type of hash table implementation.
    * @param group_by_types A vector of pointer of types which form the group by
    *        key.
-   * @param payload_sizes The sizes in bytes for the AggregationStates for the
-   *        respective AggregationHandles.
    * @param handles The AggregationHandles in this query.
    * @param storage_manager A pointer to the storage manager.
    **/
   HashTablePool(const std::size_t estimated_num_entries,
                 const HashTableImplType hash_table_impl_type,
                 const std::vector<const Type *> &group_by_types,
-                const std::vector<std::size_t> &payload_sizes,
                 const std::vector<AggregationHandle *> &handles,
                 StorageManager *storage_manager)
       : estimated_num_entries_(reduceEstimatedCardinality(estimated_num_entries)),
         hash_table_impl_type_(hash_table_impl_type),
         group_by_types_(group_by_types),
-        payload_sizes_(payload_sizes),
         handles_(handles),
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
 
   /**
    * @brief Check out a hash table for insertion.
    *
-   * @return A hash table pointer.
-   **/
-  AggregationStateHashTableBase* getHashTable() {
-    {
-      SpinMutexLock lock(mutex_);
-      if (!hash_tables_.empty()) {
-        std::unique_ptr<AggregationStateHashTableBase> ret_hash_table(
-            std::move(hash_tables_.back()));
-        hash_tables_.pop_back();
-        DCHECK(ret_hash_table != nullptr);
-        return ret_hash_table.release();
-      }
-    }
-    return createNewHashTable();
-  }
-
-  /**
-   * @brief Check out a hash table for insertion.
-   *
    * @note This method is relevant for specialized (for aggregation)
    *       hash table implementation.
    *
    * @return A hash table pointer.
    **/
-  AggregationStateHashTableBase* getHashTableFast() {
+  AggregationStateHashTableBase* getHashTable() {
     {
       SpinMutexLock lock(mutex_);
       if (!hash_tables_.empty()) {
@@ -149,7 +94,7 @@ class HashTablePool {
         return ret_hash_table.release();
       }
     }
-    return createNewHashTableFast();
+    return createNewHashTable();
   }
 
   /**
@@ -180,18 +125,10 @@ class HashTablePool {
 
  private:
   AggregationStateHashTableBase* createNewHashTable() {
-    return agg_handle_->createGroupByHashTable(hash_table_impl_type_,
-                                               group_by_types_,
-                                               estimated_num_entries_,
-                                               storage_manager_);
-  }
-
-  AggregationStateHashTableBase* createNewHashTableFast() {
-    return AggregationStateFastHashTableFactory::CreateResizable(
+    return AggregationStateHashTableFactory::CreateResizable(
                 hash_table_impl_type_,
                 group_by_types_,
                 estimated_num_entries_,
-                payload_sizes_,
                 handles_,
                 storage_manager_);
   }
@@ -214,10 +151,6 @@ class HashTablePool {
   const HashTableImplType hash_table_impl_type_;
 
   const std::vector<const Type *> group_by_types_;
-
-  std::vector<std::size_t> payload_sizes_;
-
-  AggregationHandle *agg_handle_;
   const std::vector<AggregationHandle *> handles_;
   StorageManager *storage_manager_;
 


[40/50] [abbrv] incubator-quickstep git commit: Fuse Aggregate with LeftOuterJoin to accelerate evaluation.

Posted by ji...@apache.org.
Fuse Aggregate with LeftOuterJoin to accelerate evaluation.


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

Branch: refs/heads/LIP-time-decomposition
Commit: a28b1e4d77ee12466b0801a5a7c5185f7a83e7f8
Parents: 266b9b9
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 30 14:46:39 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Wed Feb 8 23:55:32 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   6 +-
 query_optimizer/ExecutionGenerator.cpp          | 261 +++++++++++--------
 query_optimizer/ExecutionGenerator.hpp          |  20 +-
 query_optimizer/PhysicalGenerator.cpp           |   3 +
 query_optimizer/cost_model/CMakeLists.txt       |   8 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |   9 +
 query_optimizer/cost_model/SimpleCostModel.hpp  |   5 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    | 148 ++++++++++-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |  20 ++
 query_optimizer/physical/CMakeLists.txt         |  14 +
 .../CrossReferenceCoalesceAggregate.cpp         | 105 ++++++++
 .../CrossReferenceCoalesceAggregate.hpp         | 232 +++++++++++++++++
 query_optimizer/physical/PatternMatcher.hpp     |   3 +
 query_optimizer/physical/PhysicalType.hpp       |   1 +
 query_optimizer/rules/BottomUpRule.hpp          |  39 +--
 query_optimizer/rules/CMakeLists.txt            |  23 ++
 query_optimizer/rules/FuseAggregateJoin.cpp     | 170 ++++++++++++
 query_optimizer/rules/FuseAggregateJoin.hpp     |  71 +++++
 .../BuildAggregationExistenceMapOperator.cpp    | 196 ++++++++++++++
 .../BuildAggregationExistenceMapOperator.hpp    | 177 +++++++++++++
 relational_operators/CMakeLists.txt             |  30 +++
 relational_operators/WorkOrder.proto            |  12 +-
 relational_operators/WorkOrderFactory.cpp       |  37 +++
 storage/AggregationOperationState.cpp           |   8 +-
 storage/AggregationOperationState.hpp           |   9 +
 storage/CollisionFreeVectorTable.hpp            |   9 +
 utility/lip_filter/BitVectorExactFilter.hpp     |  27 +-
 utility/lip_filter/CMakeLists.txt               |  12 +-
 utility/lip_filter/SingleIdentityHashFilter.hpp |  22 +-
 29 files changed, 1489 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index e750a1e..3ff783c 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -64,7 +64,6 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_expressions_Expressions_proto
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunction_proto
-                      quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_expressions_scalar_ScalarAttribute
@@ -95,6 +94,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_CopyFrom
                       quickstep_queryoptimizer_physical_CreateIndex
                       quickstep_queryoptimizer_physical_CreateTable
+                      quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_FilterJoin
@@ -116,6 +116,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_UpdateTable
                       quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_relationaloperators_AggregationOperator
+                      quickstep_relationaloperators_BuildAggregationExistenceMapOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_CreateIndexOperator
@@ -147,12 +148,10 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_storage_StorageBlockLayout_proto
                       quickstep_storage_SubBlockTypeRegistry
                       quickstep_types_Type
-                      quickstep_types_TypeID
                       quickstep_types_Type_proto
                       quickstep_types_TypedValue
                       quickstep_types_TypedValue_proto
                       quickstep_types_containers_Tuple_proto
-                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros
                       quickstep_utility_SqlError)
 if (ENABLE_DISTRIBUTED)
@@ -213,6 +212,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
+                      quickstep_queryoptimizer_rules_FuseAggregateJoin
                       quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 1b50caa..70b69e0 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -49,7 +49,6 @@
 #include "expressions/Expressions.pb.h"
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunction.pb.h"
-#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "expressions/scalar/ScalarAttribute.hpp"
@@ -72,9 +71,11 @@
 #include "query_optimizer/expressions/Scalar.hpp"
 #include "query_optimizer/expressions/ScalarLiteral.hpp"
 #include "query_optimizer/expressions/WindowAggregateFunction.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/CopyFrom.hpp"
 #include "query_optimizer/physical/CreateIndex.hpp"
 #include "query_optimizer/physical/CreateTable.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
 #include "query_optimizer/physical/FilterJoin.hpp"
@@ -96,6 +97,7 @@
 #include "query_optimizer/physical/UpdateTable.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
 #include "relational_operators/AggregationOperator.hpp"
+#include "relational_operators/BuildAggregationExistenceMapOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
 #include "relational_operators/BuildLIPFilterOperator.hpp"
 #include "relational_operators/CreateIndexOperator.hpp"
@@ -128,11 +130,9 @@
 #include "storage/SubBlockTypeRegistry.hpp"
 #include "types/Type.hpp"
 #include "types/Type.pb.h"
-#include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "types/TypedValue.pb.h"
 #include "types/containers/Tuple.pb.h"
-#include "utility/EqualsAnyConstant.hpp"
 #include "utility/SqlError.hpp"
 
 #include "gflags/gflags.h"
@@ -163,10 +163,6 @@ static const volatile bool aggregate_hashtable_type_dummy
 
 DEFINE_bool(parallelize_load, true, "Parallelize loading data files.");
 
-DEFINE_int64(collision_free_vector_table_max_size, 1000000000,
-              "The maximum allowed key range (number of entries) for using a "
-              "CollisionFreeVectorTable.");
-
 namespace E = ::quickstep::optimizer::expressions;
 namespace P = ::quickstep::optimizer::physical;
 namespace S = ::quickstep::serialization;
@@ -266,6 +262,9 @@ void ExecutionGenerator::generatePlanInternal(
     case P::PhysicalType::kAggregate:
       return convertAggregate(
           std::static_pointer_cast<const P::Aggregate>(physical_plan));
+    case P::PhysicalType::kCrossReferenceCoalesceAggregate:
+      return convertCrossReferenceCoalesceAggregate(
+          std::static_pointer_cast<const P::CrossReferenceCoalesceAggregate>(physical_plan));
     case P::PhysicalType::kCopyFrom:
       return convertCopyFrom(
           std::static_pointer_cast<const P::CopyFrom>(physical_plan));
@@ -379,105 +378,6 @@ void ExecutionGenerator::dropAllTemporaryRelations() {
   }
 }
 
-bool ExecutionGenerator::canUseCollisionFreeAggregation(
-    const P::AggregatePtr &aggregate,
-    const std::size_t estimated_num_groups,
-    std::size_t *max_num_groups) const {
-#ifdef QUICKSTEP_DISTRIBUTED
-  // Currently we cannot do this fast path with the distributed setting. See
-  // the TODOs at InitializeAggregationOperator::getAllWorkOrderProtos() and
-  // FinalizeAggregationOperator::getAllWorkOrderProtos().
-  return false;
-#endif
-
-  // Supports only single group-by key.
-  if (aggregate->grouping_expressions().size() != 1) {
-    return false;
-  }
-
-  // We need to know the exact min/max stats of the group-by key.
-  // So it must be a CatalogAttribute (but not an expression).
-  E::AttributeReferencePtr group_by_key_attr;
-  const E::ExpressionPtr agg_expr = aggregate->grouping_expressions().front();
-  if (!E::SomeAttributeReference::MatchesWithConditionalCast(agg_expr, &group_by_key_attr)) {
-    return false;
-  }
-
-  bool min_value_stat_is_exact;
-  bool max_value_stat_is_exact;
-  const TypedValue min_value =
-      cost_model_for_aggregation_->findMinValueStat(
-          aggregate, group_by_key_attr, &min_value_stat_is_exact);
-  const TypedValue max_value =
-      cost_model_for_aggregation_->findMaxValueStat(
-          aggregate, group_by_key_attr, &max_value_stat_is_exact);
-  if (min_value.isNull() || max_value.isNull() ||
-      (!min_value_stat_is_exact) || (!max_value_stat_is_exact)) {
-    return false;
-  }
-
-  std::int64_t min_cpp_value;
-  std::int64_t max_cpp_value;
-  switch (group_by_key_attr->getValueType().getTypeID()) {
-    case TypeID::kInt: {
-      min_cpp_value = min_value.getLiteral<int>();
-      max_cpp_value = max_value.getLiteral<int>();
-      break;
-    }
-    case TypeID::kLong: {
-      min_cpp_value = min_value.getLiteral<std::int64_t>();
-      max_cpp_value = max_value.getLiteral<std::int64_t>();
-      break;
-    }
-    default:
-      return false;
-  }
-
-  // TODO(jianqiao):
-  // 1. Handle the case where min_cpp_value is below 0 or far greater than 0.
-  // 2. Reason about the table size bound (e.g. by checking memory size) instead
-  //    of hardcoding it as a gflag.
-  if (min_cpp_value < 0 ||
-      max_cpp_value >= FLAGS_collision_free_vector_table_max_size ||
-      max_cpp_value / static_cast<double>(estimated_num_groups) > 256.0) {
-    return false;
-  }
-
-  for (const auto &agg_expr : aggregate->aggregate_expressions()) {
-    const E::AggregateFunctionPtr agg_func =
-        std::static_pointer_cast<const E::AggregateFunction>(agg_expr->expression());
-
-    if (agg_func->is_distinct()) {
-      return false;
-    }
-
-    // TODO(jianqiao): Support AggregationID::AVG.
-    if (!QUICKSTEP_EQUALS_ANY_CONSTANT(agg_func->getAggregate().getAggregationID(),
-                                       AggregationID::kCount,
-                                       AggregationID::kSum)) {
-      return false;
-    }
-
-    const auto &arguments = agg_func->getArguments();
-    if (arguments.size() > 1u) {
-      return false;
-    }
-
-    if (arguments.size() == 1u) {
-      if (!QUICKSTEP_EQUALS_ANY_CONSTANT(arguments.front()->getValueType().getTypeID(),
-                                         TypeID::kInt,
-                                         TypeID::kLong,
-                                         TypeID::kFloat,
-                                         TypeID::kDouble)) {
-        return false;
-      }
-    }
-  }
-
-  *max_num_groups = static_cast<std::size_t>(max_cpp_value) + 1;
-  return true;
-}
-
 void ExecutionGenerator::convertNamedExpressions(
     const std::vector<E::NamedExpressionPtr> &named_expressions,
     S::QueryContext::ScalarGroup *scalar_group_proto) {
@@ -1608,9 +1508,10 @@ void ExecutionGenerator::convertAggregate(
         cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
 
     std::size_t max_num_groups;
-    if (canUseCollisionFreeAggregation(physical_plan,
-                                       estimated_num_groups,
-                                       &max_num_groups)) {
+    if (cost_model_for_aggregation_
+            ->canUseCollisionFreeAggregation(physical_plan,
+                                             estimated_num_groups,
+                                             &max_num_groups)) {
       aggr_state_proto->set_hash_table_impl_type(
           serialization::HashTableImplType::COLLISION_FREE_VECTOR);
       aggr_state_proto->set_estimated_num_entries(max_num_groups);
@@ -1730,6 +1631,148 @@ void ExecutionGenerator::convertAggregate(
   }
 }
 
+void ExecutionGenerator::convertCrossReferenceCoalesceAggregate(
+    const P::CrossReferenceCoalesceAggregatePtr &physical_plan) {
+  DCHECK_EQ(1u, physical_plan->left_join_attributes().size());
+  DCHECK_EQ(1u, physical_plan->right_join_attributes().size());
+
+  const CatalogRelationInfo *left_relation_info =
+      findRelationInfoOutputByPhysical(physical_plan->left_child());
+  const CatalogRelationInfo *right_relation_info =
+      findRelationInfoOutputByPhysical(physical_plan->right_child());
+
+  // Create aggr state proto.
+  const QueryContext::aggregation_state_id aggr_state_index =
+      query_context_proto_->aggregation_states_size();
+  S::AggregationOperationState *aggr_state_proto = query_context_proto_->add_aggregation_states();
+
+  aggr_state_proto->set_relation_id(right_relation_info->relation->getID());
+
+  // Group by the right join attribute.
+  std::unique_ptr<const Scalar> execution_group_by_expression(
+      physical_plan->right_join_attributes().front()->concretize(
+          attribute_substitution_map_));
+  aggr_state_proto->add_group_by_expressions()->CopyFrom(
+      execution_group_by_expression->getProto());
+
+  aggr_state_proto->set_hash_table_impl_type(
+      serialization::HashTableImplType::COLLISION_FREE_VECTOR);
+  aggr_state_proto->set_estimated_num_entries(
+      physical_plan->group_by_key_value_range());
+
+  if (physical_plan->right_filter_predicate() != nullptr) {
+    std::unique_ptr<const Predicate> predicate(
+        convertPredicate(physical_plan->right_filter_predicate()));
+    aggr_state_proto->mutable_predicate()->CopyFrom(predicate->getProto());
+  }
+
+  for (const E::AliasPtr &named_aggregate_expression : physical_plan->aggregate_expressions()) {
+    const E::AggregateFunctionPtr unnamed_aggregate_expression =
+        std::static_pointer_cast<const E::AggregateFunction>(named_aggregate_expression->expression());
+
+    // Add a new entry in 'aggregates'.
+    S::Aggregate *aggr_proto = aggr_state_proto->add_aggregates();
+
+    // Set the AggregateFunction.
+    aggr_proto->mutable_function()->CopyFrom(
+        unnamed_aggregate_expression->getAggregate().getProto());
+
+    // Add each of the aggregate's arguments.
+    for (const E::ScalarPtr &argument : unnamed_aggregate_expression->getArguments()) {
+      unique_ptr<const Scalar> concretized_argument(argument->concretize(attribute_substitution_map_));
+      aggr_proto->add_argument()->CopyFrom(concretized_argument->getProto());
+    }
+
+    // Set whether it is a DISTINCT aggregation.
+    DCHECK(!unnamed_aggregate_expression->is_distinct());
+    aggr_proto->set_is_distinct(false);
+  }
+
+  const QueryPlan::DAGNodeIndex initialize_aggregation_operator_index =
+      execution_plan_->addRelationalOperator(
+          new InitializeAggregationOperator(
+              query_handle_->query_id(),
+              aggr_state_index));
+
+  const QueryPlan::DAGNodeIndex build_aggregation_existence_map_operator_index =
+      execution_plan_->addRelationalOperator(
+          new BuildAggregationExistenceMapOperator(
+              query_handle_->query_id(),
+              *left_relation_info->relation,
+              physical_plan->left_join_attributes().front()->id(),
+              left_relation_info->isStoredRelation(),
+              aggr_state_index));
+
+  if (!left_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(build_aggregation_existence_map_operator_index,
+                                         left_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+
+  const QueryPlan::DAGNodeIndex aggregation_operator_index =
+      execution_plan_->addRelationalOperator(
+          new AggregationOperator(
+              query_handle_->query_id(),
+              *right_relation_info->relation,
+              right_relation_info->isStoredRelation(),
+              aggr_state_index));
+
+  if (!right_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(aggregation_operator_index,
+                                         right_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+
+  // Build aggregation existence map once initialization is done.
+  execution_plan_->addDirectDependency(build_aggregation_existence_map_operator_index,
+                                       initialize_aggregation_operator_index,
+                                       true /* is_pipeline_breaker */);
+
+  // Start aggregation after building existence map.
+  execution_plan_->addDirectDependency(aggregation_operator_index,
+                                       build_aggregation_existence_map_operator_index,
+                                       true /* is_pipeline_breaker */);
+
+
+  // Create InsertDestination proto.
+  const CatalogRelation *output_relation = nullptr;
+  const QueryContext::insert_destination_id insert_destination_index =
+      query_context_proto_->insert_destinations_size();
+  S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
+  createTemporaryCatalogRelation(physical_plan,
+                                 &output_relation,
+                                 insert_destination_proto);
+
+  const QueryPlan::DAGNodeIndex finalize_aggregation_operator_index =
+      execution_plan_->addRelationalOperator(
+          new FinalizeAggregationOperator(query_handle_->query_id(),
+                                          aggr_state_index,
+                                          *output_relation,
+                                          insert_destination_index));
+
+  insert_destination_proto->set_relational_op_index(finalize_aggregation_operator_index);
+
+  execution_plan_->addDirectDependency(finalize_aggregation_operator_index,
+                                       aggregation_operator_index,
+                                       true /* is_pipeline_breaker */);
+
+  physical_to_output_relation_map_.emplace(
+      std::piecewise_construct,
+      std::forward_as_tuple(physical_plan),
+      std::forward_as_tuple(finalize_aggregation_operator_index, output_relation));
+  temporary_relation_info_vec_.emplace_back(finalize_aggregation_operator_index,
+                                            output_relation);
+
+  const QueryPlan::DAGNodeIndex destroy_aggregation_state_operator_index =
+      execution_plan_->addRelationalOperator(
+          new DestroyAggregationStateOperator(query_handle_->query_id(),
+                                              aggr_state_index));
+
+  execution_plan_->addDirectDependency(destroy_aggregation_state_operator_index,
+                                       finalize_aggregation_operator_index,
+                                       true);
+}
+
 void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
   // Create sort configuration for run generation.
   vector<bool> sort_ordering(physical_sort->sort_ascending());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 987f11a..f4e614a 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -46,6 +46,7 @@
 #include "query_optimizer/physical/CopyFrom.hpp"
 #include "query_optimizer/physical/CreateIndex.hpp"
 #include "query_optimizer/physical/CreateTable.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
 #include "query_optimizer/physical/FilterJoin.hpp"
@@ -206,22 +207,6 @@ class ExecutionGenerator {
   std::string getNewRelationName();
 
   /**
-   * @brief Checks whether an aggregate node can be efficiently evaluated with
-   *        the collision-free aggregation fast path.
-   *
-   * @param aggregate The physical aggregate node to be checked.
-   * @param estimated_num_groups The estimated number of groups for the aggregate.
-   * @param exact_num_groups If collision-free aggregation is applicable, the
-   *        pointed content of this pointer will be set as the maximum possible
-   *        number of groups that the collision-free hash table need to hold.
-   * @return A bool value indicating whether collision-free aggregation can be
-   *         used to evaluate \p aggregate.
-   */
-  bool canUseCollisionFreeAggregation(const physical::AggregatePtr &aggregate,
-                                      const std::size_t estimated_num_groups,
-                                      std::size_t *max_num_groups) const;
-
-  /**
    * @brief Sets up the info of the CatalogRelation represented by TableReference.
    *        TableReference is not converted to any operator.
    *
@@ -356,6 +341,9 @@ class ExecutionGenerator {
    */
   void convertAggregate(const physical::AggregatePtr &physical_plan);
 
+  void convertCrossReferenceCoalesceAggregate(
+      const physical::CrossReferenceCoalesceAggregatePtr &physical_plan);
+
   /**
    * @brief Converts a physical Sort to SortRunGeneration and SortMergeRun.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 1b68f49..ac51c31 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -27,6 +27,7 @@
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
+#include "query_optimizer/rules/FuseAggregateJoin.hpp"
 #include "query_optimizer/rules/InjectJoinFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
@@ -145,6 +146,8 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     rules.emplace_back(new ReorderColumns());
   }
 
+  rules.emplace_back(new FuseAggregateJoin());
+
   // NOTE(jianqiao): Adding rules after InjectJoinFilters (or AttachLIPFilters) requires
   // extra handling of LIPFilterConfiguration for transformed nodes. So currently it is
   // suggested that all the new rules be placed before this point.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 5f28bb3..4042915 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -33,6 +33,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
                       quickstep_catalog_CatalogRelationStatistics
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
                       quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
@@ -51,7 +52,10 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregateFunction
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_queryoptimizer_costmodel_CostModel
+                      quickstep_queryoptimizer_expressions_AggregateFunction
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression
                       quickstep_queryoptimizer_expressions_ExprId
@@ -62,6 +66,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Predicate
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
                       quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
@@ -76,7 +81,10 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_types_NullType
+                      quickstep_types_Type
+                      quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
 
 # Module all-in-one library:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index e9d2e3a..cfd8a75 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -26,6 +26,7 @@
 #include "catalog/CatalogRelationStatistics.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
 #include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
@@ -74,6 +75,9 @@ std::size_t SimpleCostModel::estimateCardinality(
     case P::PhysicalType::kAggregate:
       return estimateCardinalityForAggregate(
           std::static_pointer_cast<const P::Aggregate>(physical_plan));
+    case P::PhysicalType::kCrossReferenceCoalesceAggregate:
+      return estimateCardinalityForCrossReferenceCoalesceAggregate(
+          std::static_pointer_cast<const P::CrossReferenceCoalesceAggregate>(physical_plan));
     case P::PhysicalType::kSharedSubplanReference: {
       const P::SharedSubplanReferencePtr shared_subplan_reference =
           std::static_pointer_cast<const P::SharedSubplanReference>(physical_plan);
@@ -149,6 +153,11 @@ std::size_t SimpleCostModel::estimateCardinalityForAggregate(
                   estimateCardinality(physical_plan->input()) / 10);
 }
 
+std::size_t SimpleCostModel::estimateCardinalityForCrossReferenceCoalesceAggregate(
+    const physical::CrossReferenceCoalesceAggregatePtr &physical_plan) {
+  return estimateCardinality(physical_plan->left_child());
+}
+
 std::size_t SimpleCostModel::estimateCardinalityForWindowAggregate(
     const physical::WindowAggregatePtr &physical_plan) {
   return estimateCardinality(physical_plan->input());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/cost_model/SimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.hpp b/query_optimizer/cost_model/SimpleCostModel.hpp
index 4edc2fe..0660c37 100644
--- a/query_optimizer/cost_model/SimpleCostModel.hpp
+++ b/query_optimizer/cost_model/SimpleCostModel.hpp
@@ -25,6 +25,7 @@
 
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
 #include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
@@ -100,6 +101,10 @@ class SimpleCostModel : public CostModel {
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
 
+  // Returns the cardinality of the left child plan.
+  std::size_t estimateCardinalityForCrossReferenceCoalesceAggregate(
+      const physical::CrossReferenceCoalesceAggregatePtr &physical_plan);
+
   // Return the estimated cardinality of the input plan.
   std::size_t estimateCardinalityForWindowAggregate(
       const physical::WindowAggregatePtr &physical_plan);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 7afa1c3..fc775c7 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -20,13 +20,18 @@
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 
 #include <algorithm>
+#include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationStatistics.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregateFunction.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/expressions/AggregateFunction.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
@@ -37,6 +42,7 @@
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
 #include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
@@ -49,8 +55,13 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "types/NullType.hpp"
+#include "utility/EqualsAnyConstant.hpp"
+
+#include "gflags/gflags.h"
 
 #include "glog/logging.h"
 
@@ -58,6 +69,10 @@ namespace quickstep {
 namespace optimizer {
 namespace cost {
 
+DEFINE_int64(collision_free_vector_table_max_size, 1000000000,
+              "The maximum allowed key range (number of entries) for using a "
+              "CollisionFreeVectorTable.");
+
 namespace E = ::quickstep::optimizer::expressions;
 namespace P = ::quickstep::optimizer::physical;
 
@@ -88,6 +103,9 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinality(
     case P::PhysicalType::kAggregate:
       return estimateCardinalityForAggregate(
           std::static_pointer_cast<const P::Aggregate>(physical_plan));
+    case P::PhysicalType::kCrossReferenceCoalesceAggregate:
+      return estimateCardinalityForCrossReferenceCoalesceAggregate(
+          std::static_pointer_cast<const P::CrossReferenceCoalesceAggregate>(physical_plan));
     case P::PhysicalType::kSharedSubplanReference: {
       const P::SharedSubplanReferencePtr shared_subplan_reference =
           std::static_pointer_cast<const P::SharedSubplanReference>(physical_plan);
@@ -175,6 +193,11 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForAggregate(
       estimateNumGroupsForAggregate(physical_plan) * filter_selectivity);
 }
 
+std::size_t StarSchemaSimpleCostModel::estimateCardinalityForCrossReferenceCoalesceAggregate(
+    const P::CrossReferenceCoalesceAggregatePtr &physical_plan) {
+  return estimateCardinality(physical_plan->left_child());
+}
+
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForWindowAggregate(
     const P::WindowAggregatePtr &physical_plan) {
   return estimateCardinality(physical_plan->input());
@@ -233,6 +256,13 @@ std::size_t StarSchemaSimpleCostModel::estimateNumDistinctValues(
       }
       break;
     }
+    case P::PhysicalType::kCrossReferenceCoalesceAggregate: {
+      const P::PhysicalPtr left_child = physical_plan->children()[0];
+      if (E::ContainsExprId(left_child->getOutputAttributes(), attribute_id)) {
+        return estimateNumDistinctValues(attribute_id, left_child);
+      }
+      break;
+    }
     case P::PhysicalType::kFilterJoin: {
       const P::FilterJoinPtr &filter_join =
           std::static_pointer_cast<const P::FilterJoin>(physical_plan);
@@ -275,6 +305,17 @@ std::size_t StarSchemaSimpleCostModel::estimateNumDistinctValues(
 double StarSchemaSimpleCostModel::estimateSelectivity(
     const physical::PhysicalPtr &physical_plan) {
   switch (physical_plan->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(physical_plan);
+      return estimateSelectivity(aggregate->input()) *
+          estimateSelectivityForFilterPredicate(aggregate);
+    }
+    case P::PhysicalType::kCrossReferenceCoalesceAggregate: {
+      const P::CrossReferenceCoalesceAggregatePtr &aggregate_on_left_outer_join =
+          std::static_pointer_cast<const P::CrossReferenceCoalesceAggregate>(physical_plan);
+      return estimateSelectivity(aggregate_on_left_outer_join->left_child());
+    }
     case P::PhysicalType::kSelection: {
       const P::SelectionPtr &selection =
           std::static_pointer_cast<const P::Selection>(physical_plan);
@@ -331,6 +372,7 @@ double StarSchemaSimpleCostModel::estimateSelectivity(
 
 double StarSchemaSimpleCostModel::estimateSelectivityForFilterPredicate(
     const physical::PhysicalPtr &physical_plan) {
+  P::PhysicalPtr target_plan = physical_plan;
   E::PredicatePtr filter_predicate = nullptr;
   switch (physical_plan->getPhysicalType()) {
     case P::PhysicalType::kSelection:
@@ -340,6 +382,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForFilterPredicate(
     case P::PhysicalType::kAggregate:
       filter_predicate =
           std::static_pointer_cast<const P::Aggregate>(physical_plan)->filter_predicate();
+      target_plan = physical_plan->children()[0];
       break;
     case P::PhysicalType::kHashJoin:
       filter_predicate =
@@ -356,7 +399,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForFilterPredicate(
   if (filter_predicate == nullptr) {
     return 1.0;
   } else {
-    return estimateSelectivityForPredicate(filter_predicate, physical_plan);
+    return estimateSelectivityForPredicate(filter_predicate, target_plan);
   }
 }
 
@@ -443,6 +486,12 @@ bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
           std::static_pointer_cast<const P::Aggregate>(physical_plan);
       return E::SubsetOfExpressions(aggregate->grouping_expressions(), attributes);
     }
+    case P::PhysicalType::kCrossReferenceCoalesceAggregate: {
+      const P::CrossReferenceCoalesceAggregatePtr &aggregate_on_left_outer_join =
+          std::static_pointer_cast<const P::CrossReferenceCoalesceAggregate>(physical_plan);
+      return E::SubsetOfExpressions(
+          aggregate_on_left_outer_join->left_join_attributes(), attributes);
+    }
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
@@ -542,6 +591,103 @@ attribute_id StarSchemaSimpleCostModel::findCatalogRelationAttributeId(
   return kInvalidAttributeID;
 }
 
+bool StarSchemaSimpleCostModel::canUseCollisionFreeAggregation(
+    const P::AggregatePtr &aggregate,
+    const std::size_t estimated_num_groups,
+    std::size_t *max_num_groups) {
+#ifdef QUICKSTEP_DISTRIBUTED
+  // Currently we cannot do this fast path with the distributed setting. See
+  // the TODOs at InitializeAggregationOperator::getAllWorkOrderProtos() and
+  // FinalizeAggregationOperator::getAllWorkOrderProtos().
+  return false;
+#endif
+
+  // Supports only single group-by key.
+  if (aggregate->grouping_expressions().size() != 1) {
+    return false;
+  }
+
+  // We need to know the exact min/max stats of the group-by key.
+  // So it must be a CatalogAttribute (but not an expression).
+  E::AttributeReferencePtr group_by_key_attr;
+  const E::ExpressionPtr agg_expr = aggregate->grouping_expressions().front();
+  if (!E::SomeAttributeReference::MatchesWithConditionalCast(agg_expr, &group_by_key_attr)) {
+    return false;
+  }
+
+  bool min_value_stat_is_exact;
+  bool max_value_stat_is_exact;
+  const TypedValue min_value = findMinValueStat(
+          aggregate, group_by_key_attr, &min_value_stat_is_exact);
+  const TypedValue max_value = findMaxValueStat(
+          aggregate, group_by_key_attr, &max_value_stat_is_exact);
+  if (min_value.isNull() || max_value.isNull() ||
+      (!min_value_stat_is_exact) || (!max_value_stat_is_exact)) {
+    return false;
+  }
+
+  std::int64_t min_cpp_value;
+  std::int64_t max_cpp_value;
+  switch (group_by_key_attr->getValueType().getTypeID()) {
+    case TypeID::kInt: {
+      min_cpp_value = min_value.getLiteral<int>();
+      max_cpp_value = max_value.getLiteral<int>();
+      break;
+    }
+    case TypeID::kLong: {
+      min_cpp_value = min_value.getLiteral<std::int64_t>();
+      max_cpp_value = max_value.getLiteral<std::int64_t>();
+      break;
+    }
+    default:
+      return false;
+  }
+
+  // TODO(jianqiao):
+  // 1. Handle the case where min_cpp_value is below 0 or far greater than 0.
+  // 2. Reason about the table size bound (e.g. by checking memory size) instead
+  //    of hardcoding it as a gflag.
+  if (min_cpp_value < 0 ||
+      max_cpp_value >= FLAGS_collision_free_vector_table_max_size ||
+      max_cpp_value / static_cast<double>(estimated_num_groups) > 256.0) {
+    return false;
+  }
+
+  for (const auto &agg_expr : aggregate->aggregate_expressions()) {
+    const E::AggregateFunctionPtr agg_func =
+        std::static_pointer_cast<const E::AggregateFunction>(agg_expr->expression());
+
+    if (agg_func->is_distinct()) {
+      return false;
+    }
+
+    // TODO(jianqiao): Support AggregationID::AVG.
+    if (!QUICKSTEP_EQUALS_ANY_CONSTANT(agg_func->getAggregate().getAggregationID(),
+                                       AggregationID::kCount,
+                                       AggregationID::kSum)) {
+      return false;
+    }
+
+    const auto &arguments = agg_func->getArguments();
+    if (arguments.size() > 1u) {
+      return false;
+    }
+
+    if (arguments.size() == 1u) {
+      if (!QUICKSTEP_EQUALS_ANY_CONSTANT(arguments.front()->getValueType().getTypeID(),
+                                         TypeID::kInt,
+                                         TypeID::kLong,
+                                         TypeID::kFloat,
+                                         TypeID::kDouble)) {
+        return false;
+      }
+    }
+  }
+
+  *max_num_groups = static_cast<std::size_t>(max_cpp_value) + 1;
+  return true;
+}
+
 }  // namespace cost
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index cbe18f4..afb2ef9 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -29,6 +29,7 @@
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
 #include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
@@ -166,10 +167,29 @@ class StarSchemaSimpleCostModel : public CostModel {
         physical_plan, attribute->id(), StatType::kMax, is_exact_stat);
   }
 
+  /**
+   * @brief Checks whether an aggregate node can be efficiently evaluated with
+   *        the collision-free aggregation fast path.
+   *
+   * @param aggregate The physical aggregate node to be checked.
+   * @param estimated_num_groups The estimated number of groups for the aggregate.
+   * @param exact_num_groups If collision-free aggregation is applicable, the
+   *        pointed content of this pointer will be set as the maximum possible
+   *        number of groups that the collision-free hash table need to hold.
+   * @return A bool value indicating whether collision-free aggregation can be
+   *         used to evaluate \p aggregate.
+   */
+  bool canUseCollisionFreeAggregation(const physical::AggregatePtr &aggregate,
+                                      const std::size_t estimated_num_groups,
+                                      std::size_t *max_num_groups);
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
 
+  std::size_t estimateCardinalityForCrossReferenceCoalesceAggregate(
+      const physical::CrossReferenceCoalesceAggregatePtr &physical_plan);
+
   std::size_t estimateCardinalityForFilterJoin(
       const physical::FilterJoinPtr &physical_plan);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index f68ed39..77ae75e 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -21,6 +21,9 @@ add_library(quickstep_queryoptimizer_physical_BinaryJoin BinaryJoin.cpp BinaryJo
 add_library(quickstep_queryoptimizer_physical_CopyFrom CopyFrom.cpp CopyFrom.hpp)
 add_library(quickstep_queryoptimizer_physical_CreateIndex CreateIndex.cpp CreateIndex.hpp)
 add_library(quickstep_queryoptimizer_physical_CreateTable CreateTable.cpp CreateTable.hpp)
+add_library(quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
+            CrossReferenceCoalesceAggregate.cpp
+            CrossReferenceCoalesceAggregate.hpp)
 add_library(quickstep_queryoptimizer_physical_DeleteTuples DeleteTuples.cpp DeleteTuples.hpp)
 add_library(quickstep_queryoptimizer_physical_DropTable DropTable.cpp DropTable.hpp)
 add_library(quickstep_queryoptimizer_physical_FilterJoin FilterJoin.cpp FilterJoin.hpp)
@@ -95,6 +98,16 @@ target_link_libraries(quickstep_queryoptimizer_physical_CreateTable
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_physical_DeleteTuples
                       glog
                       quickstep_catalog_CatalogRelation
@@ -293,6 +306,7 @@ target_link_libraries(quickstep_queryoptimizer_physical
                       quickstep_queryoptimizer_physical_CopyFrom
                       quickstep_queryoptimizer_physical_CreateIndex
                       quickstep_queryoptimizer_physical_CreateTable
+                      quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_FilterJoin

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/physical/CrossReferenceCoalesceAggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CrossReferenceCoalesceAggregate.cpp b/query_optimizer/physical/CrossReferenceCoalesceAggregate.cpp
new file mode 100644
index 0000000..6bed215
--- /dev/null
+++ b/query_optimizer/physical/CrossReferenceCoalesceAggregate.cpp
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp"
+
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "utility/Cast.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+std::vector<E::AttributeReferencePtr> CrossReferenceCoalesceAggregate
+    ::getOutputAttributes() const {
+  std::vector<E::AttributeReferencePtr> output_attributes(left_join_attributes_);
+  for (const auto &aggregate_expr : aggregate_expressions_) {
+    output_attributes.emplace_back(E::ToRef(aggregate_expr));
+  }
+  return output_attributes;
+}
+
+std::vector<E::AttributeReferencePtr> CrossReferenceCoalesceAggregate
+    ::getReferencedAttributes() const {
+  std::unordered_set<E::AttributeReferencePtr> referenced_attributes;
+
+  referenced_attributes.insert(left_join_attributes_.begin(),
+                               left_join_attributes_.end());
+  referenced_attributes.insert(right_join_attributes_.begin(),
+                               right_join_attributes_.end());
+
+  if (right_filter_predicate_ != nullptr) {
+    const std::vector<E::AttributeReferencePtr> attrs_in_predicate =
+        right_filter_predicate_->getReferencedAttributes();
+    referenced_attributes.insert(attrs_in_predicate.begin(),
+                                 attrs_in_predicate.end());
+  }
+
+  for (const auto &aggregate_expr : aggregate_expressions_) {
+    const std::vector<E::AttributeReferencePtr> attrs_in_expr =
+        aggregate_expr->getReferencedAttributes();
+    referenced_attributes.insert(attrs_in_expr.begin(), attrs_in_expr.end());
+  }
+
+  return std::vector<E::AttributeReferencePtr>(
+      referenced_attributes.begin(), referenced_attributes.end());
+}
+
+void CrossReferenceCoalesceAggregate::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  inline_field_names->push_back("group_by_key_value_range");
+  inline_field_values->push_back(std::to_string(group_by_key_value_range_));
+
+  non_container_child_field_names->push_back("left_child");
+  non_container_child_fields->push_back(left_child_);
+  non_container_child_field_names->push_back("right_child");
+  non_container_child_fields->push_back(right_child_);
+
+  container_child_field_names->push_back("left_join_attributes");
+  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 (right_filter_predicate_ != nullptr) {
+    non_container_child_field_names->push_back("right_filter_predicate");
+    non_container_child_fields->push_back(right_filter_predicate_);
+  }
+  container_child_field_names->push_back("aggregate_expressions");
+  container_child_fields->push_back(
+      CastSharedPtrVector<OptimizerTreeBase>(aggregate_expressions_));
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp b/query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp
new file mode 100644
index 0000000..44f8a33
--- /dev/null
+++ b/query_optimizer/physical/CrossReferenceCoalesceAggregate.hpp
@@ -0,0 +1,232 @@
+/**
+ * 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_QUERY_OPTIMIZER_PHYSICAL_CROSS_REFERENCE_COALESCE_AGGREGATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_CROSS_REFERENCE_COALESCE_AGGREGATE_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerLogical
+ *  @{
+ */
+
+class CrossReferenceCoalesceAggregate;
+typedef std::shared_ptr<const CrossReferenceCoalesceAggregate> CrossReferenceCoalesceAggregatePtr;
+
+/**
+ * @brief A physical node that fuses a HashJoin with an Aggregate to enable
+ *        fast-path execution.
+ *
+ * Below we briefly describe the semantics of this physical node.
+ *
+ * Let L be a table with PRIMARY KEY u. Let R be a table with FOREIGN KEY x
+ * referring to L(u). Then CrossReferenceCoalesceAggregate represents a common
+ * class of analytical queries that
+ * - For each u in L, COUNT/SUM the records in R that correspond to u (i.e.
+ *   those records satisfying R.x = L.u).
+ *   In the case that there is no record for u in R, use 0 as the result value.
+ *
+ * And we have the mapping:
+ *   L -> left_child_
+ *   R -> right_child_
+ *   u -> left_join_attributes_
+ *   x -> right_join_attributes_
+ *   COUNT/SUM -> aggregate_expressions_
+ */
+class CrossReferenceCoalesceAggregate : public Physical {
+ public:
+  PhysicalType getPhysicalType() const override {
+    return PhysicalType::kCrossReferenceCoalesceAggregate;
+  }
+
+  std::string getName() const override {
+    return "CrossReferenceCoalesceAggregate";
+  }
+
+  /**
+   * @return The left physical child.
+   */
+  const PhysicalPtr& left_child() const {
+    return left_child_;
+  }
+
+  /**
+   * @return The right physical child.
+   */
+  const PhysicalPtr& right_child() const {
+    return right_child_;
+  }
+
+  /**
+   * @return The left join attributes.
+   */
+  const std::vector<expressions::AttributeReferencePtr>& left_join_attributes() const {
+    return left_join_attributes_;
+  }
+
+  /**
+   * @return The right join attributes.
+   */
+  const std::vector<expressions::AttributeReferencePtr>& right_join_attributes() const {
+    return right_join_attributes_;
+  }
+
+  /**
+   * @return The predicate to be applied to the right child before aggregation.
+   */
+  const expressions::PredicatePtr& right_filter_predicate() const {
+    return right_filter_predicate_;
+  }
+
+  /**
+   * @return Aggregate expressions.
+   */
+  const std::vector<expressions::AliasPtr>& aggregate_expressions() const {
+    return aggregate_expressions_;
+  }
+
+  /**
+   * @return The maximum possible value of the group-by keys when mapped to
+   *         integer.
+   */
+  std::size_t group_by_key_value_range() const {
+    return group_by_key_value_range_;
+  }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override {
+    DCHECK_EQ(getNumChildren(), new_children.size());
+    return Create(new_children[0],
+                  new_children[1],
+                  left_join_attributes_,
+                  right_join_attributes_,
+                  right_filter_predicate_,
+                  aggregate_expressions_,
+                  group_by_key_value_range_);
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override;
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  bool maybeCopyWithPrunedExpressions(
+      const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+      PhysicalPtr *output) const override {
+    return false;
+  }
+
+  /**
+   * @brief Creates a physical CrossReferenceCoalesceAggregate.
+   *
+   * @param left_child The left child.
+   * @param right_child The right child.
+   * @param left_join_attributes The join attributes of the left child.
+   * @param right_join_attributes The join attributes of the right child.
+   * @param right_filter_predicate Optional filtering predicate evaluated on
+   *        the left child before aggregation.
+   * @param aggregate_expressions The aggregate expressions.
+   * @param group_by_key_value_range The maximum possible value of the group-by
+   *        keys when mapped to integer.
+   * @return An immutable physical CrossReferenceCoalesceAggregate.
+   */
+  static CrossReferenceCoalesceAggregatePtr Create(
+      const PhysicalPtr &left_child,
+      const PhysicalPtr &right_child,
+      const std::vector<expressions::AttributeReferencePtr> &left_join_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
+      const expressions::PredicatePtr right_filter_predicate,
+      const std::vector<expressions::AliasPtr> &aggregate_expressions,
+      const std::size_t group_by_key_value_range) {
+    return CrossReferenceCoalesceAggregatePtr(
+        new CrossReferenceCoalesceAggregate(left_child,
+                                            right_child,
+                                            left_join_attributes,
+                                            right_join_attributes,
+                                            right_filter_predicate,
+                                            aggregate_expressions,
+                                            group_by_key_value_range));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  CrossReferenceCoalesceAggregate(
+      const PhysicalPtr &left_child,
+      const PhysicalPtr &right_child,
+      const std::vector<expressions::AttributeReferencePtr> &left_join_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
+      const expressions::PredicatePtr right_filter_predicate,
+      const std::vector<expressions::AliasPtr> &aggregate_expressions,
+      const std::size_t group_by_key_value_range)
+      : left_child_(left_child),
+        right_child_(right_child),
+        left_join_attributes_(left_join_attributes),
+        right_join_attributes_(right_join_attributes),
+        right_filter_predicate_(right_filter_predicate),
+        aggregate_expressions_(aggregate_expressions),
+        group_by_key_value_range_(group_by_key_value_range) {
+    addChild(left_child_);
+    addChild(right_child_);
+  }
+
+  // TODO(jianqiao): For the left child, support filter predicate fusing and
+  // attachment of LIPFilters.
+  PhysicalPtr left_child_;
+  PhysicalPtr right_child_;
+  std::vector<expressions::AttributeReferencePtr> left_join_attributes_;
+  std::vector<expressions::AttributeReferencePtr> right_join_attributes_;
+  expressions::PredicatePtr right_filter_predicate_;
+  std::vector<expressions::AliasPtr> aggregate_expressions_;
+  std::size_t group_by_key_value_range_;
+
+  DISALLOW_COPY_AND_ASSIGN(CrossReferenceCoalesceAggregate);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_CROSS_REFERENCE_COALESCE_AGGREGATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/physical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PatternMatcher.hpp b/query_optimizer/physical/PatternMatcher.hpp
index 4336767..0204504 100644
--- a/query_optimizer/physical/PatternMatcher.hpp
+++ b/query_optimizer/physical/PatternMatcher.hpp
@@ -33,6 +33,7 @@ class Aggregate;
 class BinaryJoin;
 class CopyFrom;
 class CreateTable;
+class CrossReferenceCoalesceAggregate;
 class DeleteTuples;
 class DropTable;
 class FilterJoin;
@@ -112,6 +113,8 @@ using SomeAggregate = SomePhysicalNode<Aggregate, PhysicalType::kAggregate>;
 using SomeBinaryJoin = SomePhysicalNode<BinaryJoin, PhysicalType::kHashJoin, PhysicalType::kNestedLoopsJoin>;
 using SomeCopyFrom = SomePhysicalNode<CopyFrom, PhysicalType::kCopyFrom>;
 using SomeCreateTable = SomePhysicalNode<CreateTable, PhysicalType::kCreateTable>;
+using SomeCrossReferenceCoalesceAggregate = SomePhysicalNode<CrossReferenceCoalesceAggregate,
+                                                             PhysicalType::kCrossReferenceCoalesceAggregate>;
 using SomeDeleteTuples = SomePhysicalNode<DeleteTuples, PhysicalType::kDeleteTuples>;
 using SomeDropTable = SomePhysicalNode<DropTable, PhysicalType::kDropTable>;
 using SomeFilterJoin = SomePhysicalNode<FilterJoin, PhysicalType::kFilterJoin>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index 1da5929..077bd54 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -36,6 +36,7 @@ enum class PhysicalType {
   kCopyFrom,
   kCreateIndex,
   kCreateTable,
+  kCrossReferenceCoalesceAggregate,
   kDeleteTuples,
   kDropTable,
   kFilterJoin,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/rules/BottomUpRule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/BottomUpRule.hpp b/query_optimizer/rules/BottomUpRule.hpp
index 53dff0d..6c14e64 100644
--- a/query_optimizer/rules/BottomUpRule.hpp
+++ b/query_optimizer/rules/BottomUpRule.hpp
@@ -57,21 +57,7 @@ class BottomUpRule : public Rule<TreeType> {
     DCHECK(tree != nullptr);
 
     init(tree);
-    std::vector<std::shared_ptr<const TreeType>> new_children;
-    bool has_changed_children = false;
-    for (const std::shared_ptr<const TreeType> &child : tree->children()) {
-      std::shared_ptr<const TreeType> new_child = apply(child);
-      if (child != new_child && !has_changed_children) {
-        has_changed_children = true;
-      }
-      new_children.push_back(new_child);
-    }
-
-    if (has_changed_children) {
-      return applyToNode(tree->copyWithNewChildren(new_children));
-    } else {
-      return applyToNode(tree);
-    }
+    return applyInternal(tree);
   }
 
  protected:
@@ -89,10 +75,29 @@ class BottomUpRule : public Rule<TreeType> {
    *
    * @param input The input tree.
    */
-  virtual void init(const TreeNodePtr &input) {
-  }
+  virtual void init(const TreeNodePtr &input) {}
 
  private:
+  TreeNodePtr applyInternal(const TreeNodePtr &tree) {
+    DCHECK(tree != nullptr);
+
+    std::vector<std::shared_ptr<const TreeType>> new_children;
+    bool has_changed_children = false;
+    for (const std::shared_ptr<const TreeType> &child : tree->children()) {
+      std::shared_ptr<const TreeType> new_child = applyInternal(child);
+      if (child != new_child && !has_changed_children) {
+        has_changed_children = true;
+      }
+      new_children.push_back(new_child);
+    }
+
+    if (has_changed_children) {
+      return applyToNode(tree->copyWithNewChildren(new_children));
+    } else {
+      return applyToNode(tree);
+    }
+  }
+
   DISALLOW_COPY_AND_ASSIGN(BottomUpRule);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 029d816..427500d 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -21,6 +21,7 @@ add_subdirectory(tests)
 add_library(quickstep_queryoptimizer_rules_AttachLIPFilters AttachLIPFilters.cpp AttachLIPFilters.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_FuseAggregateJoin FuseAggregateJoin.cpp FuseAggregateJoin.hpp)
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
 add_library(quickstep_queryoptimizer_rules_InjectJoinFilters InjectJoinFilters.cpp InjectJoinFilters.hpp)
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
@@ -75,6 +76,27 @@ target_link_libraries(quickstep_queryoptimizer_rules_CollapseProject
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_FuseAggregateJoin
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AggregateFunction
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_CrossReferenceCoalesceAggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_BottomUpRule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_GenerateJoins
                       glog
                       quickstep_queryoptimizer_expressions_AttributeReference
@@ -288,6 +310,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_BottomUpRule
                       quickstep_queryoptimizer_rules_CollapseProject
+                      quickstep_queryoptimizer_rules_FuseAggregateJoin
                       quickstep_queryoptimizer_rules_GenerateJoins
                       quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/rules/FuseAggregateJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/FuseAggregateJoin.cpp b/query_optimizer/rules/FuseAggregateJoin.cpp
new file mode 100644
index 0000000..6efc7e8
--- /dev/null
+++ b/query_optimizer/rules/FuseAggregateJoin.cpp
@@ -0,0 +1,170 @@
+/**
+ * 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 "query_optimizer/rules/FuseAggregateJoin.hpp"
+
+#include <algorithm>
+#include <cstddef>
+#include <unordered_set>
+#include <vector>
+
+#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"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/CrossReferenceCoalesceAggregate.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/Selection.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 FuseAggregateJoin::applyToNode(
+    const P::PhysicalPtr &node) {
+  // Currently we consider only Aggregate on HashLeftOuterJoin.
+  P::AggregatePtr aggregate;
+  if (!P::SomeAggregate::MatchesWithConditionalCast(node, &aggregate) ||
+      aggregate->filter_predicate() != nullptr) {
+    return node;
+  }
+
+  P::HashJoinPtr hash_join;
+  if ((!P::SomeHashJoin::MatchesWithConditionalCast(aggregate->input(), &hash_join)) ||
+      hash_join->join_type() != P::HashJoin::JoinType::kLeftOuterJoin ||
+      hash_join->residual_predicate() != nullptr) {
+    return node;
+  }
+
+  // Single left join attribute with unique values.
+  const std::vector<E::AttributeReferencePtr> &left_join_attributes =
+      hash_join->left_join_attributes();
+  if (left_join_attributes.size() != 1u ||
+      (!cost_model_->impliesUniqueAttributes(hash_join->left(), left_join_attributes))) {
+    return node;
+  }
+
+  // Single group-by attribute that is the same as the right join attribute.
+  const std::vector<E::NamedExpressionPtr> &grouping_expressions =
+      aggregate->grouping_expressions();
+  if (grouping_expressions.size() != 1u ||
+      grouping_expressions.front()->id() != left_join_attributes.front()->id()) {
+    return node;
+  }
+
+  std::unordered_set<E::ExprId> right_side_attr_ids;
+  for (const auto &attr : hash_join->right()->getOutputAttributes()) {
+    right_side_attr_ids.insert(attr->id());
+  }
+
+  // Aggregate expressions only depend on attributes from the right child.
+  const std::vector<E::AliasPtr> &aggregate_expressions =
+      aggregate->aggregate_expressions();
+  for (const auto &expr : aggregate_expressions) {
+    const E::AggregateFunctionPtr aggr_expr =
+        std::static_pointer_cast<const E::AggregateFunction>(expr->expression());
+
+    const std::vector<E::ScalarPtr> &arguments = aggr_expr->getArguments();
+    if (arguments.size() != 1u) {
+      return node;
+    }
+
+    E::AttributeReferencePtr arg_attr;
+    if (!E::SomeAttributeReference::MatchesWithConditionalCast(arguments.front(), &arg_attr) ||
+        right_side_attr_ids.find(arg_attr->id()) == right_side_attr_ids.end()) {
+      return node;
+    }
+  }
+
+  // Collision-free vector aggregation is applicable, and both the left and right
+  // join attributes are range-bounded integer values.
+  const std::size_t estimated_num_groups =
+      cost_model_->estimateNumGroupsForAggregate(aggregate);
+
+  std::size_t max_num_groups_left;
+  if (!cost_model_->canUseCollisionFreeAggregation(aggregate,
+                                                   estimated_num_groups,
+                                                   &max_num_groups_left)) {
+    return node;
+  }
+
+  std::size_t max_num_groups_right;
+  if (!cost_model_->canUseCollisionFreeAggregation(
+           P::Aggregate::Create(hash_join->right(),
+                                E::ToNamedExpressions(hash_join->right_join_attributes()),
+                                aggregate->aggregate_expressions(),
+                                nullptr),
+           estimated_num_groups,
+           &max_num_groups_right)) {
+    return node;
+  }
+
+  // Fuse right child's filter predicate.
+  P::PhysicalPtr right_child = hash_join->right();
+  const std::vector<E::AttributeReferencePtr> &right_join_attributes =
+      hash_join->right_join_attributes();
+  E::PredicatePtr right_filter_predicate = nullptr;
+
+  P::SelectionPtr selection;
+  if (P::SomeSelection::MatchesWithConditionalCast(right_child, &selection)) {
+    if (E::SubsetOfExpressions(right_join_attributes,
+                               selection->input()->getOutputAttributes())) {
+      right_child = selection->input();
+      right_filter_predicate = selection->filter_predicate();
+    }
+  }
+
+  const std::size_t max_num_groups =
+      std::max(max_num_groups_left, max_num_groups_right);
+
+  return P::CrossReferenceCoalesceAggregate::Create(hash_join->left(),
+                                                    right_child,
+                                                    left_join_attributes,
+                                                    right_join_attributes,
+                                                    right_filter_predicate,
+                                                    aggregate_expressions,
+                                                    max_num_groups);
+}
+
+void FuseAggregateJoin::init(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  const P::TopLevelPlanPtr top_level_plan =
+      std::static_pointer_cast<const P::TopLevelPlan>(input);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(top_level_plan->shared_subplans()));
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/query_optimizer/rules/FuseAggregateJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/FuseAggregateJoin.hpp b/query_optimizer/rules/FuseAggregateJoin.hpp
new file mode 100644
index 0000000..f2d4c47
--- /dev/null
+++ b/query_optimizer/rules/FuseAggregateJoin.hpp
@@ -0,0 +1,71 @@
+/**
+ * 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_QUERY_OPTIMIZER_RULES_FUSE_AGGREGATE_JOIN_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_FUSE_AGGREGATE_JOIN_HPP_
+
+#include <memory>
+#include <string>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/BottomUpRule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to fuse Aggregate nodes with
+ *        HashJoin nodes.
+ */
+class FuseAggregateJoin : public BottomUpRule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  FuseAggregateJoin() {}
+
+  ~FuseAggregateJoin() override {}
+
+  std::string getName() const override {
+    return "FuseAggregateJoin";
+  }
+
+ protected:
+  physical::PhysicalPtr applyToNode(const physical::PhysicalPtr &node) override;
+
+  void init(const physical::PhysicalPtr &input) override;
+
+ private:
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  DISALLOW_COPY_AND_ASSIGN(FuseAggregateJoin);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_FUSE_AGGREGATE_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a28b1e4d/relational_operators/BuildAggregationExistenceMapOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildAggregationExistenceMapOperator.cpp b/relational_operators/BuildAggregationExistenceMapOperator.cpp
new file mode 100644
index 0000000..648e291
--- /dev/null
+++ b/relational_operators/BuildAggregationExistenceMapOperator.cpp
@@ -0,0 +1,196 @@
+/**
+ * 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 "relational_operators/BuildAggregationExistenceMapOperator.hpp"
+
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogAttribute.hpp"
+#include "catalog/CatalogRelationSchema.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
+#include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
+#include "storage/AggregationOperationState.hpp"
+#include "storage/CollisionFreeVectorTable.hpp"
+#include "storage/StorageBlock.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/TupleStorageSubBlock.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+namespace {
+
+template <typename CppType, bool is_attr_nullable>
+void ExecuteBuild(const attribute_id attr_id,
+                  ValueAccessor *accessor,
+                  BarrieredReadWriteConcurrentBitVector *existence_map) {
+  InvokeOnAnyValueAccessor(
+      accessor,
+      [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+    accessor->beginIteration();
+    while (accessor->next()) {
+      const void *value = accessor->template getUntypedValue<is_attr_nullable>(attr_id);
+      if (!is_attr_nullable || value != nullptr) {
+        existence_map->setBit(*reinterpret_cast<const CppType *>(value));
+      }
+    }
+  });
+}
+
+// Dispatch helper.
+template <typename CppType>
+void ExecuteHelper(const attribute_id attr_id,
+                   const bool is_attr_nullable,
+                   ValueAccessor *accessor,
+                   BarrieredReadWriteConcurrentBitVector *existence_map)  {
+  if (is_attr_nullable) {
+    ExecuteBuild<CppType, true>(attr_id, accessor, existence_map);
+  } else {
+    ExecuteBuild<CppType, false>(attr_id, accessor, existence_map);
+  }
+}
+
+}  // namespace
+
+bool BuildAggregationExistenceMapOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addNormalWorkOrder(
+            new BuildAggregationExistenceMapWorkOrder(
+                query_id_,
+                input_relation_,
+                input_block_id,
+                build_attribute_,
+                query_context->getAggregationState(aggr_state_index_),
+                storage_manager),
+            op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addNormalWorkOrder(
+          new BuildAggregationExistenceMapWorkOrder(
+                query_id_,
+                input_relation_,
+                input_relation_block_ids_[num_workorders_generated_],
+                build_attribute_,
+                query_context->getAggregationState(aggr_state_index_),
+                storage_manager),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+bool BuildAggregationExistenceMapOperator
+    ::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id block : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(block), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* BuildAggregationExistenceMapOperator
+    ::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::BUILD_LIP_FILTER);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::BuildAggregationExistenceMapWorkOrder::relation_id,
+                      input_relation_.getID());
+  proto->SetExtension(serialization::BuildAggregationExistenceMapWorkOrder::build_block_id,
+                      block);
+  proto->SetExtension(serialization::BuildAggregationExistenceMapWorkOrder::build_attribute,
+                      build_attribute_);
+  proto->SetExtension(serialization::BuildAggregationExistenceMapWorkOrder::aggr_state_index,
+                      aggr_state_index_);
+  return proto;
+}
+
+void BuildAggregationExistenceMapWorkOrder::execute() {
+  BlockReference block(
+      storage_manager_->getBlock(build_block_id_, input_relation_));
+  std::unique_ptr<ValueAccessor> accessor(
+      block->getTupleStorageSubBlock().createValueAccessor());
+
+  CollisionFreeVectorTable *aggregate_table =
+      state_->getCollisionFreeVectorTable();
+  DCHECK(aggregate_table != nullptr);
+
+  BarrieredReadWriteConcurrentBitVector *existence_map =
+      aggregate_table->getExistenceMap();
+
+  const Type &attr_type =
+      input_relation_.getAttributeById(build_attribute_)->getType();
+  switch (attr_type.getTypeID()) {
+    case TypeID::kInt:
+      ExecuteHelper<int>(build_attribute_,
+                         attr_type.isNullable(),
+                         accessor.get(),
+                         existence_map);
+      return;
+    case TypeID::kLong:
+      ExecuteHelper<std::int64_t>(build_attribute_,
+                                  attr_type.isNullable(),
+                                  accessor.get(),
+                                  existence_map);
+      return;
+    default:
+      LOG(FATAL) << "Build attribute type not supported by "
+                 << "BuildAggregationExistenceMapOperator: "
+                 << attr_type.getName();
+  }
+}
+
+}  // namespace quickstep


[50/50] [abbrv] 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/0ba3eb69
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/0ba3eb69
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/0ba3eb69

Branch: refs/heads/LIP-time-decomposition
Commit: 0ba3eb695a3ef17409ef1b64c08ac37f5ba7773b
Parents: f6480fb
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Feb 20 21:38:55 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Feb 20 21:38:55 2017 -0600

----------------------------------------------------------------------
 cli/QuickstepCli.cpp                           |  15 ++
 relational_operators/BuildHashOperator.cpp     |   9 +
 relational_operators/CMakeLists.txt            |   2 +
 relational_operators/HashJoinOperator.cpp      |   7 +
 utility/CMakeLists.txt                         |   4 +
 utility/EventProfiler.cpp                      |  30 ++++
 utility/EventProfiler.hpp                      | 190 ++++++++++++++++++++
 utility/lip_filter/CMakeLists.txt              |   1 +
 utility/lip_filter/LIPFilterAdaptiveProber.hpp |  10 +-
 9 files changed, 266 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index eddee8c..17bfba9 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -23,6 +23,7 @@
 #include <cstddef>
 #include <cstdio>
 #include <exception>
+#include <fstream>
 #include <memory>
 #include <string>
 #include <utility>
@@ -69,6 +70,7 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "storage/StorageConstants.hpp"
 #include "storage/StorageManager.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/ExecutionDAGVisualizer.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
@@ -156,6 +158,8 @@ 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 dumping the profiled events.");
 
 DECLARE_bool(profile_and_report_workorder_perf);
 DECLARE_bool(visualize_execution_dag);
@@ -344,6 +348,7 @@ int main(int argc, char* argv[]) {
         const std::size_t query_id = query_processor->query_id();
         const CatalogRelation *query_result_relation = nullptr;
         std::unique_ptr<quickstep::ExecutionDAGVisualizer> dag_visualizer;
+        auto *event_container = quickstep::simple_profiler.getContainer();
 
         try {
           auto query_handle = std::make_unique<QueryHandle>(query_id,
@@ -360,6 +365,8 @@ int main(int argc, char* argv[]) {
           query_result_relation = query_handle->getQueryResultRelation();
 
           start = std::chrono::steady_clock::now();
+          quickstep::simple_profiler.clear();
+          event_container->startEvent("overall");
           QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
               main_thread_client_id,
               foreman.getBusClientID(),
@@ -374,6 +381,7 @@ int main(int argc, char* argv[]) {
         try {
           QueryExecutionUtil::ReceiveQueryCompletionMessage(
               main_thread_client_id, &bus);
+          event_container->endEvent("overall");
           end = std::chrono::steady_clock::now();
 
           if (query_result_relation) {
@@ -405,6 +413,13 @@ int main(int argc, char* argv[]) {
             dag_visualizer->bindProfilingStats(profiling_stats);
             std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
           }
+          if (!quickstep::FLAGS_profile_output.empty()) {
+            std::ofstream ofs(
+                quickstep::FLAGS_profile_output + std::to_string(query_processor->query_id()),
+                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/0ba3eb69/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 8f40fbb..0ec2915 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -34,6 +34,7 @@
 #include "storage/TupleReference.hpp"
 #include "storage/TupleStorageSubBlock.hpp"
 #include "storage/ValueAccessor.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/lip_filter/LIPFilterBuilder.hpp"
 #include "utility/lip_filter/LIPFilterUtil.hpp"
 
@@ -157,12 +158,19 @@ void BuildHashWorkOrder::execute() {
   TupleReferenceGenerator generator(build_block_id_);
   std::unique_ptr<ValueAccessor> accessor(block->getTupleStorageSubBlock().createValueAccessor());
 
+  auto *container = simple_profiler.getContainer();
+
   // Build LIPFilters if enabled.
   if (lip_filter_builder_ != nullptr) {
+    auto *event_lip = container->getEventLine("BuildLIP");
+    event_lip->emplace_back();
     lip_filter_builder_->insertValueAccessor(accessor.get());
     accessor->beginIterationVirtual();
+    event_lip->back().endEvent();
   }
 
+  auto *event_hash = container->getEventLine("BuildHash");
+  event_hash->emplace_back();
   HashTablePutResult result;
   if (join_key_attributes_.size() == 1) {
     result = hash_table_->putValueAccessor(accessor.get(),
@@ -175,6 +183,7 @@ void BuildHashWorkOrder::execute() {
                                                        any_join_key_attributes_nullable_,
                                                        &generator);
   }
+  event_hash->back().endEvent();
 
   CHECK(result == HashTablePutResult::kOK)
       << "Failed to add entries to join hash table.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 457d58a..fd44cfc 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -141,6 +141,7 @@ target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       quickstep_storage_TupleReference
                       quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       quickstep_utility_lipfilter_LIPFilterBuilder
                       quickstep_utility_lipfilter_LIPFilterUtil
@@ -281,6 +282,7 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_utility_lipfilter_LIPFilterUtil

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 0e75411..3f6eaab 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -50,6 +50,7 @@
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
 #include "utility/lip_filter/LIPFilterUtil.hpp"
 
@@ -470,11 +471,17 @@ void HashInnerJoinWorkOrder::execute() {
         base_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
   }
 
+  auto *container = simple_profiler.getContainer();
+  auto *event_hash = container->getEventLine("ProbeHash");
+  event_hash->emplace_back();
+
   if (probe_accessor->getImplementationType() == ValueAccessor::Implementation::kSplitRowStore) {
     executeWithCopyElision(probe_accessor.get());
   } else {
     executeWithoutCopyElision(probe_accessor.get());
   }
+
+  event_hash->back().endEvent();
 }
 
 void HashInnerJoinWorkOrder::executeWithoutCopyElision(ValueAccessor *probe_accessor) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index ca04462..c4f8160 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -178,6 +178,7 @@ add_library(quickstep_utility_BarrieredReadWriteConcurrentBitVector
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
+add_library(quickstep_utility_EventProfiler EventProfiler.cpp EventProfiler.hpp)
 add_library(quickstep_utility_ExecutionDAGVisualizer
             ExecutionDAGVisualizer.cpp
             ExecutionDAGVisualizer.hpp)
@@ -249,6 +250,8 @@ target_link_libraries(quickstep_utility_DAG
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_DisjointTreeForest
                       glog)
+target_link_libraries(quickstep_utility_EventProfiler
+                      quickstep_threading_Mutex)
 target_link_libraries(quickstep_utility_ExecutionDAGVisualizer
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_queryexecution_QueryExecutionTypedefs
@@ -348,6 +351,7 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_DAG
                       quickstep_utility_DisjointTreeForest
                       quickstep_utility_EqualsAnyConstant
+                      quickstep_utility_EventProfiler
                       quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Glob
                       quickstep_utility_HashPair

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/utility/EventProfiler.hpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.hpp b/utility/EventProfiler.hpp
new file mode 100644
index 0000000..c28f49b
--- /dev/null
+++ b/utility/EventProfiler.hpp
@@ -0,0 +1,190 @@
+/**
+ * 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_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 at thread " << thread_id
+              << ": " << event_group.first;
+
+          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, 0> {
+    static void Print(std::ostream &os, const Tuple &t, const std::string &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<std::string> simple_profiler;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/utility/lip_filter/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/lip_filter/CMakeLists.txt b/utility/lip_filter/CMakeLists.txt
index 519d3e9..4f7dd9a 100644
--- a/utility/lip_filter/CMakeLists.txt
+++ b/utility/lip_filter/CMakeLists.txt
@@ -51,6 +51,7 @@ target_link_libraries(quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_storage_TupleIdSequence
                       quickstep_storage_ValueAccessor
                       quickstep_types_Type
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       quickstep_utility_lipfilter_LIPFilter)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilterBuilder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0ba3eb69/utility/lip_filter/LIPFilterAdaptiveProber.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterAdaptiveProber.hpp b/utility/lip_filter/LIPFilterAdaptiveProber.hpp
index e1a75d6..b2f498f 100644
--- a/utility/lip_filter/LIPFilterAdaptiveProber.hpp
+++ b/utility/lip_filter/LIPFilterAdaptiveProber.hpp
@@ -31,6 +31,7 @@
 #include "storage/TupleIdSequence.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "types/Type.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/Macros.hpp"
 #include "utility/lip_filter/LIPFilter.hpp"
 
@@ -87,12 +88,17 @@ class LIPFilterAdaptiveProber {
    * @return A TupleIdSequence for the hit tuples in the ValueAccessor.
    */
   TupleIdSequence* filterValueAccessor(ValueAccessor *accessor) {
+    TupleIdSequence *matches;
     const TupleIdSequence *existence_map = accessor->getTupleIdSequenceVirtual();
+    auto *event_lip = simple_profiler.getContainer()->getEventLine("ProbeLIP");
+    event_lip->emplace_back();
     if (existence_map == nullptr) {
-      return filterValueAccessorNoExistenceMap(accessor);
+      matches = filterValueAccessorNoExistenceMap(accessor);
     } else {
-      return filterValueAccessorWithExistenceMap(accessor, existence_map);
+      matches = filterValueAccessorWithExistenceMap(accessor, existence_map);
     }
+    event_lip->back().endEvent();
+    return matches;
   }
 
  private:


[26/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index b942c1b..0b34908 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -20,7 +20,7 @@
 #include "storage/AggregationOperationState.hpp"
 
 #include <cstddef>
-#include <cstdio>
+#include <cstdint>
 #include <memory>
 #include <string>
 #include <utility>
@@ -34,29 +34,32 @@
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunctionFactory.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "expressions/aggregation/AggregationHandleDistinct.hpp"
-#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/AggregationOperationState.pb.h"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/CollisionFreeVectorTable.hpp"
 #include "storage/HashTableFactory.hpp"
+#include "storage/HashTableBase.hpp"
 #include "storage/InsertDestination.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/SubBlocksReference.hpp"
 #include "storage/TupleIdSequence.hpp"
+#include "storage/TupleStorageSubBlock.hpp"
 #include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.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/lip_filter/LIPFilterAdaptiveProber.hpp"
 
-#include "glog/logging.h"
+#include "gflags/gflags.h"
 
-using std::unique_ptr;
+#include "glog/logging.h"
 
 namespace quickstep {
 
@@ -80,148 +83,145 @@ AggregationOperationState::AggregationOperationState(
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
     : input_relation_(input_relation),
-      is_aggregate_partitioned_(checkAggregatePartitioned(
-          estimated_num_entries, is_distinct, group_by, aggregate_functions)),
+      is_aggregate_collision_free_(false),
+      is_aggregate_partitioned_(false),
       predicate_(predicate),
-      group_by_list_(std::move(group_by)),
-      arguments_(std::move(arguments)),
       is_distinct_(std::move(is_distinct)),
       storage_manager_(storage_manager) {
+  if (!group_by.empty()) {
+    if (hash_table_impl_type == HashTableImplType::kCollisionFreeVector) {
+      is_aggregate_collision_free_ = true;
+    } else {
+      is_aggregate_partitioned_ = checkAggregatePartitioned(
+          estimated_num_entries, is_distinct_, group_by, aggregate_functions);
+    }
+  }
+
   // Sanity checks: each aggregate has a corresponding list of arguments.
-  DCHECK(aggregate_functions.size() == arguments_.size());
+  DCHECK(aggregate_functions.size() == arguments.size());
 
   // Get the types of GROUP BY expressions for creating HashTables below.
-  std::vector<const Type *> group_by_types;
-  for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
-    group_by_types.emplace_back(&group_by_element->getType());
+  for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
+    group_by_types_.emplace_back(&group_by_element->getType());
+  }
+
+  // Prepare group-by key ids and non-trivial expressions.
+  for (std::unique_ptr<const Scalar> &group_by_element : group_by) {
+    const attribute_id attr_id =
+        group_by_element->getAttributeIdForValueAccessor();
+    if (attr_id != kInvalidAttributeID) {
+      group_by_key_ids_.emplace_back(ValueAccessorSource::kBase, attr_id);
+    } else {
+      group_by_key_ids_.emplace_back(ValueAccessorSource::kDerived,
+                                     non_trivial_expressions_.size());
+      non_trivial_expressions_.emplace_back(group_by_element.release());
+    }
   }
 
   std::vector<AggregationHandle *> group_by_handles;
-  group_by_handles.clear();
-
-  if (aggregate_functions.size() == 0) {
-    // If there is no aggregation function, then it is a distinctify operation
-    // on the group-by expressions.
-    DCHECK_GT(group_by_list_.size(), 0u);
-
-    handles_.emplace_back(new AggregationHandleDistinct());
-    arguments_.push_back({});
-    is_distinct_.emplace_back(false);
-    group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
-                                                     hash_table_impl_type,
-                                                     group_by_types,
-                                                     {1},
-                                                     handles_,
-                                                     storage_manager));
-  } else {
-    // Set up each individual aggregate in this operation.
-    std::vector<const AggregateFunction *>::const_iterator agg_func_it =
-        aggregate_functions.begin();
-    std::vector<std::vector<std::unique_ptr<const Scalar>>>::const_iterator
-        args_it = arguments_.begin();
-    std::vector<bool>::const_iterator is_distinct_it = is_distinct_.begin();
-    std::vector<HashTableImplType>::const_iterator
-        distinctify_hash_table_impl_types_it =
-            distinctify_hash_table_impl_types.begin();
-    std::vector<std::size_t> payload_sizes;
-    for (; agg_func_it != aggregate_functions.end();
-         ++agg_func_it, ++args_it, ++is_distinct_it) {
-      // Get the Types of this aggregate's arguments so that we can create an
-      // AggregationHandle.
-      std::vector<const Type *> argument_types;
-      for (const std::unique_ptr<const Scalar> &argument : *args_it) {
-        argument_types.emplace_back(&argument->getType());
-      }
 
-      // Sanity checks: aggregate function exists and can apply to the specified
-      // arguments.
-      DCHECK(*agg_func_it != nullptr);
-      DCHECK((*agg_func_it)->canApplyToTypes(argument_types));
-
-      // Have the AggregateFunction create an AggregationHandle that we can use
-      // to do actual aggregate computation.
-      handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
-
-      if (!group_by_list_.empty()) {
-        // Aggregation with GROUP BY: combined payload is partially updated in
-        // the presence of DISTINCT.
-        if (*is_distinct_it) {
-          handles_.back()->blockUpdate();
-        }
-        group_by_handles.emplace_back(handles_.back());
-        payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
+  // Set up each individual aggregate in this operation.
+  std::vector<const AggregateFunction *>::const_iterator agg_func_it =
+      aggregate_functions.begin();
+  std::vector<std::vector<std::unique_ptr<const Scalar>>>::iterator
+      args_it = arguments.begin();
+  std::vector<bool>::const_iterator is_distinct_it = is_distinct_.begin();
+  std::vector<HashTableImplType>::const_iterator
+      distinctify_hash_table_impl_types_it =
+          distinctify_hash_table_impl_types.begin();
+  for (; agg_func_it != aggregate_functions.end();
+       ++agg_func_it, ++args_it, ++is_distinct_it) {
+    // Get the Types of this aggregate's arguments so that we can create an
+    // AggregationHandle.
+    std::vector<const Type *> argument_types;
+    for (const std::unique_ptr<const Scalar> &argument : *args_it) {
+      argument_types.emplace_back(&argument->getType());
+    }
+
+    // Prepare argument attribute ids and non-trivial expressions.
+    std::vector<MultiSourceAttributeId> argument_ids;
+    for (std::unique_ptr<const Scalar> &argument : *args_it) {
+      const attribute_id attr_id =
+          argument->getAttributeIdForValueAccessor();
+      if (attr_id != kInvalidAttributeID) {
+        argument_ids.emplace_back(ValueAccessorSource::kBase, attr_id);
       } else {
-        // Aggregation without GROUP BY: create a single global state.
-        single_states_.emplace_back(handles_.back()->createInitialState());
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-        // See if all of this 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 aggregation.
-        std::vector<attribute_id> local_arguments_as_attributes;
-        local_arguments_as_attributes.reserve(args_it->size());
-        for (const std::unique_ptr<const Scalar> &argument : *args_it) {
-          const attribute_id argument_id =
-              argument->getAttributeIdForValueAccessor();
-          if (argument_id == -1) {
-            local_arguments_as_attributes.clear();
-            break;
-          } else {
-            DCHECK_EQ(input_relation_.getID(),
-                      argument->getRelationIdForValueAccessor());
-            local_arguments_as_attributes.push_back(argument_id);
-          }
-        }
-
-        arguments_as_attributes_.emplace_back(
-            std::move(local_arguments_as_attributes));
-#endif
+        argument_ids.emplace_back(ValueAccessorSource::kDerived,
+                                  non_trivial_expressions_.size());
+        non_trivial_expressions_.emplace_back(argument.release());
       }
+    }
+    argument_ids_.emplace_back(std::move(argument_ids));
+
+    // Sanity checks: aggregate function exists and can apply to the specified
+    // arguments.
+    DCHECK(*agg_func_it != nullptr);
+    DCHECK((*agg_func_it)->canApplyToTypes(argument_types));
 
-      // Initialize the corresponding distinctify hash table if this is a
-      // DISTINCT aggregation.
+    // Have the AggregateFunction create an AggregationHandle that we can use
+    // to do actual aggregate computation.
+    handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
+
+    if (!group_by_key_ids_.empty()) {
+      // Aggregation with GROUP BY: combined payload is partially updated in
+      // the presence of DISTINCT.
       if (*is_distinct_it) {
-        std::vector<const Type *> key_types(group_by_types);
-        key_types.insert(
-            key_types.end(), argument_types.begin(), argument_types.end());
-        // TODO(jianqiao): estimated_num_entries is quite inaccurate for
-        // estimating the number of entries in the distinctify hash table.
-        // We may estimate for each distinct aggregation an
-        // estimated_num_distinct_keys value during query optimization, if it's
-        // worth.
-        distinctify_hashtables_.emplace_back(
-            AggregationStateFastHashTableFactory::CreateResizable(
-                *distinctify_hash_table_impl_types_it,
-                key_types,
-                estimated_num_entries,
-                {0},
-                {},
-                storage_manager));
-        ++distinctify_hash_table_impl_types_it;
-      } else {
-        distinctify_hashtables_.emplace_back(nullptr);
+        handles_.back()->blockUpdate();
       }
+      group_by_handles.emplace_back(handles_.back().get());
+    } else {
+      // Aggregation without GROUP BY: create a single global state.
+      single_states_.emplace_back(handles_.back()->createInitialState());
     }
 
-    if (!group_by_handles.empty()) {
-      // Aggregation with GROUP BY: create a HashTable pool.
-      if (!is_aggregate_partitioned_) {
-        group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
-                                                         hash_table_impl_type,
-                                                         group_by_types,
-                                                         payload_sizes,
-                                                         group_by_handles,
-                                                         storage_manager));
-      } else {
-        partitioned_group_by_hashtable_pool_.reset(
-            new PartitionedHashTablePool(estimated_num_entries,
-                                         FLAGS_num_aggregation_partitions,
-                                         hash_table_impl_type,
-                                         group_by_types,
-                                         payload_sizes,
-                                         group_by_handles,
-                                         storage_manager));
-      }
+    // Initialize the corresponding distinctify hash table if this is a
+    // DISTINCT aggregation.
+    if (*is_distinct_it) {
+      std::vector<const Type *> key_types(group_by_types_);
+      key_types.insert(
+          key_types.end(), argument_types.begin(), argument_types.end());
+      // TODO(jianqiao): estimated_num_entries is quite inaccurate for
+      // estimating the number of entries in the distinctify hash table.
+      // We need to estimate for each distinct aggregation an
+      // estimated_num_distinct_keys value during query optimization.
+      distinctify_hashtables_.emplace_back(
+          AggregationStateHashTableFactory::CreateResizable(
+              *distinctify_hash_table_impl_types_it,
+              key_types,
+              estimated_num_entries,
+              {} /* handles */,
+              storage_manager));
+      ++distinctify_hash_table_impl_types_it;
+    } else {
+      distinctify_hashtables_.emplace_back(nullptr);
+    }
+  }
+
+  if (!group_by_key_ids_.empty()) {
+    // Aggregation with GROUP BY: create the hash table (pool).
+    if (is_aggregate_collision_free_) {
+      collision_free_hashtable_.reset(
+          AggregationStateHashTableFactory::CreateResizable(
+              hash_table_impl_type,
+              group_by_types_,
+              estimated_num_entries,
+              group_by_handles,
+              storage_manager));
+    } else if (is_aggregate_partitioned_) {
+      partitioned_group_by_hashtable_pool_.reset(
+          new PartitionedHashTablePool(estimated_num_entries,
+                                       FLAGS_num_aggregation_partitions,
+                                       hash_table_impl_type,
+                                       group_by_types_,
+                                       group_by_handles,
+                                       storage_manager));
+    } else {
+      group_by_hashtable_pool_.reset(
+          new HashTablePool(estimated_num_entries,
+                            hash_table_impl_type,
+                            group_by_types_,
+                            group_by_handles,
+                            storage_manager));
     }
   }
 }
@@ -269,7 +269,7 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
         proto.group_by_expressions(group_by_idx), database));
   }
 
-  unique_ptr<Predicate> predicate;
+  std::unique_ptr<Predicate> predicate;
   if (proto.has_predicate()) {
     predicate.reset(
         PredicateFactory::ReconstructFromProto(proto.predicate(), database));
@@ -353,153 +353,210 @@ bool AggregationOperationState::ProtoIsValid(
   return true;
 }
 
-void AggregationOperationState::aggregateBlock(const block_id input_block,
-                                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
-  if (group_by_list_.empty()) {
-    aggregateBlockSingleState(input_block);
-  } else {
-    aggregateBlockHashTable(input_block, lip_filter_adaptive_prober);
+bool AggregationOperationState::checkAggregatePartitioned(
+    const std::size_t estimated_num_groups,
+    const std::vector<bool> &is_distinct,
+    const std::vector<std::unique_ptr<const Scalar>> &group_by,
+    const std::vector<const AggregateFunction *> &aggregate_functions) const {
+  // If there's no aggregation, return false.
+  if (aggregate_functions.empty()) {
+    return false;
+  }
+  // Check if there's a distinct operation involved in any aggregate, if so
+  // the aggregate can't be partitioned.
+  for (auto distinct : is_distinct) {
+    if (distinct) {
+      return false;
+    }
+  }
+  // There's no distinct aggregation involved, Check if there's at least one
+  // GROUP BY operation.
+  if (group_by.empty()) {
+    return false;
+  }
+
+  // Currently we require that all the group-by keys are ScalarAttributes for
+  // the convenient of implementing copy elision.
+  // TODO(jianqiao): relax this requirement.
+  for (const auto &group_by_element : group_by) {
+    if (group_by_element->getAttributeIdForValueAccessor() == kInvalidAttributeID) {
+      return false;
+    }
   }
+
+  // There are GROUP BYs without DISTINCT. Check if the estimated number of
+  // groups is large enough to warrant a partitioned aggregation.
+  return estimated_num_groups >
+         static_cast<std::size_t>(
+             FLAGS_partition_aggregation_num_groups_threshold);
+  return false;
 }
 
-void AggregationOperationState::finalizeAggregate(
-    InsertDestination *output_destination) {
-  if (group_by_list_.empty()) {
-    finalizeSingleState(output_destination);
+std::size_t AggregationOperationState::getNumInitializationPartitions() const {
+  if (is_aggregate_collision_free_) {
+    return static_cast<CollisionFreeVectorTable *>(
+        collision_free_hashtable_.get())->getNumInitializationPartitions();
   } else {
-    finalizeHashTable(output_destination);
+    return 0u;
   }
 }
 
-void AggregationOperationState::mergeSingleState(
-    const std::vector<std::unique_ptr<AggregationState>> &local_state) {
-  DEBUG_ASSERT(local_state.size() == single_states_.size());
-  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    if (!is_distinct_[agg_idx]) {
-      handles_[agg_idx]->mergeStates(*local_state[agg_idx],
-                                     single_states_[agg_idx].get());
-    }
+std::size_t AggregationOperationState::getNumFinalizationPartitions() const {
+  if (is_aggregate_collision_free_) {
+    return static_cast<CollisionFreeVectorTable *>(
+        collision_free_hashtable_.get())->getNumFinalizationPartitions();
+  } else if (is_aggregate_partitioned_) {
+    return partitioned_group_by_hashtable_pool_->getNumPartitions();
+  } else  {
+    return 1u;
   }
 }
 
-void AggregationOperationState::aggregateBlockSingleState(
-    const block_id input_block) {
-  // Aggregate per-block state for each aggregate.
-  std::vector<std::unique_ptr<AggregationState>> local_state;
+void AggregationOperationState::initialize(const std::size_t partition_id) {
+  if (is_aggregate_collision_free_) {
+    static_cast<CollisionFreeVectorTable *>(
+        collision_free_hashtable_.get())->initialize(partition_id);
+  } else {
+    LOG(FATAL) << "AggregationOperationState::initializeState() "
+               << "is not supported by this aggregation";
+  }
+}
 
+void AggregationOperationState::aggregateBlock(const block_id input_block,
+                                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
   BlockReference block(
       storage_manager_->getBlock(input_block, input_relation_));
+  const auto &tuple_store = block->getTupleStorageSubBlock();
+  std::unique_ptr<ValueAccessor> base_accessor(tuple_store.createValueAccessor());
+  std::unique_ptr<ValueAccessor> shared_accessor;
+  ValueAccessor *accessor = base_accessor.get();
 
+  // Apply the predicate first, then the LIPFilters, to generate a TupleIdSequence
+  // as the existence map for the tuples.
   std::unique_ptr<TupleIdSequence> matches;
   if (predicate_ != nullptr) {
-    std::unique_ptr<ValueAccessor> accessor(
-        block->getTupleStorageSubBlock().createValueAccessor());
-    matches.reset(block->getMatchesForPredicate(predicate_.get(), matches.get()));
+    matches.reset(block->getMatchesForPredicate(predicate_.get()));
+    shared_accessor.reset(
+        base_accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+    accessor = shared_accessor.get();
+  }
+  if (lip_filter_adaptive_prober != nullptr) {
+    matches.reset(lip_filter_adaptive_prober->filterValueAccessor(accessor));
+    shared_accessor.reset(
+        base_accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+    accessor = shared_accessor.get();
   }
 
-  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    const std::vector<attribute_id> *local_arguments_as_attributes = nullptr;
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-    // If all arguments are attributes of the input relation, elide a copy.
-    if (!arguments_as_attributes_[agg_idx].empty()) {
-      local_arguments_as_attributes = &(arguments_as_attributes_[agg_idx]);
+  std::unique_ptr<ColumnVectorsValueAccessor> non_trivial_results;
+  if (!non_trivial_expressions_.empty()) {
+    non_trivial_results.reset(new ColumnVectorsValueAccessor());
+    SubBlocksReference sub_blocks_ref(tuple_store,
+                                      block->getIndices(),
+                                      block->getIndicesConsistent());
+    for (const auto &expression : non_trivial_expressions_) {
+      non_trivial_results->addColumn(
+          expression->getAllValues(accessor, &sub_blocks_ref));
     }
-#endif
+  }
+
+  accessor->beginIterationVirtual();
+
+  ValueAccessorMultiplexer accessor_mux(accessor, non_trivial_results.get());
+  if (group_by_key_ids_.empty()) {
+    aggregateBlockSingleState(accessor_mux);
+  } else {
+    aggregateBlockHashTable(accessor_mux);
+  }
+}
+
+void AggregationOperationState::aggregateBlockSingleState(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  // Aggregate per-block state for each aggregate.
+  std::vector<std::unique_ptr<AggregationState>> local_state;
+
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
+    const auto &argument_ids = argument_ids_[agg_idx];
+    const auto &handle = handles_[agg_idx];
+
+    AggregationState *state = nullptr;
     if (is_distinct_[agg_idx]) {
-      // Call StorageBlock::aggregateDistinct() to put the arguments as keys
-      // directly into the (threadsafe) shared global distinctify HashTable
-      // for this aggregate.
-      block->aggregateDistinct(*handles_[agg_idx],
-                               arguments_[agg_idx],
-                               local_arguments_as_attributes,
-                               {}, /* group_by */
-                               matches.get(),
-                               distinctify_hashtables_[agg_idx].get(),
-                               nullptr /* reuse_group_by_vectors */);
-      local_state.emplace_back(nullptr);
+      handle->insertValueAccessorIntoDistinctifyHashTable(
+          argument_ids,
+          {},
+          accessor_mux,
+          distinctify_hashtables_[agg_idx].get());
     } else {
-      // Call StorageBlock::aggregate() to actually do the aggregation.
-      local_state.emplace_back(block->aggregate(*handles_[agg_idx],
-                                                arguments_[agg_idx],
-                                                local_arguments_as_attributes,
-                                                matches.get()));
+      if (argument_ids.empty()) {
+        // Special case. This is a nullary aggregate (i.e. COUNT(*)).
+        ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+        DCHECK(base_accessor != nullptr);
+        state = handle->accumulateNullary(base_accessor->getNumTuplesVirtual());
+      } else {
+        // Have the AggregationHandle actually do the aggregation.
+        state = handle->accumulateValueAccessor(argument_ids, accessor_mux);
+      }
     }
+    local_state.emplace_back(state);
   }
 
   // Merge per-block aggregation states back with global state.
   mergeSingleState(local_state);
 }
 
-void AggregationOperationState::aggregateBlockHashTable(
-    const block_id input_block,
-    LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
-  BlockReference block(
-      storage_manager_->getBlock(input_block, input_relation_));
-
-  // Apply the predicate first, then the LIPFilters, to generate a TupleIdSequence
-  // as the existence map for the tuples.
-  std::unique_ptr<TupleIdSequence> matches;
-  if (predicate_ != nullptr) {
-    matches.reset(block->getMatchesForPredicate(predicate_.get()));
-  }
-  if (lip_filter_adaptive_prober != nullptr) {
-    std::unique_ptr<ValueAccessor> accessor(
-        block->getTupleStorageSubBlock().createValueAccessor(matches.get()));
-    matches.reset(lip_filter_adaptive_prober->filterValueAccessor(accessor.get()));
-  }
-
-  // This holds values of all the GROUP BY attributes so that the can be reused
-  // across multiple aggregates (i.e. we only pay the cost of evaluatin the
-  // GROUP BY expressions once).
-  std::vector<std::unique_ptr<ColumnVector>> reuse_group_by_vectors;
-
+void AggregationOperationState::mergeSingleState(
+    const std::vector<std::unique_ptr<AggregationState>> &local_state) {
+  DCHECK_EQ(local_state.size(), single_states_.size());
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    if (is_distinct_[agg_idx]) {
-      // Call StorageBlock::aggregateDistinct() to insert the GROUP BY
-      // expression
-      // values and the aggregation arguments together as keys directly into the
-      // (threadsafe) shared global distinctify HashTable for this aggregate.
-      block->aggregateDistinct(*handles_[agg_idx],
-                               arguments_[agg_idx],
-                               nullptr, /* arguments_as_attributes */
-                               group_by_list_,
-                               matches.get(),
-                               distinctify_hashtables_[agg_idx].get(),
-                               &reuse_group_by_vectors);
+    if (!is_distinct_[agg_idx]) {
+      handles_[agg_idx]->mergeStates(*local_state[agg_idx],
+                                     single_states_[agg_idx].get());
     }
   }
+}
+
+void AggregationOperationState::mergeGroupByHashTables(
+    AggregationStateHashTableBase *src,
+    AggregationStateHashTableBase *dst) const {
+  HashTableMerger merger(static_cast<PackedPayloadHashTable *>(dst));
+  static_cast<PackedPayloadHashTable *>(src)->forEachCompositeKey(&merger);
+}
 
-  if (!is_aggregate_partitioned_) {
-    // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
-    // directly into the (threadsafe) shared global HashTable for this
-    // aggregate.
-    DCHECK(group_by_hashtable_pool_ != nullptr);
-    AggregationStateHashTableBase *agg_hash_table =
-      group_by_hashtable_pool_->getHashTableFast();
-    DCHECK(agg_hash_table != nullptr);
-    block->aggregateGroupBy(arguments_,
-                            group_by_list_,
-                            matches.get(),
-                            agg_hash_table,
-                            &reuse_group_by_vectors);
-    group_by_hashtable_pool_->returnHashTable(agg_hash_table);
+void AggregationOperationState::aggregateBlockHashTable(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  if (is_aggregate_collision_free_) {
+    aggregateBlockHashTableImplCollisionFree(accessor_mux);
+  } else if (is_aggregate_partitioned_) {
+    aggregateBlockHashTableImplPartitioned(accessor_mux);
   } else {
-    ColumnVectorsValueAccessor temp_result;
-    // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-    std::vector<attribute_id> argument_ids;
+    aggregateBlockHashTableImplThreadPrivate(accessor_mux);
+  }
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplCollisionFree(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(collision_free_hashtable_ != nullptr);
+
+  collision_free_hashtable_->upsertValueAccessorCompositeKey(argument_ids_,
+                                                             group_by_key_ids_,
+                                                             accessor_mux);
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplPartitioned(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(partitioned_group_by_hashtable_pool_ != nullptr);
+
+  std::vector<attribute_id> group_by_key_ids;
+  for (const MultiSourceAttributeId &key_id : group_by_key_ids_) {
+    DCHECK(key_id.source == ValueAccessorSource::kBase);
+    group_by_key_ids.emplace_back(key_id.attr_id);
+  }
 
-    // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
-    std::vector<attribute_id> key_ids;
+  InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      accessor_mux.getBaseAccessor(),
+      [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+    // TODO(jianqiao): handle the situation when keys in non_trivial_results
     const std::size_t num_partitions = partitioned_group_by_hashtable_pool_->getNumPartitions();
-    block->aggregateGroupByPartitioned(
-        arguments_,
-        group_by_list_,
-        matches.get(),
-        num_partitions,
-        &temp_result,
-        &argument_ids,
-        &key_ids,
-        &reuse_group_by_vectors);
+
     // Compute the partitions for the tuple formed by group by values.
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
     partition_membership.resize(num_partitions);
@@ -507,32 +564,74 @@ void AggregationOperationState::aggregateBlockHashTable(
     // Create a tuple-id sequence for each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
       partition_membership[partition].reset(
-          new TupleIdSequence(temp_result.getEndPosition()));
+          new TupleIdSequence(accessor->getEndPosition()));
     }
 
     // Iterate over ValueAccessor for each tuple,
     // set a bit in the appropriate TupleIdSequence.
-    temp_result.beginIteration();
-    while (temp_result.next()) {
+    while (accessor->next()) {
       // We need a unique_ptr because getTupleWithAttributes() uses "new".
-      std::unique_ptr<Tuple> curr_tuple(temp_result.getTupleWithAttributes(key_ids));
+      std::unique_ptr<Tuple> curr_tuple(
+          accessor->getTupleWithAttributes(group_by_key_ids));
       const std::size_t curr_tuple_partition_id =
           curr_tuple->getTupleHash() % num_partitions;
       partition_membership[curr_tuple_partition_id]->set(
-          temp_result.getCurrentPosition(), true);
+          accessor->getCurrentPosition(), true);
     }
-    // For each partition, create an adapter around Value Accessor and
-    // TupleIdSequence.
-    std::vector<std::unique_ptr<
-        TupleIdSequenceAdapterValueAccessor<ColumnVectorsValueAccessor>>> adapter;
-    adapter.resize(num_partitions);
+
+    // Aggregate each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      adapter[partition].reset(temp_result.createSharedTupleIdSequenceAdapter(
-          *(partition_membership)[partition]));
+      std::unique_ptr<ValueAccessor> base_adapter(
+          accessor->createSharedTupleIdSequenceAdapter(
+              *partition_membership[partition]));
+
+      std::unique_ptr<ValueAccessor> derived_adapter;
+      if (accessor_mux.getDerivedAccessor() != nullptr) {
+        derived_adapter.reset(
+            accessor_mux.getDerivedAccessor()->createSharedTupleIdSequenceAdapterVirtual(
+                *partition_membership[partition]));
+      }
+
+      ValueAccessorMultiplexer local_mux(base_adapter.get(), derived_adapter.get());
       partitioned_group_by_hashtable_pool_->getHashTable(partition)
-          ->upsertValueAccessorCompositeKeyFast(
-              argument_ids, adapter[partition].get(), key_ids, true);
+          ->upsertValueAccessorCompositeKey(argument_ids_,
+                                            group_by_key_ids_,
+                                            local_mux);
     }
+  });
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplThreadPrivate(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(group_by_hashtable_pool_ != nullptr);
+
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
+    if (is_distinct_[agg_idx]) {
+      handles_[agg_idx]->insertValueAccessorIntoDistinctifyHashTable(
+          argument_ids_[agg_idx],
+          group_by_key_ids_,
+          accessor_mux,
+          distinctify_hashtables_[agg_idx].get());
+    }
+  }
+
+  AggregationStateHashTableBase *agg_hash_table =
+      group_by_hashtable_pool_->getHashTable();
+
+  agg_hash_table->upsertValueAccessorCompositeKey(argument_ids_,
+                                                  group_by_key_ids_,
+                                                  accessor_mux);
+  group_by_hashtable_pool_->returnHashTable(agg_hash_table);
+}
+
+void AggregationOperationState::finalizeAggregate(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  if (group_by_key_ids_.empty()) {
+    DCHECK_EQ(0u, partition_id);
+    finalizeSingleState(output_destination);
+  } else {
+    finalizeHashTable(partition_id, output_destination);
   }
 }
 
@@ -556,80 +655,83 @@ void AggregationOperationState::finalizeSingleState(
   output_destination->insertTuple(Tuple(std::move(attribute_values)));
 }
 
-void AggregationOperationState::mergeGroupByHashTables(
-    AggregationStateHashTableBase *src, AggregationStateHashTableBase *dst) {
-  HashTableMergerFast merger(dst);
-  (static_cast<FastHashTable<true, false, true, false> *>(src))
-      ->forEachCompositeKeyFast(&merger);
+void AggregationOperationState::finalizeHashTable(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  if (is_aggregate_collision_free_) {
+    finalizeHashTableImplCollisionFree(partition_id, output_destination);
+  } else if (is_aggregate_partitioned_) {
+    finalizeHashTableImplPartitioned(partition_id, output_destination);
+  } else {
+    DCHECK_EQ(0u, partition_id);
+    finalizeHashTableImplThreadPrivate(output_destination);
+  }
 }
 
-void AggregationOperationState::finalizeHashTable(
+void AggregationOperationState::finalizeHashTableImplCollisionFree(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  std::vector<std::unique_ptr<ColumnVector>> final_values;
+  CollisionFreeVectorTable *hash_table =
+      static_cast<CollisionFreeVectorTable *>(collision_free_hashtable_.get());
+
+  const std::size_t max_length =
+      hash_table->getNumTuplesInFinalizationPartition(partition_id);
+  ColumnVectorsValueAccessor complete_result;
+
+  DCHECK_EQ(1u, group_by_types_.size());
+  const Type *key_type = group_by_types_.front();
+  DCHECK(NativeColumnVector::UsableForType(*key_type));
+
+  std::unique_ptr<NativeColumnVector> key_cv(
+      std::make_unique<NativeColumnVector>(*key_type, max_length));
+  hash_table->finalizeKey(partition_id, key_cv.get());
+  complete_result.addColumn(key_cv.release());
+
+  for (std::size_t i = 0; i < handles_.size(); ++i) {
+    const Type *result_type = handles_[i]->getResultType();
+    DCHECK(NativeColumnVector::UsableForType(*result_type));
+
+    std::unique_ptr<NativeColumnVector> result_cv(
+        std::make_unique<NativeColumnVector>(*result_type, max_length));
+    hash_table->finalizeState(partition_id, i, result_cv.get());
+    complete_result.addColumn(result_cv.release());
+  }
+
+  // Bulk-insert the complete result.
+  output_destination->bulkInsertTuples(&complete_result);
+}
+
+void AggregationOperationState::finalizeHashTableImplPartitioned(
+    const std::size_t partition_id,
     InsertDestination *output_destination) {
+  PackedPayloadHashTable *hash_table =
+      static_cast<PackedPayloadHashTable *>(
+          partitioned_group_by_hashtable_pool_->getHashTable(partition_id));
+
   // Each element of 'group_by_keys' is a vector of values for a particular
   // group (which is also the prefix of the finalized Tuple for that group).
   std::vector<std::vector<TypedValue>> group_by_keys;
 
-  // TODO(harshad) - The merge phase may be slower when each hash table contains
-  // large number of entries. We should find ways in which we can perform a
-  // parallel merge.
+  if (handles_.empty()) {
+    const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
+                                                 const std::uint8_t *dumb_placeholder) -> void {
+      group_by_keys.emplace_back(std::move(group_by_key));
+    };
 
-  // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
-  // e.g. Keep merging entries from smaller hash tables to larger.
-
-  auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
-  if (hash_tables->size() > 1) {
-    for (int hash_table_index = 0;
-         hash_table_index < static_cast<int>(hash_tables->size() - 1);
-         ++hash_table_index) {
-      // Merge each hash table to the last hash table.
-      mergeGroupByHashTables((*hash_tables)[hash_table_index].get(),
-                             hash_tables->back().get());
-    }
+    hash_table->forEachCompositeKey(&keys_retriever);
   }
 
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    if (is_distinct_[agg_idx]) {
-      DCHECK(group_by_hashtable_pool_ != nullptr);
-      auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
-      DCHECK(hash_tables != nullptr);
-      if (hash_tables->empty()) {
-        // We may have a case where hash_tables is empty, e.g. no input blocks.
-        // However for aggregateOnDistinctifyHashTableForGroupBy to work
-        // correctly, we should create an empty group by hash table.
-        AggregationStateHashTableBase *new_hash_table =
-            group_by_hashtable_pool_->getHashTableFast();
-        group_by_hashtable_pool_->returnHashTable(new_hash_table);
-        hash_tables = group_by_hashtable_pool_->getAllHashTables();
-      }
-      DCHECK(hash_tables->back() != nullptr);
-      AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
-      DCHECK(agg_hash_table != nullptr);
-      handles_[agg_idx]->allowUpdate();
-      handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
-          *distinctify_hashtables_[agg_idx], agg_hash_table, agg_idx);
-    }
-
-    auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
-    DCHECK(hash_tables != nullptr);
-    if (hash_tables->empty()) {
-      // We may have a case where hash_tables is empty, e.g. no input blocks.
-      // However for aggregateOnDistinctifyHashTableForGroupBy to work
-      // correctly, we should create an empty group by hash table.
-      AggregationStateHashTableBase *new_hash_table =
-          group_by_hashtable_pool_->getHashTableFast();
-      group_by_hashtable_pool_->returnHashTable(new_hash_table);
-      hash_tables = group_by_hashtable_pool_->getAllHashTables();
-    }
-    AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
-    DCHECK(agg_hash_table != nullptr);
     ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
-        *agg_hash_table, &group_by_keys, agg_idx);
+        *hash_table, agg_idx, &group_by_keys);
     if (agg_result_col != nullptr) {
       final_values.emplace_back(agg_result_col);
     }
   }
+  hash_table->destroyPayload();
 
   // Reorganize 'group_by_keys' in column-major order so that we can make a
   // ColumnVectorsValueAccessor to bulk-insert results.
@@ -640,23 +742,20 @@ void AggregationOperationState::finalizeHashTable(
   // in a single HashTable.
   std::vector<std::unique_ptr<ColumnVector>> group_by_cvs;
   std::size_t group_by_element_idx = 0;
-  for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
-    const Type &group_by_type = group_by_element->getType();
-    if (NativeColumnVector::UsableForType(group_by_type)) {
+  for (const Type *group_by_type : group_by_types_) {
+    if (NativeColumnVector::UsableForType(*group_by_type)) {
       NativeColumnVector *element_cv =
-          new NativeColumnVector(group_by_type, group_by_keys.size());
+          new NativeColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(
-            std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
       }
     } else {
       IndirectColumnVector *element_cv =
-          new IndirectColumnVector(group_by_type, group_by_keys.size());
+          new IndirectColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(
-            std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
       }
     }
     ++group_by_element_idx;
@@ -676,42 +775,64 @@ void AggregationOperationState::finalizeHashTable(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
-void AggregationOperationState::destroyAggregationHashTablePayload() {
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>> *all_hash_tables =
-      nullptr;
-  if (!is_aggregate_partitioned_) {
-    if (group_by_hashtable_pool_ != nullptr) {
-      all_hash_tables = group_by_hashtable_pool_->getAllHashTables();
-    }
-  } else {
-    if (partitioned_group_by_hashtable_pool_ != nullptr) {
-      all_hash_tables = partitioned_group_by_hashtable_pool_->getAllHashTables();
-    }
+void AggregationOperationState::finalizeHashTableImplThreadPrivate(
+    InsertDestination *output_destination) {
+  // TODO(harshad) - The merge phase may be slower when each hash table contains
+  // large number of entries. We should find ways in which we can perform a
+  // parallel merge.
+
+  // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
+  // e.g. Keep merging entries from smaller hash tables to larger.
+
+  auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
+  DCHECK(hash_tables != nullptr);
+  if (hash_tables->empty()) {
+    return;
   }
-  if (all_hash_tables != nullptr) {
-    for (std::size_t ht_index = 0; ht_index < all_hash_tables->size(); ++ht_index) {
-      (*all_hash_tables)[ht_index]->destroyPayload();
-    }
+
+  std::unique_ptr<AggregationStateHashTableBase> final_hash_table_ptr(
+      hash_tables->back().release());
+  for (std::size_t i = 0; i < hash_tables->size() - 1; ++i) {
+    std::unique_ptr<AggregationStateHashTableBase> hash_table(
+        hash_tables->at(i).release());
+    mergeGroupByHashTables(hash_table.get(), final_hash_table_ptr.get());
+    hash_table->destroyPayload();
   }
-}
 
-void AggregationOperationState::finalizeAggregatePartitioned(
-    const std::size_t partition_id, InsertDestination *output_destination) {
+  PackedPayloadHashTable *final_hash_table =
+      static_cast<PackedPayloadHashTable *>(final_hash_table_ptr.get());
+
   // Each element of 'group_by_keys' is a vector of values for a particular
   // group (which is also the prefix of the finalized Tuple for that group).
   std::vector<std::vector<TypedValue>> group_by_keys;
 
+  if (handles_.empty()) {
+    const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
+                                                 const std::uint8_t *dumb_placeholder) -> void {
+      group_by_keys.emplace_back(std::move(group_by_key));
+    };
+
+    final_hash_table->forEachCompositeKey(&keys_retriever);
+  }
+
+
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    AggregationStateHashTableBase *hash_table =
-        partitioned_group_by_hashtable_pool_->getHashTable(partition_id);
-    ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
-        *hash_table, &group_by_keys, agg_idx);
-    if (agg_result_col != nullptr) {
-      final_values.emplace_back(agg_result_col);
+    if (is_distinct_[agg_idx]) {
+      handles_[agg_idx]->allowUpdate();
+      handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
+          *distinctify_hashtables_[agg_idx], agg_idx, final_hash_table);
     }
+
+    ColumnVector *agg_result_col =
+        handles_[agg_idx]->finalizeHashTable(
+            *final_hash_table, agg_idx, &group_by_keys);
+    DCHECK(agg_result_col != nullptr);
+
+    final_values.emplace_back(agg_result_col);
   }
+  final_hash_table->destroyPayload();
 
   // Reorganize 'group_by_keys' in column-major order so that we can make a
   // ColumnVectorsValueAccessor to bulk-insert results.
@@ -722,19 +843,22 @@ void AggregationOperationState::finalizeAggregatePartitioned(
   // in a single HashTable.
   std::vector<std::unique_ptr<ColumnVector>> group_by_cvs;
   std::size_t group_by_element_idx = 0;
-  for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
-    const Type &group_by_type = group_by_element->getType();
-    if (NativeColumnVector::UsableForType(group_by_type)) {
-      NativeColumnVector *element_cv = new NativeColumnVector(group_by_type, group_by_keys.size());
+  for (const Type *group_by_type : group_by_types_) {
+    if (NativeColumnVector::UsableForType(*group_by_type)) {
+      NativeColumnVector *element_cv =
+          new NativeColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(
+            std::move(group_key[group_by_element_idx]));
       }
     } else {
-      IndirectColumnVector *element_cv = new IndirectColumnVector(group_by_type, group_by_keys.size());
+      IndirectColumnVector *element_cv =
+          new IndirectColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(
+            std::move(group_key[group_by_element_idx]));
       }
     }
     ++group_by_element_idx;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 591e3a1..13ee377 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -24,31 +24,27 @@
 #include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
-#include "storage/AggregationOperationState.pb.h"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTablePool.hpp"
 #include "storage/PartitionedHashTablePool.hpp"
 #include "storage/StorageBlockInfo.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "utility/Macros.hpp"
 
-#include "gflags/gflags.h"
-
 namespace quickstep {
 
+namespace serialization { class AggregationOperationState; }
+
 class AggregateFunction;
 class CatalogDatabaseLite;
 class CatalogRelationSchema;
 class InsertDestination;
 class LIPFilterAdaptiveProber;
 class StorageManager;
-
-DECLARE_int32(num_aggregation_partitions);
-DECLARE_int32(partition_aggregation_num_groups_threshold);
+class Type;
 
 /** \addtogroup Storage
  *  @{
@@ -156,6 +152,29 @@ class AggregationOperationState {
       const CatalogDatabaseLite &database);
 
   /**
+   * @brief Get the number of partitions to be used for initializing the
+   *        aggregation.
+   *
+   * @return The number of partitions to be used for initializing the aggregation.
+   **/
+  std::size_t getNumInitializationPartitions() const;
+
+  /**
+   * @brief Get the number of partitions to be used for finalizing the
+   *        aggregation.
+   *
+   * @return The number of partitions to be used for finalizing the aggregation.
+   **/
+  std::size_t getNumFinalizationPartitions() const;
+
+  /**
+   * @brief Initialize the specified partition of this aggregation.
+   *
+   * @param partition_id ID of the partition to be initialized.
+   */
+  void initialize(const std::size_t partition_id);
+
+  /**
    * @brief Compute aggregates on the tuples of the given storage block,
    *        updating the running state maintained by this
    *        AggregationOperationState.
@@ -166,127 +185,95 @@ class AggregationOperationState {
    *        the block.
    **/
   void aggregateBlock(const block_id input_block,
-                      LIPFilterAdaptiveProber *lip_filter_adaptive_prober);
+                      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr);
 
   /**
    * @brief Generate the final results for the aggregates managed by this
    *        AggregationOperationState and write them out to StorageBlock(s).
    *
+   * @param partition_id The partition id of this finalize operation.
    * @param output_destination An InsertDestination where the finalized output
    *        tuple(s) from this aggregate are to be written.
    **/
-  void finalizeAggregate(InsertDestination *output_destination);
-
-  /**
-   * @brief Destroy the payloads in the aggregation hash tables.
-   **/
-  void destroyAggregationHashTablePayload();
-
-  /**
-   * @brief Generate the final results for the aggregates managed by this
-   *        AggregationOperationState and write them out to StorageBlock(s).
-   *        In this implementation, each thread picks a hash table belonging to
-   *        a partition and writes its values to StorageBlock(s). There is no
-   *        need to merge multiple hash tables in one, because there is no
-   *        overlap in the keys across two hash tables.
-   *
-   * @param partition_id The ID of the partition for which finalize is being
-   *        performed.
-   * @param output_destination An InsertDestination where the finalized output
-   *        tuple(s) from this aggregate are to be written.
-   **/
-  void finalizeAggregatePartitioned(
-      const std::size_t partition_id, InsertDestination *output_destination);
-
-  static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
-                                     AggregationStateHashTableBase *dst);
+  void finalizeAggregate(const std::size_t partition_id,
+                         InsertDestination *output_destination);
 
-  bool isAggregatePartitioned() const {
-    return is_aggregate_partitioned_;
-  }
+ private:
+  // Check whether partitioned aggregation can be applied.
+  bool checkAggregatePartitioned(
+      const std::size_t estimated_num_groups,
+      const std::vector<bool> &is_distinct,
+      const std::vector<std::unique_ptr<const Scalar>> &group_by,
+      const std::vector<const AggregateFunction *> &aggregate_functions) const;
 
-  /**
-   * @brief Get the number of partitions to be used for the aggregation.
-   *        For non-partitioned aggregations, we return 1.
-   **/
-  std::size_t getNumPartitions() const {
-    return is_aggregate_partitioned_
-               ? partitioned_group_by_hashtable_pool_->getNumPartitions()
-               : 1;
-  }
+  // Aggregate on input block.
+  void aggregateBlockSingleState(const ValueAccessorMultiplexer &accessor_mux);
 
-  int dflag;
+  void aggregateBlockHashTable(const ValueAccessorMultiplexer &accessor_mux);
 
- private:
   // Merge locally (per storage block) aggregated states with global aggregation
   // states.
   void mergeSingleState(
       const std::vector<std::unique_ptr<AggregationState>> &local_state);
 
-  // Aggregate on input block.
-  void aggregateBlockSingleState(const block_id input_block);
-  void aggregateBlockHashTable(const block_id input_block,
-                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober);
+  void mergeGroupByHashTables(AggregationStateHashTableBase *src,
+                              AggregationStateHashTableBase *dst) const;
 
+  // Finalize the aggregation results into output_destination.
   void finalizeSingleState(InsertDestination *output_destination);
-  void finalizeHashTable(InsertDestination *output_destination);
 
-  bool checkAggregatePartitioned(
-      const std::size_t estimated_num_groups,
-      const std::vector<bool> &is_distinct,
-      const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const std::vector<const AggregateFunction *> &aggregate_functions) const {
-    // If there's no aggregation, return false.
-    if (aggregate_functions.empty()) {
-      return false;
-    }
-    // Check if there's a distinct operation involved in any aggregate, if so
-    // the aggregate can't be partitioned.
-    for (auto distinct : is_distinct) {
-      if (distinct) {
-        return false;
-      }
-    }
-    // There's no distinct aggregation involved, Check if there's at least one
-    // GROUP BY operation.
-    if (group_by.empty()) {
-      return false;
-    }
-    // There are GROUP BYs without DISTINCT. Check if the estimated number of
-    // groups is large enough to warrant a partitioned aggregation.
-    return estimated_num_groups >
-           static_cast<std::size_t>(
-               FLAGS_partition_aggregation_num_groups_threshold);
-  }
+  void finalizeHashTable(const std::size_t partition_id,
+                         InsertDestination *output_destination);
+
+  // Specialized implementations for aggregateBlockHashTable.
+  void aggregateBlockHashTableImplCollisionFree(
+      const ValueAccessorMultiplexer &accessor_mux);
+
+  void aggregateBlockHashTableImplPartitioned(
+      const ValueAccessorMultiplexer &accessor_mux);
+
+  void aggregateBlockHashTableImplThreadPrivate(
+      const ValueAccessorMultiplexer &accessor_mux);
+
+  // Specialized implementations for finalizeHashTable.
+  void finalizeHashTableImplCollisionFree(const std::size_t partition_id,
+                                          InsertDestination *output_destination);
+
+  void finalizeHashTableImplPartitioned(const std::size_t partition_id,
+                                        InsertDestination *output_destination);
+
+  void finalizeHashTableImplThreadPrivate(InsertDestination *output_destination);
 
   // Common state for all aggregates in this operation: the input relation, the
   // filter predicate (if any), and the list of GROUP BY expressions (if any).
   const CatalogRelationSchema &input_relation_;
 
+  // Whether the aggregation is collision free or not.
+  bool is_aggregate_collision_free_;
+
   // Whether the aggregation is partitioned or not.
-  const bool is_aggregate_partitioned_;
+  bool is_aggregate_partitioned_;
 
   std::unique_ptr<const Predicate> predicate_;
-  std::vector<std::unique_ptr<const Scalar>> group_by_list_;
 
   // Each individual aggregate in this operation has an AggregationHandle and
-  // some number of Scalar arguments.
-  std::vector<AggregationHandle *> handles_;
-  std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments_;
+  // zero (indicated by -1) or one argument.
+  std::vector<std::unique_ptr<AggregationHandle>> handles_;
 
   // For each aggregate, whether DISTINCT should be applied to the aggregate's
   // arguments.
   std::vector<bool> is_distinct_;
 
-  // Hash table for obtaining distinct (i.e. unique) arguments.
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
-      distinctify_hashtables_;
+  // Non-trivial group-by/argument expressions that need to be evaluated.
+  std::vector<std::unique_ptr<const Scalar>> non_trivial_expressions_;
+
+  std::vector<MultiSourceAttributeId> group_by_key_ids_;
+  std::vector<std::vector<MultiSourceAttributeId>> argument_ids_;
 
-#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<std::vector<attribute_id>> arguments_as_attributes_;
-#endif
+  std::vector<const Type *> group_by_types_;
+
+  // Hash table for obtaining distinct (i.e. unique) arguments.
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>> distinctify_hashtables_;
 
   // Per-aggregate global states for aggregation without GROUP BY.
   std::vector<std::unique_ptr<AggregationState>> single_states_;
@@ -295,14 +282,15 @@ class AggregationOperationState {
   //
   // TODO(shoban): We should ideally store the aggregation state together in one
   // hash table to prevent multiple lookups.
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
-      group_by_hashtables_;
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>> group_by_hashtables_;
 
   // A vector of group by hash table pools.
   std::unique_ptr<HashTablePool> group_by_hashtable_pool_;
 
   std::unique_ptr<PartitionedHashTablePool> partitioned_group_by_hashtable_pool_;
 
+  std::unique_ptr<AggregationStateHashTableBase> collision_free_hashtable_;
+
   StorageManager *storage_manager_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationOperationState);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index a44c3a7..293be17 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -165,6 +165,9 @@ if(QUICKSTEP_HAVE_BITWEAVING)
               bitweaving/BitWeavingVIndexSubBlock.hpp)
 endif()
 # CMAKE_VALIDATE_IGNORE_END
+add_library(quickstep_storage_CollisionFreeVectorTable
+            CollisionFreeVectorTable.cpp
+            CollisionFreeVectorTable.hpp)
 add_library(quickstep_storage_ColumnStoreUtil ColumnStoreUtil.cpp ColumnStoreUtil.hpp)
 add_library(quickstep_storage_CompressedBlockBuilder CompressedBlockBuilder.cpp CompressedBlockBuilder.hpp)
 add_library(quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
@@ -194,9 +197,6 @@ if (ENABLE_DISTRIBUTED)
 endif()
 
 add_library(quickstep_storage_EvictionPolicy EvictionPolicy.cpp EvictionPolicy.hpp)
-add_library(quickstep_storage_FastHashTable ../empty_src.cpp FastHashTable.hpp)
-add_library(quickstep_storage_FastHashTableFactory ../empty_src.cpp FastHashTableFactory.hpp)
-add_library(quickstep_storage_FastSeparateChainingHashTable ../empty_src.cpp FastSeparateChainingHashTable.hpp)
 add_library(quickstep_storage_FileManager ../empty_src.cpp FileManager.hpp)
 if (QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
   add_library(quickstep_storage_FileManagerHdfs FileManagerHdfs.cpp FileManagerHdfs.hpp)
@@ -226,6 +226,7 @@ add_library(quickstep_storage_InsertDestination_proto
 add_library(quickstep_storage_LinearOpenAddressingHashTable
             ../empty_src.cpp
             LinearOpenAddressingHashTable.hpp)
+add_library(quickstep_storage_PackedPayloadHashTable PackedPayloadHashTable.cpp PackedPayloadHashTable.hpp)
 add_library(quickstep_storage_PartitionedHashTablePool ../empty_src.cpp PartitionedHashTablePool.hpp)
 add_library(quickstep_storage_PreloaderThread PreloaderThread.cpp PreloaderThread.hpp)
 add_library(quickstep_storage_SMAIndexSubBlock SMAIndexSubBlock.cpp SMAIndexSubBlock.hpp)
@@ -253,6 +254,7 @@ add_library(quickstep_storage_TupleIdSequence ../empty_src.cpp TupleIdSequence.h
 add_library(quickstep_storage_TupleReference ../empty_src.cpp TupleReference.hpp)
 add_library(quickstep_storage_TupleStorageSubBlock TupleStorageSubBlock.cpp TupleStorageSubBlock.hpp)
 add_library(quickstep_storage_ValueAccessor ../empty_src.cpp ValueAccessor.hpp)
+add_library(quickstep_storage_ValueAccessorMultiplexer ../empty_src.cpp ValueAccessorMultiplexer.hpp)
 add_library(quickstep_storage_ValueAccessorUtil ../empty_src.cpp ValueAccessorUtil.hpp)
 add_library(quickstep_storage_WindowAggregationOperationState
             WindowAggregationOperationState.hpp
@@ -272,22 +274,25 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunctionFactory
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_expressions_aggregation_AggregationHandleDistinct
-                      quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_storage_AggregationOperationState_proto
-                      quickstep_storage_HashTable
+                      quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTablePool
                       quickstep_storage_InsertDestination
                       quickstep_storage_PartitionedHashTablePool
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
+                      quickstep_storage_SubBlocksReference
                       quickstep_storage_TupleIdSequence
+                      quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
+                      quickstep_storage_ValueAccessorUtil
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
@@ -430,6 +435,24 @@ if(QUICKSTEP_HAVE_BITWEAVING)
                         quickstep_utility_Macros)
 endif()
 # CMAKE_VALIDATE_IGNORE_END
+target_link_libraries(quickstep_storage_CollisionFreeVectorTable
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_ColumnStoreUtil
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelationSchema
@@ -627,52 +650,6 @@ target_link_libraries(quickstep_storage_EvictionPolicy
                       quickstep_threading_SpinMutex
                       quickstep_threading_SpinSharedMutex
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_storage_FastHashTable
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_StorageBlob
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageConstants
-                      quickstep_storage_StorageManager
-                      quickstep_storage_TupleReference
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_threading_SpinMutex
-                      quickstep_threading_SpinSharedMutex
-                      quickstep_types_Type
-                      quickstep_types_TypedValue
-                      quickstep_utility_HashPair
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_storage_FastHashTableFactory
-                      glog
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastSeparateChainingHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTable_proto
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
-                      quickstep_storage_LinearOpenAddressingHashTable
-                      quickstep_storage_SeparateChainingHashTable
-                      quickstep_storage_SimpleScalarSeparateChainingHashTable
-                      quickstep_storage_TupleReference
-                      quickstep_types_TypeFactory
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_storage_FastSeparateChainingHashTable
-                      quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableKeyManager
-                      quickstep_storage_StorageBlob
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageConstants
-                      quickstep_storage_StorageManager
-                      quickstep_threading_SpinSharedMutex
-                      quickstep_types_Type
-                      quickstep_types_TypedValue
-                      quickstep_utility_Alignment
-                      quickstep_utility_Macros
-                      quickstep_utility_PrimeNumber)
 target_link_libraries(quickstep_storage_FileManager
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros
@@ -731,16 +708,19 @@ target_link_libraries(quickstep_storage_HashTable
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTableBase
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTable_proto
                       quickstep_types_Type_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_storage_HashTableFactory
                       glog
+                      quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTable_proto
                       quickstep_storage_HashTableBase
                       quickstep_storage_LinearOpenAddressingHashTable
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_SeparateChainingHashTable
                       quickstep_storage_SimpleScalarSeparateChainingHashTable
                       quickstep_storage_TupleReference
@@ -759,13 +739,10 @@ target_link_libraries(quickstep_storage_HashTableKeyManager
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTablePool
                       glog
-                      quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
                       quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
                       quickstep_threading_SpinMutex
-                      quickstep_utility_Macros
-                      quickstep_utility_StringUtil)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_IndexSubBlock
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_predicate_PredicateCost
@@ -820,14 +797,32 @@ target_link_libraries(quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_utility_Alignment
                       quickstep_utility_Macros
                       quickstep_utility_PrimeNumber)
-target_link_libraries(quickstep_storage_PartitionedHashTablePool
-                      glog
+target_link_libraries(quickstep_storage_PackedPayloadHashTable
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
                       quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableKeyManager
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_threading_SpinMutex
+                      quickstep_threading_SpinSharedMutex
+                      quickstep_types_Type
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_Alignment
+                      quickstep_utility_HashPair
                       quickstep_utility_Macros
-                      quickstep_utility_StringUtil)
+                      quickstep_utility_PrimeNumber)
+target_link_libraries(quickstep_storage_PartitionedHashTablePool
+                      glog
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_PreloaderThread
                       glog
                       quickstep_catalog_CatalogDatabase
@@ -936,7 +931,6 @@ target_link_libraries(quickstep_storage_StorageBlock
                       glog
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_storage_BasicColumnStoreTupleStorageSubBlock
@@ -945,7 +939,6 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
                       quickstep_storage_CompressedPackedRowStoreTupleStorageSubBlock
                       quickstep_storage_CountedReference
-                      quickstep_storage_HashTableBase
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_InsertDestinationInterface
                       quickstep_storage_SMAIndexSubBlock
@@ -1068,6 +1061,10 @@ target_link_libraries(quickstep_storage_ValueAccessor
                       quickstep_types_TypedValue
                       quickstep_types_containers_Tuple
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_ValueAccessorMultiplexer
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_ValueAccessorUtil
                       glog
                       quickstep_storage_BasicColumnStoreValueAccessor
@@ -1115,6 +1112,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_BasicColumnStoreValueAccessor
                       quickstep_storage_BloomFilterIndexSubBlock
                       quickstep_storage_CSBTreeIndexSubBlock
+                      quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_ColumnStoreUtil
                       quickstep_storage_CompressedBlockBuilder
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
@@ -1125,9 +1123,6 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_CompressedTupleStorageSubBlock
                       quickstep_storage_CountedReference
                       quickstep_storage_EvictionPolicy
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
-                      quickstep_storage_FastSeparateChainingHashTable
                       quickstep_storage_FileManager
                       quickstep_storage_FileManagerLocal
                       quickstep_storage_Flags
@@ -1144,6 +1139,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_InsertDestination_proto
                       quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_storage_PartitionedHashTablePool
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_PreloaderThread
                       quickstep_storage_SMAIndexSubBlock
                       quickstep_storage_SeparateChainingHashTable
@@ -1166,6 +1162,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_TupleReference
                       quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_storage_ValueAccessorUtil
                       quickstep_storage_WindowAggregationOperationState
                       quickstep_storage_WindowAggregationOperationState_proto)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/CollisionFreeVectorTable.cpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.cpp b/storage/CollisionFreeVectorTable.cpp
new file mode 100644
index 0000000..d836014
--- /dev/null
+++ b/storage/CollisionFreeVectorTable.cpp
@@ -0,0 +1,285 @@
+/**
+ * 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 "storage/CollisionFreeVectorTable.hpp"
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+CollisionFreeVectorTable::CollisionFreeVectorTable(
+    const Type *key_type,
+    const std::size_t num_entries,
+    const std::vector<AggregationHandle *> &handles,
+    StorageManager *storage_manager)
+    : key_type_(key_type),
+      num_entries_(num_entries),
+      num_handles_(handles.size()),
+      handles_(handles),
+      num_finalize_partitions_(CalculateNumFinalizationPartitions(num_entries_)),
+      storage_manager_(storage_manager) {
+  DCHECK_GT(num_entries, 0u);
+
+  std::size_t required_memory = 0;
+  const std::size_t existence_map_offset = 0;
+  std::vector<std::size_t> state_offsets;
+
+  required_memory += CacheLineAlignedBytes(
+      BarrieredReadWriteConcurrentBitVector::BytesNeeded(num_entries));
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    const AggregationHandle *handle = handles_[i];
+    const std::vector<const Type *> argument_types = handle->getArgumentTypes();
+
+    std::size_t state_size = 0;
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        state_size = sizeof(std::atomic<std::size_t>);
+        break;
+      }
+      case AggregationID::kSum: {
+        DCHECK_EQ(1u, argument_types.size());
+        switch (argument_types.front()->getTypeID()) {
+          case TypeID::kInt:  // Fall through
+          case TypeID::kLong:
+            state_size = sizeof(std::atomic<std::int64_t>);
+            break;
+          case TypeID::kFloat:  // Fall through
+          case TypeID::kDouble:
+            state_size = sizeof(std::atomic<double>);
+            break;
+          default:
+            LOG(FATAL) << "Not implemented";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Not implemented";
+    }
+
+    state_offsets.emplace_back(required_memory);
+    required_memory += CacheLineAlignedBytes(state_size * num_entries);
+  }
+
+  const std::size_t num_storage_slots =
+      storage_manager_->SlotsNeededForBytes(required_memory);
+
+  const block_id blob_id = storage_manager_->createBlob(num_storage_slots);
+  blob_ = storage_manager_->getBlobMutable(blob_id);
+
+  void *memory_start = blob_->getMemoryMutable();
+  existence_map_.reset(new BarrieredReadWriteConcurrentBitVector(
+      reinterpret_cast<char *>(memory_start) + existence_map_offset,
+      num_entries,
+      false /* initialize */));
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    // Columnwise layout.
+    vec_tables_.emplace_back(
+        reinterpret_cast<char *>(memory_start) + state_offsets.at(i));
+  }
+
+  memory_size_ = required_memory;
+  num_init_partitions_ = CalculateNumInitializationPartitions(memory_size_);
+}
+
+CollisionFreeVectorTable::~CollisionFreeVectorTable() {
+  const block_id blob_id = blob_->getID();
+  blob_.release();
+  storage_manager_->deleteBlockOrBlobFile(blob_id);
+}
+
+void CollisionFreeVectorTable::destroyPayload() {
+}
+
+bool CollisionFreeVectorTable::upsertValueAccessorCompositeKey(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_ids,
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK_EQ(1u, key_ids.size());
+
+  if (handles_.empty()) {
+    InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor_mux.getValueAccessorBySource(key_ids.front().source),
+        [&key_ids, this](auto *accessor) -> void {  // NOLINT(build/c++11)
+      this->upsertValueAccessorKeyOnlyHelper(key_type_->isNullable(),
+                                             key_type_,
+                                             key_ids.front().attr_id,
+                                             accessor);
+    });
+    return true;
+  }
+
+  DCHECK(accessor_mux.getDerivedAccessor() == nullptr ||
+         accessor_mux.getDerivedAccessor()->getImplementationType()
+             == ValueAccessor::Implementation::kColumnVectors);
+
+  ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+  ColumnVectorsValueAccessor *derived_accesor =
+      static_cast<ColumnVectorsValueAccessor *>(accessor_mux.getDerivedAccessor());
+
+  // Dispatch to specialized implementations to achieve maximum performance.
+  InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      base_accessor,
+      [&argument_ids, &key_ids, &derived_accesor, this](auto *accessor) -> void {  // NOLINT(build/c++11)
+    const ValueAccessorSource key_source = key_ids.front().source;
+    const attribute_id key_id = key_ids.front().attr_id;
+    const bool is_key_nullable = key_type_->isNullable();
+
+    for (std::size_t i = 0; i < num_handles_; ++i) {
+      DCHECK_LE(argument_ids[i].size(), 1u);
+
+      const AggregationHandle *handle = handles_[i];
+      const auto &argument_types = handle->getArgumentTypes();
+      const auto &argument_ids_i = argument_ids[i];
+
+      ValueAccessorSource argument_source;
+      attribute_id argument_id;
+      const Type *argument_type;
+      bool is_argument_nullable;
+
+      if (argument_ids_i.empty()) {
+        argument_source = ValueAccessorSource::kInvalid;
+        argument_id = kInvalidAttributeID;
+
+        DCHECK(argument_types.empty());
+        argument_type = nullptr;
+        is_argument_nullable = false;
+      } else {
+        DCHECK_EQ(1u, argument_ids_i.size());
+        argument_source = argument_ids_i.front().source;
+        argument_id = argument_ids_i.front().attr_id;
+
+        DCHECK_EQ(1u, argument_types.size());
+        argument_type = argument_types.front();
+        is_argument_nullable = argument_type->isNullable();
+      }
+
+      if (key_source == ValueAccessorSource::kBase) {
+        if (argument_source == ValueAccessorSource::kBase) {
+          this->upsertValueAccessorDispatchHelper<false>(is_key_nullable,
+                                                         is_argument_nullable,
+                                                         key_type_,
+                                                         argument_type,
+                                                         handle->getAggregationID(),
+                                                         key_id,
+                                                         argument_id,
+                                                         vec_tables_[i],
+                                                         accessor,
+                                                         accessor);
+        } else {
+          this->upsertValueAccessorDispatchHelper<true>(is_key_nullable,
+                                                        is_argument_nullable,
+                                                        key_type_,
+                                                        argument_type,
+                                                        handle->getAggregationID(),
+                                                        key_id,
+                                                        argument_id,
+                                                        vec_tables_[i],
+                                                        accessor,
+                                                        derived_accesor);
+        }
+      } else {
+        if (argument_source == ValueAccessorSource::kBase) {
+          this->upsertValueAccessorDispatchHelper<true>(is_key_nullable,
+                                                        is_argument_nullable,
+                                                        key_type_,
+                                                        argument_type,
+                                                        handle->getAggregationID(),
+                                                        key_id,
+                                                        argument_id,
+                                                        vec_tables_[i],
+                                                        derived_accesor,
+                                                        accessor);
+        } else {
+          this->upsertValueAccessorDispatchHelper<false>(is_key_nullable,
+                                                         is_argument_nullable,
+                                                         key_type_,
+                                                         argument_type,
+                                                         handle->getAggregationID(),
+                                                         key_id,
+                                                         argument_id,
+                                                         vec_tables_[i],
+                                                         derived_accesor,
+                                                         derived_accesor);
+        }
+      }
+    }
+  });
+  return true;
+}
+
+void CollisionFreeVectorTable::finalizeKey(const std::size_t partition_id,
+                                           NativeColumnVector *output_cv) const {
+  const std::size_t start_position =
+      calculatePartitionStartPosition(partition_id);
+  const std::size_t end_position =
+      calculatePartitionEndPosition(partition_id);
+
+  switch (key_type_->getTypeID()) {
+    case TypeID::kInt:
+      finalizeKeyInternal<int>(start_position, end_position, output_cv);
+      return;
+    case TypeID::kLong:
+      finalizeKeyInternal<std::int64_t>(start_position, end_position, output_cv);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+void CollisionFreeVectorTable::finalizeState(const std::size_t partition_id,
+                                             const std::size_t handle_id,
+                                             NativeColumnVector *output_cv) const {
+  const std::size_t start_position =
+      calculatePartitionStartPosition(partition_id);
+  const std::size_t end_position =
+      calculatePartitionEndPosition(partition_id);
+
+  const AggregationHandle *handle = handles_[handle_id];
+  const auto &argument_types = handle->getArgumentTypes();
+  const Type *argument_type =
+      argument_types.empty() ? nullptr : argument_types.front();
+
+  finalizeStateDispatchHelper(handle->getAggregationID(),
+                              argument_type,
+                              vec_tables_[handle_id],
+                              start_position,
+                              end_position,
+                              output_cv);
+}
+
+}  // namespace quickstep


[46/50] [abbrv] incubator-quickstep git commit: Printed out the row number of the query result in the distributed version.

Posted by ji...@apache.org.
Printed out the row number of the query result in 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/fead6f82
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/fead6f82
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/fead6f82

Branch: refs/heads/LIP-time-decomposition
Commit: fead6f8204b59c0037a7a72120f0231289a59d36
Parents: a8e5618
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 9 21:31:02 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 21:31:47 2017 -0800

----------------------------------------------------------------------
 cli/distributed/Cli.cpp | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/fead6f82/cli/distributed/Cli.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Cli.cpp b/cli/distributed/Cli.cpp
index 386654d..60b9c8d 100644
--- a/cli/distributed/Cli.cpp
+++ b/cli/distributed/Cli.cpp
@@ -197,6 +197,7 @@ void Cli::run() {
               const CatalogRelation result_relation(proto.result_relation());
 
               PrintToScreen::PrintRelation(result_relation, storage_manager_.get(), stdout);
+              PrintToScreen::PrintOutputSize(result_relation, storage_manager_.get(), stdout);
 
               const vector<block_id> blocks(result_relation.getBlocksSnapshot());
               for (const block_id block : blocks) {


[36/50] [abbrv] incubator-quickstep git commit: Refactor building.md

Posted by ji...@apache.org.
Refactor building.md


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

Branch: refs/heads/LIP-time-decomposition
Commit: 5773027ffcc55eed51a693df524a5558ac7305db
Parents: a9fe07d
Author: cramja <ma...@gmail.com>
Authored: Tue Feb 7 18:41:16 2017 -0600
Committer: cramja <ma...@gmail.com>
Committed: Wed Feb 8 19:27:14 2017 -0600

----------------------------------------------------------------------
 BUILDING.md | 167 +++++++++++++++++++++++++++++++------------------------
 1 file changed, 94 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5773027f/BUILDING.md
----------------------------------------------------------------------
diff --git a/BUILDING.md b/BUILDING.md
index 02a3a58..cc362fc 100644
--- a/BUILDING.md
+++ b/BUILDING.md
@@ -1,60 +1,73 @@
-Quickstep Build Guide
-=====================
+# Quickstep Build Guide
 
+**Contents**
+* [Basic Instructions](#basic-instructions)
+  * [Prerequisites](#prerequisites)
+  * [Building](#building)
+  * [Running Quickstep](#running-quickstep)
+  * [Running Tests](#running-tests)
+  * [Configuring with CMake](#configuring-with-cmake)
+* [Advanced Configuration](#advanced-configuration)
+* [Appendix](#appendix)
+  * [Building on Windows](#building-on-windows)
+  * [Building in Vagrant](#building-in-vagrant)
 
-What You Will Need
-------------------
 
-To build quickstep, you will need a C++ compiler that supports the C++14
-standard (GCC 4.9 or Clang 3.4 or higher are known to work), and CMake. If you
-have GNU Bison and Flex as well, they will be used to build the parser and
-lexer sources (otherwise, preprocessed sources made with Bison and Flex will be
-used).
+**Short Version**
 
-### Vagrant
+```sh
+git submodule init
+git submodule update
+cd third_party
+./download_and_patch_prerequisites.sh
+cd ../build
+cmake ..
+make quickstep_cli_shell
+./quickstep_cli_shell -initialize_db=true
+```
 
-For your convenience, we have provided Vagrant virtual machine configurations
-that have a complete development environment for Quickstep with all necessary
-tools and dependencies already installed. [See here for instructions on how to
-use them](build/vagrant/README.md).
+# Basic Instructions
 
-### Getting CMake
+## Prerequisites
 
-Quickstep uses the CMake build system.
+- C++ compiler that supports the C++14 standard (GCC 4.9+ or Clang 3.4+ are good)
+- [cmake](http://www.cmake.org/download/) 2.8.6+
+- curl
 
-If you're on a Linux machine, most distributions' package managers have a
-package for CMake that you can install. The same goes for all of the major
-flavors of BSD UNIX (Free/Net/Open/Dragonfly), OpenSolaris, and Cygwin.
+All these programs should be available on your distro's package manager.
 
-If you're using Mac OS X or Windows, you can download CMake binaries from:
-http://www.cmake.org/download/
+**Optional**
 
-On Mac or Windows, be sure to let the installer put links to the CMake command-
-line tools in bin or add them to your PATH.
+- GNU Bison and Flex (They will be used to build the parser and lexer, but pre-processed copies are provided)
 
-### Special Note: Building on Windows
+## Building
 
-To build on Windows, you will need some variety of Microsoft's C++ compiler and
-the nmake tool (either from Visual Studio, Visual C++ Express, or the Windows
-SDK). Only Visual Studio 2015 or higher is sufficiently modern to build
-Quickstep.
+Once cmake finishes, you are ready to actually build quickstep by running
+`make` (or `nmake` on Windows) (this will also build bundled third-party
+libraries as necesary). If you want to see the actual commands that make is
+running, you can do `make VERBOSE=1`. It is highly recommended to do a parallel
+make to speed up the build time, which you can do with `make -jX`, where X is
+the number of parallel jobs (the number of CPU cores on your system is a good
+choice, unless you are low on RAM, in which case you may want to reduce the
+number of jobs).
 
-Once you have the necessary tools installed, run the "Visual Studio Command
-Prompt" (use the 64-bit version if you have it). Change into the build
-directory and run:
+## Running Quickstep
 
-    cmake -G "NMake Makefiles" ..
+To use quickstep, just run `quickstep_cli_shell` in the build directory. For the
+first time user, run once with `-initialize_db=true` to set up an empty catalog.
+Quickstep has number of command-line flags that control its behavior. Run
+`quickstep_cli_shell --help` to see a listing of the options and how to use
+them.
 
-The `-G "NMake Makefiles"` option tells CMake to generate makefiles for the nmake
-tool instead of project files for Visual Studio. You can also specify the usual
-cmake options described below like `-D CMAKE_BUILD_TYPE=Release`.
+## Running Tests
 
-Once cmake finishes, run `nmake` to actually build quickstep. Unfortunately,
-nmake does not support parallel jobs like UNIX make, so you're in for a bit of
-a wait.
+Quickstep comes with an extensive suite of unit tests. After a successful
+build, you can run the whole test suite by doing `make test` or `ctest`. If
+you use `ctest`, you may also run tests in parallel with `ctest -jX`, where
+X is the number of parallel jobs (as with `make`, your number of CPU cores is
+usually a good choice).
 
-Configuring with CMake
-----------------------
+## Configuring with CMake
 
 CMake recommends building outside of the source tree (a recommendation which we
 follow). For your convenience, a "build" directory with a skeleton of files
@@ -65,21 +78,25 @@ Like a conventional configure script, you can configure some settings about how
 quickstep is built when you invoke cmake. The most important is the build type.
 You can build an unoptimized build with debugging information by doing:
 
-    cmake -D CMAKE_BUILD_TYPE=Debug ..
+```
+cmake -D CMAKE_BUILD_TYPE=Debug ..
+```
 
 You can build a fast, optimized release build by doing:
 
-    cmake -D CMAKE_BUILD_TYPE=Release ..
+```
+cmake -D CMAKE_BUILD_TYPE=Release ..
+```
 
 The first time you check out the Quickstep source repo, you will also need to
-fetch some third-party dependencies that are packaged as git submodules. Do
-this by running the following 2 commands in the root quickstep directory:
+fetch some third-party dependencies. Do this by running the following commands 
+in the root quickstep directory:
 
-    git submodule init
-    git submodule update
-    cd third_party && ./download_and_patch_prerequisites.sh
+```
+cd third_party && ./download_and_patch_prerequisites.sh
+```
 
-### Advanced Configuration
+# Advanced Configuration
 
 There are a number of advanced options you can pass to CMake to control how
 Quickstep is built. These all have sensible defaults, so you may skip this
@@ -91,7 +108,9 @@ section and go straight to "Building" below if you are not interested.
   `CMAKE_C_COMPILER` and `CMAKE_CXX_COMPILER` options. For example, if you
   wish to use clang instead of gcc, you would do this:
 
-      cmake -D CMAKE_BUILD_TYPE=Release -D CMAKE_C_COMPILER=clang -D CMAKE_CXX_COMPILER=clang++ ../
+```
+cmake -D CMAKE_BUILD_TYPE=Release -D CMAKE_C_COMPILER=clang -D CMAKE_CXX_COMPILER=clang++ ../
+```
 
 * **Disabling TCMalloc**: You can configure whether quickstep should use
   tcmalloc (it does by default). tcmalloc stands for thread-cacheing malloc, it
@@ -160,6 +179,7 @@ section and go straight to "Building" below if you are not interested.
   default, the Quickstep storage engine will always try to rebuild an index if
   it runs out of space, but this behavior can be disabled by setting
   `-D REBUILD_INDEX_ON_UPDATE_OVERFLOW=0`.
+
 * **Building With libc++**: The Clang compiler is usually used with the
   system-default C++ standard library (on most Linux systems, this is GNU
   libstdc++, which is packaged with GCC). Clang can also be used with the LLVM
@@ -168,6 +188,7 @@ section and go straight to "Building" below if you are not interested.
   standard library). If you are using Clang on a system that has libc++
   installed but doesn't use it by default, add `-D USE_LIBCXX=1` to make
   Clang use libc++.
+
 * **Link-Time Optimization**: Some compilers support link-time optimization,
   where all the objects linked into an executable are analyzed and optimized
   together as if they were a single translation unit. This potentially enables
@@ -176,32 +197,32 @@ section and go straight to "Building" below if you are not interested.
   release builds with GCC or ICC by doing `-D ENABLE_LTO=1`. Be aware that the
   build may take a very long time.
 
-Building
---------
+# Appendix
 
-Once cmake finishes, you are ready to actually build quickstep by running
-`make` (or `nmake` on Windows) (this will also build bundled third-party
-libraries as necesary). If you want to see the actual commands that make is
-running, you can do `make VERBOSE=1`. It is highly recommended to do a parallel
-make to speed up the build time, which you can do with `make -jX`, where X is
-the number of parallel jobs (the number of CPU cores on your system is a good
-choice, unless you are low on RAM, in which case you may want to reduce the
-number of jobs).
+## Building on Windows
 
-Running Quickstep
------------------
+To build on Windows, you will need some variety of Microsoft's C++ compiler and
+the nmake tool (either from Visual Studio, Visual C++ Express, or the Windows
+SDK). Only Visual Studio 2015 or higher is sufficiently modern to build
+Quickstep.
 
-To use quickstep, just run `quickstep_cli_shell` in the build directory. For the
-first time user, run once with `-initialize_db=true` to set up an empty catalog.
-Quickstep has number of command-line flags that control its behavior. Run
-`quickstep_cli_shell --help` to see a listing of the options and how to use
-them.
+Once you have the necessary tools installed, run the "Visual Studio Command
+Prompt" (use the 64-bit version if you have it). Change into the build
+directory and run:
 
-Running Tests
--------------
+    cmake -G "NMake Makefiles" ..
 
-Quickstep comes with an extensive suite of unit tests. After a successful
-build, you can run the whole test suite by doing `make test` or `ctest`. If
-you use `ctest`, you may also run tests in parallel with `ctest -jX`, where
-X is the number of parallel jobs (as with `make`, your number of CPU cores is
-usually a good choice).
+The `-G "NMake Makefiles"` option tells CMake to generate makefiles for the nmake
+tool instead of project files for Visual Studio. You can also specify the usual
+cmake options described below like `-D CMAKE_BUILD_TYPE=Release`.
+
+Once cmake finishes, run `nmake` to actually build quickstep. Unfortunately,
+nmake does not support parallel jobs like UNIX make, so you're in for a bit of
+a wait.
+
+## Building in Vagrant
+
+For your convenience, we have provided Vagrant virtual machine configurations
+that have a complete development environment for Quickstep with all necessary
+tools and dependencies already installed. [See here for instructions on how to
+use them](build/vagrant/README.md).


[20/50] [abbrv] incubator-quickstep git commit: Adds regex to specify 16.+ versions of Darwin

Posted by ji...@apache.org.
Adds regex to specify 16.+ versions of Darwin

Before this change, only one recent version of Darwin was correcly
having the compiler flags modified to allow for deprecated syscalls.
After this change, all versions of Darwin 16+ should correcly apply the
cxx flags. This is still a somewhat temporary fix because the real
issue (replacing the deprecated calls) has not been addressed.


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

Branch: refs/heads/LIP-time-decomposition
Commit: ec76096bc766c1a2c0f21929b3a998c51e417208
Parents: 669a74c
Author: cramja <ma...@gmail.com>
Authored: Tue Feb 7 11:28:39 2017 -0600
Committer: cramja <ma...@gmail.com>
Committed: Tue Feb 7 14:33:41 2017 -0600

----------------------------------------------------------------------
 CMakeLists.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ec76096b/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index e74a274..918069c 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -299,7 +299,7 @@ else()
   # builds to fail. As a short-term workaround for now, we turn off deprecated
   # warnings so that they do not cause build failures anymore.
   # TODO: Remove this workaround by fixing the protobuf_cmake and glog_cmake.
-  if (${CMAKE_SYSTEM} MATCHES "Darwin-16.1.0")
+  if (${CMAKE_SYSTEM} MATCHES "Darwin-16.[0-9]*.[0-9]*")
     if (${CMAKE_CXX_COMPILER_ID} MATCHES "Clang")
       CHECK_CXX_COMPILER_FLAG("-Wno-error=deprecated-declarations" COMPILER_HAS_WNO_DEPRECATED)
       if (COMPILER_HAS_WNO_DEPRECATED)


[25/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/CollisionFreeVectorTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.hpp b/storage/CollisionFreeVectorTable.hpp
new file mode 100644
index 0000000..4f3e238
--- /dev/null
+++ b/storage/CollisionFreeVectorTable.hpp
@@ -0,0 +1,730 @@
+/**
+ * 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_STORAGE_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_
+
+#include <algorithm>
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class AggregationHandle;
+class StorageManager;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class CollisionFreeVectorTable : public AggregationStateHashTableBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param key_type The group-by key type.
+   * @param num_entries The estimated number of entries this table will hold.
+   * @param handles The aggregation handles.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold this table's contents).
+   **/
+  CollisionFreeVectorTable(
+      const Type *key_type,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  ~CollisionFreeVectorTable() override;
+
+  void destroyPayload() override;
+
+  /**
+   * @brief Get the number of partitions to be used for initializing the table.
+   *
+   * @return The number of partitions to be used for initializing the table.
+   */
+  inline std::size_t getNumInitializationPartitions() const {
+    return num_init_partitions_;
+  }
+
+  /**
+   * @brief Get the number of partitions to be used for finalizing the aggregation.
+   *
+   * @return The number of partitions to be used for finalizing the aggregation.
+   */
+  inline std::size_t getNumFinalizationPartitions() const {
+    return num_finalize_partitions_;
+  }
+
+  /**
+   * @brief Get the exact number of tuples in the specified finalization partition.
+   *
+   * @return The exact number of tuples in the specified finalization partition.
+   */
+  inline std::size_t getNumTuplesInFinalizationPartition(
+      const std::size_t partition_id) const {
+    const std::size_t start_position =
+        calculatePartitionStartPosition(partition_id);
+    const std::size_t end_position =
+        calculatePartitionEndPosition(partition_id);
+    return existence_map_->onesCountInRange(start_position, end_position);
+  }
+
+  /**
+   * @brief Initialize the specified partition of this aggregation table.
+   *
+   * @param partition_id ID of the partition to be initialized.
+   */
+  inline void initialize(const std::size_t partition_id) {
+    const std::size_t memory_segment_size =
+        (memory_size_ + num_init_partitions_ - 1) / num_init_partitions_;
+    const std::size_t memory_start = memory_segment_size * partition_id;
+    std::memset(reinterpret_cast<char *>(blob_->getMemoryMutable()) + memory_start,
+                0,
+                std::min(memory_segment_size, memory_size_ - memory_start));
+  }
+
+  /**
+   * @brief Use aggregation handles to update (multiple) aggregation states in
+   *        this vector table, with group-by keys and arguments drawn from the
+   *        given ValueAccessors.
+   *
+   * @param argument_ids The multi-source attribute IDs of each argument
+   *        component to be read from \p accessor_mux.
+   * @param key_ids The multi-source attribute IDs of each group-by key
+   *        component to be read from \p accessor_mux.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors which will be used to access keys. beginIteration()
+   *        should be called on the accessors before calling this method.
+   * @return Always return true.
+   **/
+  bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux) override;
+
+  /**
+   * @brief Copy the keys from this table to a NativeColumnVector, for the
+   *        specified partition.
+   *
+   * @param partition_id ID of the partition to copy keys from.
+   * @param output_cv The NativeColumnVector to copy keys to.
+   */
+  void finalizeKey(const std::size_t partition_id,
+                   NativeColumnVector *output_cv) const;
+
+
+  /**
+   * @brief Finalize the aggregation states to a NativeColumnVector, for the
+   *        specified partition and aggregation handle.
+   *
+   * @param partition_id ID of the partition to finalize.
+   * @param handle_id ID of the aggregation handle to finalize.
+   * @param output_cv The NativeColumnVector to write finalized values to.
+   */
+  void finalizeState(const std::size_t partition_id,
+                     const std::size_t handle_id,
+                     NativeColumnVector *output_cv) const;
+
+ private:
+  inline static std::size_t CacheLineAlignedBytes(const std::size_t actual_bytes) {
+    return (actual_bytes + kCacheLineBytes - 1) / kCacheLineBytes * kCacheLineBytes;
+  }
+
+  inline static std::size_t CalculateNumInitializationPartitions(
+      const std::size_t memory_size) {
+    // Set initialization memory block size as 4MB.
+    constexpr std::size_t kInitBlockSize = 4uL * 1024u * 1024u;
+
+    // At least 1 partition, at most 80 partitions.
+    // TODO(jianqiao): set the upbound as (# of workers * 2) instead of the
+    // hardcoded 80.
+    return std::max(1uL, std::min(memory_size / kInitBlockSize, 80uL));
+  }
+
+  inline static std::size_t CalculateNumFinalizationPartitions(
+      const std::size_t num_entries) {
+    // Set finalization segment size as 4096 entries.
+    constexpr std::size_t kFinalizeSegmentSize = 4uL * 1024L;
+
+    // At least 1 partition, at most 80 partitions.
+    // TODO(jianqiao): set the upbound as (# of workers * 2) instead of the
+    // hardcoded 80.
+    return std::max(1uL, std::min(num_entries / kFinalizeSegmentSize, 80uL));
+  }
+
+  inline std::size_t calculatePartitionLength() const {
+    const std::size_t partition_length =
+        (num_entries_ + num_finalize_partitions_ - 1) / num_finalize_partitions_;
+    DCHECK_GE(partition_length, 0u);
+    return partition_length;
+  }
+
+  inline std::size_t calculatePartitionStartPosition(
+      const std::size_t partition_id) const {
+    return calculatePartitionLength() * partition_id;
+  }
+
+  inline std::size_t calculatePartitionEndPosition(
+      const std::size_t partition_id) const {
+    return std::min(calculatePartitionLength() * (partition_id + 1),
+                    num_entries_);
+  }
+
+  template <bool use_two_accessors, typename ...ArgTypes>
+  inline void upsertValueAccessorDispatchHelper(
+      const bool is_key_nullable,
+      const bool is_argument_nullable,
+      ArgTypes &&...args);
+
+  template <bool ...bool_values, typename ...ArgTypes>
+  inline void upsertValueAccessorDispatchHelper(
+      const Type *key_type,
+      ArgTypes &&...args);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename ...ArgTypes>
+  inline void upsertValueAccessorDispatchHelper(
+      const Type *argument_type,
+      const AggregationID agg_id,
+      ArgTypes &&...args);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorCountHelper(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      void *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorSumHelper(
+      const Type *argument_type,
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      void *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <typename ...ArgTypes>
+  inline void upsertValueAccessorKeyOnlyHelper(
+      const bool is_key_nullable,
+      const Type *key_type,
+      ArgTypes &&...args);
+
+  template <bool is_key_nullable, typename KeyT, typename KeyValueAccessorT>
+  inline void upsertValueAccessorKeyOnly(
+      const attribute_id key_attr_id,
+      KeyValueAccessorT *key_accessor);
+
+  template <bool is_key_nullable, typename KeyT, typename KeyValueAccessorT>
+  inline void upsertValueAccessorCountNullary(
+      const attribute_id key_attr_id,
+      std::atomic<std::size_t> *vec_table,
+      KeyValueAccessorT *key_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, typename KeyT,
+            typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorCountUnary(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      std::atomic<std::size_t> *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename ArgumentT, typename StateT,
+            typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorIntegerSum(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      std::atomic<StateT> *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename ArgumentT, typename StateT,
+            typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorGenericSum(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      std::atomic<StateT> *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <typename KeyT>
+  inline void finalizeKeyInternal(const std::size_t start_position,
+                                  const std::size_t end_position,
+                                  NativeColumnVector *output_cv) const;
+
+  template <typename ...ArgTypes>
+  inline void finalizeStateDispatchHelper(const AggregationID agg_id,
+                                          const Type *argument_type,
+                                          const void *vec_table,
+                                          ArgTypes &&...args) const;
+
+  template <typename ...ArgTypes>
+  inline void finalizeStateSumHelper(const Type *argument_type,
+                                     const void *vec_table,
+                                     ArgTypes &&...args) const;
+
+  inline void finalizeStateCount(const std::atomic<std::size_t> *vec_table,
+                                 const std::size_t start_position,
+                                 const std::size_t end_position,
+                                 NativeColumnVector *output_cv) const;
+
+  template <typename ResultT, typename StateT>
+  inline void finalizeStateSum(const std::atomic<StateT> *vec_table,
+                               const std::size_t start_position,
+                               const std::size_t end_position,
+                               NativeColumnVector *output_cv) const;
+
+  const Type *key_type_;
+  const std::size_t num_entries_;
+
+  const std::size_t num_handles_;
+  const std::vector<AggregationHandle *> handles_;
+
+  std::unique_ptr<BarrieredReadWriteConcurrentBitVector> existence_map_;
+  std::vector<void *> vec_tables_;
+
+  const std::size_t num_finalize_partitions_;
+
+  StorageManager *storage_manager_;
+  MutableBlobReference blob_;
+
+  std::size_t memory_size_;
+  std::size_t num_init_partitions_;
+
+  DISALLOW_COPY_AND_ASSIGN(CollisionFreeVectorTable);
+};
+
+// ----------------------------------------------------------------------------
+// Implementations of template methods follow.
+
+template <bool use_two_accessors, typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorDispatchHelper(const bool is_key_nullable,
+                                        const bool is_argument_nullable,
+                                        ArgTypes &&...args) {
+  if (is_key_nullable) {
+    if (is_argument_nullable) {
+      upsertValueAccessorDispatchHelper<use_two_accessors, true, true>(
+          std::forward<ArgTypes>(args)...);
+    } else {
+      upsertValueAccessorDispatchHelper<use_two_accessors, true, false>(
+          std::forward<ArgTypes>(args)...);
+    }
+  } else {
+    if (is_argument_nullable) {
+      upsertValueAccessorDispatchHelper<use_two_accessors, false, true>(
+          std::forward<ArgTypes>(args)...);
+    } else {
+      upsertValueAccessorDispatchHelper<use_two_accessors, false, false>(
+          std::forward<ArgTypes>(args)...);
+    }
+  }
+}
+
+template <bool ...bool_values, typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorDispatchHelper(const Type *key_type,
+                                        ArgTypes &&...args) {
+  switch (key_type->getTypeID()) {
+    case TypeID::kInt:
+      upsertValueAccessorDispatchHelper<bool_values..., int>(
+          std::forward<ArgTypes>(args)...);
+      return;
+    case TypeID::kLong:
+      upsertValueAccessorDispatchHelper<bool_values..., std::int64_t>(
+          std::forward<ArgTypes>(args)...);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorDispatchHelper(const Type *argument_type,
+                                        const AggregationID agg_id,
+                                        ArgTypes &&...args) {
+  switch (agg_id) {
+     case AggregationID::kCount:
+       upsertValueAccessorCountHelper<
+           use_two_accessors, is_key_nullable, is_argument_nullable, KeyT>(
+               std::forward<ArgTypes>(args)...);
+       return;
+     case AggregationID::kSum:
+       upsertValueAccessorSumHelper<
+           use_two_accessors, is_key_nullable, is_argument_nullable, KeyT>(
+               argument_type, std::forward<ArgTypes>(args)...);
+       return;
+     default:
+       LOG(FATAL) << "Not supported";
+  }
+}
+
+template <typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorKeyOnlyHelper(const bool is_key_nullable,
+                                       const Type *key_type,
+                                       ArgTypes &&...args) {
+  switch (key_type->getTypeID()) {
+    case TypeID::kInt: {
+      if (is_key_nullable) {
+        upsertValueAccessorKeyOnly<true, int>(std::forward<ArgTypes>(args)...);
+      } else {
+        upsertValueAccessorKeyOnly<false, int>(std::forward<ArgTypes>(args)...);
+      }
+      return;
+    }
+    case TypeID::kLong: {
+      if (is_key_nullable) {
+        upsertValueAccessorKeyOnly<true, std::int64_t>(std::forward<ArgTypes>(args)...);
+      } else {
+        upsertValueAccessorKeyOnly<false, std::int64_t>(std::forward<ArgTypes>(args)...);
+      }
+      return;
+    }
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool is_key_nullable, typename KeyT, typename ValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorKeyOnly(const attribute_id key_attr_id,
+                                 ValueAccessorT *accessor) {
+  accessor->beginIteration();
+  while (accessor->next()) {
+    const KeyT *key = static_cast<const KeyT *>(
+        accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    existence_map_->setBit(*key);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorCountHelper(const attribute_id key_attr_id,
+                                     const attribute_id argument_id,
+                                     void *vec_table,
+                                     KeyValueAccessorT *key_accessor,
+                                     ArgumentValueAccessorT *argument_accessor) {
+  DCHECK_GE(key_attr_id, 0);
+
+  if (is_argument_nullable && argument_id != kInvalidAttributeID) {
+    upsertValueAccessorCountUnary<use_two_accessors, is_key_nullable, KeyT>(
+        key_attr_id,
+        argument_id,
+        static_cast<std::atomic<std::size_t> *>(vec_table),
+        key_accessor,
+        argument_accessor);
+    return;
+  } else {
+    upsertValueAccessorCountNullary<is_key_nullable, KeyT>(
+        key_attr_id,
+        static_cast<std::atomic<std::size_t> *>(vec_table),
+        key_accessor);
+    return;
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorSumHelper(const Type *argument_type,
+                                   const attribute_id key_attr_id,
+                                   const attribute_id argument_id,
+                                   void *vec_table,
+                                   KeyValueAccessorT *key_accessor,
+                                   ArgumentValueAccessorT *argument_accessor) {
+  DCHECK_GE(key_attr_id, 0);
+  DCHECK_GE(argument_id, 0);
+  DCHECK(argument_type != nullptr);
+
+  switch (argument_type->getTypeID()) {
+    case TypeID::kInt:
+      upsertValueAccessorIntegerSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, int>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<std::int64_t> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    case TypeID::kLong:
+      upsertValueAccessorIntegerSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, std::int64_t>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<std::int64_t> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    case TypeID::kFloat:
+      upsertValueAccessorGenericSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, float>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<double> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    case TypeID::kDouble:
+      upsertValueAccessorGenericSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, double>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<double> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool is_key_nullable, typename KeyT, typename ValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorCountNullary(const attribute_id key_attr_id,
+                                      std::atomic<std::size_t> *vec_table,
+                                      ValueAccessorT *accessor) {
+  accessor->beginIteration();
+  while (accessor->next()) {
+    const KeyT *key = static_cast<const KeyT *>(
+        accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    vec_table[loc].fetch_add(1u, std::memory_order_relaxed);
+    existence_map_->setBit(loc);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, typename KeyT,
+          typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorCountUnary(const attribute_id key_attr_id,
+                                    const attribute_id argument_id,
+                                    std::atomic<std::size_t> *vec_table,
+                                    KeyValueAccessorT *key_accessor,
+                                    ArgumentValueAccessorT *argument_accessor) {
+  key_accessor->beginIteration();
+  if (use_two_accessors) {
+    argument_accessor->beginIteration();
+  }
+  while (key_accessor->next()) {
+    if (use_two_accessors) {
+      argument_accessor->next();
+    }
+    const KeyT *key = static_cast<const KeyT *>(
+        key_accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    existence_map_->setBit(loc);
+    if (argument_accessor->getUntypedValue(argument_id) == nullptr) {
+      continue;
+    }
+    vec_table[loc].fetch_add(1u, std::memory_order_relaxed);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename ArgumentT, typename StateT,
+          typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorIntegerSum(const attribute_id key_attr_id,
+                                    const attribute_id argument_id,
+                                    std::atomic<StateT> *vec_table,
+                                    KeyValueAccessorT *key_accessor,
+                                    ArgumentValueAccessorT *argument_accessor) {
+  key_accessor->beginIteration();
+  if (use_two_accessors) {
+    argument_accessor->beginIteration();
+  }
+  while (key_accessor->next()) {
+    if (use_two_accessors) {
+      argument_accessor->next();
+    }
+    const KeyT *key = static_cast<const KeyT *>(
+        key_accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    existence_map_->setBit(loc);
+    const ArgumentT *argument = static_cast<const ArgumentT *>(
+        argument_accessor->template getUntypedValue<is_argument_nullable>(argument_id));
+    if (is_argument_nullable && argument == nullptr) {
+      continue;
+    }
+    vec_table[loc].fetch_add(*argument, std::memory_order_relaxed);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename ArgumentT, typename StateT,
+          typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorGenericSum(const attribute_id key_attr_id,
+                                    const attribute_id argument_id,
+                                    std::atomic<StateT> *vec_table,
+                                    KeyValueAccessorT *key_accessor,
+                                    ArgumentValueAccessorT *argument_accessor) {
+  key_accessor->beginIteration();
+  if (use_two_accessors) {
+    argument_accessor->beginIteration();
+  }
+  while (key_accessor->next()) {
+    if (use_two_accessors) {
+      argument_accessor->next();
+    }
+    const KeyT *key = static_cast<const KeyT *>(
+        key_accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    existence_map_->setBit(loc);
+    const ArgumentT *argument = static_cast<const ArgumentT *>(
+        argument_accessor->template getUntypedValue<is_argument_nullable>(argument_id));
+    if (is_argument_nullable && argument == nullptr) {
+      continue;
+    }
+    const ArgumentT arg_val = *argument;
+    std::atomic<StateT> &state = vec_table[loc];
+    StateT state_val = state.load(std::memory_order_relaxed);
+    while (!state.compare_exchange_weak(state_val, state_val + arg_val)) {}
+  }
+}
+
+template <typename KeyT>
+inline void CollisionFreeVectorTable
+    ::finalizeKeyInternal(const std::size_t start_position,
+                          const std::size_t end_position,
+                          NativeColumnVector *output_cv) const {
+  std::size_t loc = start_position - 1;
+  while ((loc = existence_map_->nextOne(loc)) < end_position) {
+    *static_cast<KeyT *>(output_cv->getPtrForDirectWrite()) = loc;
+  }
+}
+
+template <typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::finalizeStateDispatchHelper(const AggregationID agg_id,
+                                  const Type *argument_type,
+                                  const void *vec_table,
+                                  ArgTypes &&...args) const {
+  switch (agg_id) {
+     case AggregationID::kCount:
+       finalizeStateCount(static_cast<const std::atomic<std::size_t> *>(vec_table),
+                          std::forward<ArgTypes>(args)...);
+       return;
+     case AggregationID::kSum:
+       finalizeStateSumHelper(argument_type,
+                              vec_table,
+                              std::forward<ArgTypes>(args)...);
+       return;
+     default:
+       LOG(FATAL) << "Not supported";
+  }
+}
+
+template <typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::finalizeStateSumHelper(const Type *argument_type,
+                             const void *vec_table,
+                             ArgTypes &&...args) const {
+  DCHECK(argument_type != nullptr);
+
+  switch (argument_type->getTypeID()) {
+    case TypeID::kInt:    // Fall through
+    case TypeID::kLong:
+      finalizeStateSum<std::int64_t>(
+          static_cast<const std::atomic<std::int64_t> *>(vec_table),
+          std::forward<ArgTypes>(args)...);
+      return;
+    case TypeID::kFloat:  // Fall through
+    case TypeID::kDouble:
+      finalizeStateSum<double>(
+          static_cast<const std::atomic<double> *>(vec_table),
+          std::forward<ArgTypes>(args)...);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+inline void CollisionFreeVectorTable
+    ::finalizeStateCount(const std::atomic<std::size_t> *vec_table,
+                         const std::size_t start_position,
+                         const std::size_t end_position,
+                         NativeColumnVector *output_cv) const {
+  std::size_t loc = start_position - 1;
+  while ((loc = existence_map_->nextOne(loc)) < end_position) {
+    *static_cast<std::int64_t *>(output_cv->getPtrForDirectWrite()) =
+        vec_table[loc].load(std::memory_order_relaxed);
+  }
+}
+
+template <typename ResultT, typename StateT>
+inline void CollisionFreeVectorTable
+    ::finalizeStateSum(const std::atomic<StateT> *vec_table,
+                       const std::size_t start_position,
+                       const std::size_t end_position,
+                       NativeColumnVector *output_cv) const {
+  std::size_t loc = start_position - 1;
+  while ((loc = existence_map_->nextOne(loc)) < end_position) {
+    *static_cast<ResultT *>(output_cv->getPtrForDirectWrite()) =
+        vec_table[loc].load(std::memory_order_relaxed);
+  }
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_


[16/50] [abbrv] incubator-quickstep git commit: Moved hdfs-related flags into a file.

Posted by ji...@apache.org.
Moved hdfs-related flags into a file.


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

Branch: refs/heads/LIP-time-decomposition
Commit: dda085c7aeaa4f7f353c3392b547ca788d1507b8
Parents: 43a9b39
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sat Feb 4 14:46:44 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Sat Feb 4 16:25:03 2017 -0800

----------------------------------------------------------------------
 CMakeLists.txt              |  1 +
 cli/QuickstepCli.cpp        |  1 +
 storage/CMakeLists.txt      | 13 ++++++---
 storage/FileManagerHdfs.cpp | 35 +----------------------
 storage/FileManagerHdfs.hpp |  6 ----
 storage/Flags.cpp           | 62 ++++++++++++++++++++++++++++++++++++++++
 storage/Flags.hpp           | 51 +++++++++++++++++++++++++++++++++
 storage/StorageManager.cpp  |  5 +---
 storage/StorageManager.hpp  |  4 ---
 9 files changed, 126 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 85210ef..e74a274 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -789,6 +789,7 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryoptimizer_QueryHandle
                       quickstep_queryoptimizer_QueryProcessor
+                      quickstep_storage_Flags
                       quickstep_storage_PreloaderThread
                       quickstep_storage_StorageConstants
                       quickstep_storage_StorageManager

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 9db7577..02ec4ec 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -58,6 +58,7 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryProcessor.hpp"
+#include "storage/Flags.hpp"
 #include "storage/StorageConfig.h"  // For QUICKSTEP_HAVE_FILE_MANAGER_HDFS.
 
 #ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index fddea1f..a44c3a7 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -207,6 +207,7 @@ if (QUICKSTEP_HAVE_FILE_MANAGER_POSIX)
 elseif (QUICKSTEP_HAVE_FILE_MANAGER_WINDOWS)
   add_library(quickstep_storage_FileManagerWindows FileManagerWindows.cpp FileManagerWindows.hpp)
 endif()
+add_library(quickstep_storage_Flags Flags.cpp Flags.hpp)
 add_library(quickstep_storage_HashTable ../empty_src.cpp HashTable.hpp)
 add_library(quickstep_storage_HashTable_proto ${storage_HashTable_proto_srcs})
 add_library(quickstep_storage_HashTableBase ../empty_src.cpp HashTableBase.hpp)
@@ -679,8 +680,8 @@ target_link_libraries(quickstep_storage_FileManager
 if (QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
   target_link_libraries(quickstep_storage_FileManagerHdfs
                         glog
-                        ${GFLAGS_LIB_NAME}
                         quickstep_storage_FileManager
+                        quickstep_storage_Flags
                         quickstep_storage_StorageBlockInfo
                         quickstep_storage_StorageConstants
                         quickstep_storage_StorageErrors
@@ -711,6 +712,8 @@ elseif (QUICKSTEP_HAVE_FILE_MANAGER_WINDOWS)
                         quickstep_utility_Macros
                         quickstep_utility_StringUtil)
 endif()
+target_link_libraries(quickstep_storage_Flags
+                      ${GFLAGS_LIB_NAME})
 target_link_libraries(quickstep_storage_HashTable
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_HashTableBase
@@ -1000,6 +1003,7 @@ target_link_libraries(quickstep_storage_StorageManager
                       quickstep_storage_EvictionPolicy
                       quickstep_storage_FileManager
                       quickstep_storage_FileManagerLocal
+                      quickstep_storage_Flags
                       quickstep_storage_StorageBlob
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockBase
@@ -1121,17 +1125,18 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_CompressedTupleStorageSubBlock
                       quickstep_storage_CountedReference
                       quickstep_storage_EvictionPolicy
-                      quickstep_storage_FileManager
-                      quickstep_storage_FileManagerLocal
                       quickstep_storage_FastHashTable
                       quickstep_storage_FastHashTableFactory
                       quickstep_storage_FastSeparateChainingHashTable
+                      quickstep_storage_FileManager
+                      quickstep_storage_FileManagerLocal
+                      quickstep_storage_Flags
                       quickstep_storage_HashTable
-                      quickstep_storage_HashTable_proto
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTableKeyManager
                       quickstep_storage_HashTablePool
+                      quickstep_storage_HashTable_proto
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_IndexSubBlockDescriptionFactory
                       quickstep_storage_InsertDestination

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/FileManagerHdfs.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.cpp b/storage/FileManagerHdfs.cpp
index e28ed3b..937750a 100644
--- a/storage/FileManagerHdfs.cpp
+++ b/storage/FileManagerHdfs.cpp
@@ -31,12 +31,12 @@
 #include <cstring>
 #include <string>
 
+#include "storage/Flags.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageConstants.hpp"
 #include "storage/StorageErrors.hpp"
 #include "utility/StringUtil.hpp"
 
-#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 using std::size_t;
@@ -46,41 +46,8 @@ using std::string;
 
 namespace quickstep {
 
-DEFINE_string(hdfs_namenode_host, "localhost", "Host name or IP address of HDFS namenode.");
-
-static bool ValidateHdfsNameNodePort(const char *flagname,
-                                     std::int32_t value) {
-  if (value > 0 && value < 65536) {
-    return true;
-  } else {
-    std::fprintf(stderr, "--%s must be between 1 and 65535 (inclusive)\n", flagname);
-    return false;
-  }
-}
-DEFINE_int32(hdfs_namenode_port, 9000, "Port of HDFS namenode.");
-static const bool hdfs_namenode_port_dummy
-    = gflags::RegisterFlagValidator(&FLAGS_hdfs_namenode_port, &ValidateHdfsNameNodePort);
-
-static bool ValidateHdfsNumReplications(const char *flagname,
-                                        std::int32_t value) {
-  // The default dfs.namenode.replication.min is 1, while the default
-  // dfs.replication.max is 512.
-  if (value > 0) {
-    return true;
-  } else {
-    std::fprintf(stderr, "--%s must be at least 1\n", flagname);
-    return false;
-  }
-}
-DEFINE_int32(hdfs_num_replications, 1, "Number of HDFS file replications.");
-static const bool hdfs_num_replications_dummy
-    = gflags::RegisterFlagValidator(&FLAGS_hdfs_num_replications, &ValidateHdfsNumReplications);
-
 FileManagerHdfs::FileManagerHdfs(const string &storage_path)
     : FileManager(storage_path) {
-  DCHECK(hdfs_namenode_port_dummy);
-  DCHECK(hdfs_num_replications_dummy);
-
   struct hdfsBuilder *builder = hdfsNewBuilder();
   hdfsBuilderSetNameNode(builder, FLAGS_hdfs_namenode_host.c_str());
   hdfsBuilderSetNameNodePort(builder, FLAGS_hdfs_namenode_port);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/FileManagerHdfs.hpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.hpp b/storage/FileManagerHdfs.hpp
index 3c751b5..f47e4a8 100644
--- a/storage/FileManagerHdfs.hpp
+++ b/storage/FileManagerHdfs.hpp
@@ -29,14 +29,8 @@
 #include "storage/StorageBlockInfo.hpp"
 #include "utility/Macros.hpp"
 
-#include "gflags/gflags.h"
-
 namespace quickstep {
 
-DECLARE_string(hdfs_namenode_host);
-DECLARE_int32(hdfs_namenode_port);
-DECLARE_int32(hdfs_num_replications);
-
 /** \addtogroup Storage
  *  @{
  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/Flags.cpp
----------------------------------------------------------------------
diff --git a/storage/Flags.cpp b/storage/Flags.cpp
new file mode 100644
index 0000000..4672f81
--- /dev/null
+++ b/storage/Flags.cpp
@@ -0,0 +1,62 @@
+/**
+ * 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 "storage/Flags.hpp"
+
+#include "storage/StorageConfig.h"
+
+#include "gflags/gflags.h"
+
+namespace quickstep {
+
+#ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
+DEFINE_bool(use_hdfs, false, "Use HDFS as the persistent storage, instead of the local disk.");
+
+DEFINE_string(hdfs_namenode_host, "localhost", "Host name or IP address of HDFS namenode.");
+
+static bool ValidateHdfsNameNodePort(const char *flagname,
+                                     std::int32_t value) {
+  if (value > 0 && value < 65536) {
+    return true;
+  } else {
+    std::fprintf(stderr, "--%s must be between 1 and 65535 (inclusive)\n", flagname);
+    return false;
+  }
+}
+DEFINE_int32(hdfs_namenode_port, 9000, "Port of HDFS namenode.");
+static const bool hdfs_namenode_port_dummy
+    = gflags::RegisterFlagValidator(&FLAGS_hdfs_namenode_port, &ValidateHdfsNameNodePort);
+
+static bool ValidateHdfsNumReplications(const char *flagname,
+                                        std::int32_t value) {
+  // The default dfs.namenode.replication.min is 1, while the default
+  // dfs.replication.max is 512.
+  if (value > 0) {
+    return true;
+  } else {
+    std::fprintf(stderr, "--%s must be at least 1\n", flagname);
+    return false;
+  }
+}
+DEFINE_int32(hdfs_num_replications, 1, "Number of HDFS file replications.");
+static const bool hdfs_num_replications_dummy
+    = gflags::RegisterFlagValidator(&FLAGS_hdfs_num_replications, &ValidateHdfsNumReplications);
+#endif
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/Flags.hpp
----------------------------------------------------------------------
diff --git a/storage/Flags.hpp b/storage/Flags.hpp
new file mode 100644
index 0000000..1d5527c
--- /dev/null
+++ b/storage/Flags.hpp
@@ -0,0 +1,51 @@
+/**
+ * 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_STORAGE_FLAGS_HPP_
+#define QUICKSTEP_STORAGE_FLAGS_HPP_
+
+#include "storage/StorageConfig.h"  // For QUICKSTEP_HAVE_FILE_MANAGER_HDFS.
+
+#include "gflags/gflags_declare.h"
+
+namespace quickstep {
+
+/** \addtogroup STORAGE
+ *  @{
+ */
+
+/**
+ * @brief A collection of common flags shared by Quickstep STORAGEs in both the
+ * single-node and the distributed version.
+ **/
+
+#ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
+DECLARE_bool(use_hdfs);
+
+DECLARE_string(hdfs_namenode_host);
+DECLARE_int32(hdfs_namenode_port);
+DECLARE_int32(hdfs_num_replications);
+
+#endif
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_FLAGS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/StorageManager.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.cpp b/storage/StorageManager.cpp
index 6299cda..6f7d38b 100644
--- a/storage/StorageManager.cpp
+++ b/storage/StorageManager.cpp
@@ -76,6 +76,7 @@
 
 #include "storage/EvictionPolicy.hpp"
 #include "storage/FileManagerLocal.hpp"
+#include "storage/Flags.hpp"
 #include "storage/StorageBlob.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockBase.hpp"
@@ -189,10 +190,6 @@ DEFINE_uint64(buffer_pool_slots, 0,
 static const volatile bool buffer_pool_slots_dummy
     = gflags::RegisterFlagValidator(&FLAGS_buffer_pool_slots, &SetOrValidateBufferPoolSlots);
 
-#ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
-DEFINE_bool(use_hdfs, false, "Use HDFS as the persistent storage, instead of the local disk.");
-#endif
-
 StorageManager::StorageManager(
     const std::string &storage_path,
     const block_id_domain block_domain,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dda085c7/storage/StorageManager.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageManager.hpp b/storage/StorageManager.hpp
index b61f10a..42176ee 100644
--- a/storage/StorageManager.hpp
+++ b/storage/StorageManager.hpp
@@ -63,10 +63,6 @@ namespace quickstep {
 DECLARE_int32(block_domain);
 DECLARE_uint64(buffer_pool_slots);
 
-#ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
-DECLARE_bool(use_hdfs);
-#endif
-
 class CatalogRelationSchema;
 
 #ifdef QUICKSTEP_DISTRIBUTED


[33/50] [abbrv] incubator-quickstep git commit: Revert "A workaround to remove query result relation in the distributed version."

Posted by ji...@apache.org.
Revert "A workaround to remove query result relation in the distributed version."

This reverts commit aef1c3586580cfa72eb031fafe08700f6d5d9a86.


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

Branch: refs/heads/LIP-time-decomposition
Commit: dd8747fd7aba95ce2f0f325297e108bbac5d958d
Parents: 3b65b0f
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 00:02:27 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:35:53 2017 -0800

----------------------------------------------------------------------
 query_execution/ForemanDistributed.cpp | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dd8747fd/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index e6f22ec..4d95f16 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -201,12 +201,8 @@ void ForemanDistributed::run() {
 
         // TODO(quickstep-team): Dynamically scale-up/down Shiftbosses.
         if (query_result_saved_shiftbosses_[query_id].size() == shiftboss_directory_.size()) {
-          const relation_id result_relation_id = proto.relation_id();
-          processSaveQueryResultResponseMessage(proto.cli_id(), result_relation_id);
+          processSaveQueryResultResponseMessage(proto.cli_id(), proto.relation_id());
           query_result_saved_shiftbosses_.erase(query_id);
-
-          // TODO(zuyu): Refactor to clean-up blocks in Shiftbosses.
-          catalog_database_->dropRelationById(result_relation_id);
         }
         break;
       }


[34/50] [abbrv] incubator-quickstep git commit: Removed the temp query result relation in the distributed version.

Posted by ji...@apache.org.
Removed the temp query result relation in 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/aa7f6fe4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/aa7f6fe4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/aa7f6fe4

Branch: refs/heads/LIP-time-decomposition
Commit: aa7f6fe4e07804524aca0f1574935ae3f73c985d
Parents: dd8747f
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 00:33:31 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:35:53 2017 -0800

----------------------------------------------------------------------
 cli/distributed/CMakeLists.txt               |  1 +
 cli/distributed/Cli.cpp                      | 20 +++++++++++++++++++-
 cli/distributed/Conductor.cpp                | 13 ++++++++++++-
 cli/distributed/Conductor.hpp                |  4 ++++
 query_execution/QueryExecutionMessages.proto |  4 ++++
 query_execution/QueryExecutionTypedefs.hpp   |  2 ++
 6 files changed, 42 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/distributed/CMakeLists.txt b/cli/distributed/CMakeLists.txt
index b46082f..5804321 100644
--- a/cli/distributed/CMakeLists.txt
+++ b/cli/distributed/CMakeLists.txt
@@ -25,6 +25,7 @@ add_library(quickstep_cli_distributed_Role Role.cpp Role.hpp)
 # Link dependencies:
 target_link_libraries(quickstep_cli_distributed_Conductor
                       glog
+                      quickstep_catalog_CatalogDatabase
                       quickstep_cli_DefaultsConfigurator
                       quickstep_cli_Flags
                       quickstep_cli_distributed_Role

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/Cli.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Cli.cpp b/cli/distributed/Cli.cpp
index 5af70e6..386654d 100644
--- a/cli/distributed/Cli.cpp
+++ b/cli/distributed/Cli.cpp
@@ -122,7 +122,10 @@ void Cli::init() {
 
   // Prepare for submitting a query.
   bus_.RegisterClientAsSender(cli_id_, kSqlQueryMessage);
+
   bus_.RegisterClientAsReceiver(cli_id_, kQueryExecutionSuccessMessage);
+  bus_.RegisterClientAsSender(cli_id_, kQueryResultTeardownMessage);
+
   bus_.RegisterClientAsReceiver(cli_id_, kQueryExecutionErrorMessage);
 }
 
@@ -191,7 +194,7 @@ void Cli::run() {
             CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 
             if (proto.has_result_relation()) {
-              CatalogRelation result_relation(proto.result_relation());
+              const CatalogRelation result_relation(proto.result_relation());
 
               PrintToScreen::PrintRelation(result_relation, storage_manager_.get(), stdout);
 
@@ -199,6 +202,21 @@ void Cli::run() {
               for (const block_id block : blocks) {
                 storage_manager_->deleteBlockOrBlobFile(block);
               }
+
+              // Notify Conductor to remove the temp query result relation in the Catalog.
+              S::QueryResultTeardownMessage proto_response;
+              proto_response.set_relation_id(result_relation.getID());
+
+              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 response_message(static_cast<const void*>(proto_response_bytes),
+                                             proto_response_length,
+                                             kQueryResultTeardownMessage);
+              free(proto_response_bytes);
+
+              QueryExecutionUtil::SendTMBMessage(&bus_, cli_id_, conductor_client_id_, move(response_message));
             }
 
             std::chrono::duration<double, std::milli> time_in_ms = end - start;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/Conductor.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Conductor.cpp b/cli/distributed/Conductor.cpp
index 13d4d57..cf2eb4b 100644
--- a/cli/distributed/Conductor.cpp
+++ b/cli/distributed/Conductor.cpp
@@ -28,6 +28,7 @@
 #include <string>
 #include <utility>
 
+#include "catalog/CatalogDatabase.hpp"
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/Flags.hpp"
 #include "parser/ParseStatement.hpp"
@@ -73,6 +74,7 @@ void Conductor::init() {
     }
 
     query_processor_ = make_unique<QueryProcessor>(move(catalog_path));
+    catalog_database_ = query_processor_->getDefaultDatabase();
   } catch (const std::exception &e) {
     LOG(FATAL) << "FATAL ERROR DURING STARTUP: " << e.what()
                << "\nIf you intended to create a new database, "
@@ -93,12 +95,14 @@ void Conductor::init() {
   bus_.RegisterClientAsSender(conductor_client_id_, kQueryExecutionErrorMessage);
   bus_.RegisterClientAsSender(conductor_client_id_, kAdmitRequestMessage);
 
+  bus_.RegisterClientAsReceiver(conductor_client_id_, kQueryResultTeardownMessage);
+
   block_locator_ = make_unique<BlockLocator>(&bus_);
   block_locator_->start();
 
   foreman_ = make_unique<ForemanDistributed>(*block_locator_,
                                              std::bind(&QueryProcessor::saveCatalog, query_processor_.get()), &bus_,
-                                             query_processor_->getDefaultDatabase());
+                                             catalog_database_);
   foreman_->start();
 }
 
@@ -129,6 +133,13 @@ void Conductor::run() {
         processSqlQueryMessage(sender, new string(move(proto.sql_query())));
         break;
       }
+      case kQueryResultTeardownMessage: {
+        S::QueryResultTeardownMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        catalog_database_->dropRelationById(proto.relation_id());
+        break;
+      }
       default:
         LOG(FATAL) << "Unknown TMB message type";
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/Conductor.hpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Conductor.hpp b/cli/distributed/Conductor.hpp
index e8c9582..09bf2b9 100644
--- a/cli/distributed/Conductor.hpp
+++ b/cli/distributed/Conductor.hpp
@@ -34,6 +34,8 @@
 
 namespace quickstep {
 
+class CatalogDatabase;
+
 /** \addtogroup CliDistributed
  *  @{
  */
@@ -63,6 +65,8 @@ class Conductor final : public Role {
   SqlParserWrapper parser_wrapper_;
 
   std::unique_ptr<QueryProcessor> query_processor_;
+  // Not owned.
+  CatalogDatabase *catalog_database_;
 
   tmb::client_id conductor_client_id_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 115a9a3..68f286d 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -138,6 +138,10 @@ message QueryExecutionSuccessMessage {
   optional CatalogRelationSchema result_relation = 1;
 }
 
+message QueryResultTeardownMessage {
+  required int32 relation_id = 1;
+}
+
 message QueryExecutionErrorMessage {
   required string error_message = 1;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 9f78302..994bd60 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -105,6 +105,8 @@ enum QueryExecutionMessageType : message_type_id {
   kQueryExecutionSuccessMessage,
   kQueryExecutionErrorMessage,
 
+  kQueryResultTeardownMessage,  // From CLI to Conductor.
+
   // BlockLocator related messages, sorted in a life cycle of StorageManager
   // with a unique block domain.
   kBlockDomainRegistrationMessage,  // From Worker to BlockLocator.


[11/50] [abbrv] incubator-quickstep git commit: Revert "Fixed the linking issue for the distributed cli."

Posted by ji...@apache.org.
Revert "Fixed the linking issue for the distributed cli."

This reverts commit dff4a145e2c2d3d7b84fb259e48e425310a52a8a.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 6ec9e9d3d33dd9e154a00c2a7d1beaf60ea9e279
Parents: 50f3ba6
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 2 18:14:12 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 2 18:14:12 2017 -0800

----------------------------------------------------------------------
 cli/distributed/CMakeLists.txt | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6ec9e9d3/cli/distributed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/distributed/CMakeLists.txt b/cli/distributed/CMakeLists.txt
index 1069abd..a00ffda 100644
--- a/cli/distributed/CMakeLists.txt
+++ b/cli/distributed/CMakeLists.txt
@@ -28,7 +28,6 @@ target_link_libraries(quickstep_cli_distributed_Cli
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_cli_Flags
-                      quickstep_cli_LineReader
                       quickstep_cli_PrintToScreen
                       quickstep_cli_distributed_Role
                       quickstep_parser_ParseStatement


[44/50] [abbrv] incubator-quickstep git commit: Checked the query completion upon each rebuild message in the distributed version.

Posted by ji...@apache.org.
Checked the query completion upon each rebuild message in 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/167ee872
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/167ee872
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/167ee872

Branch: refs/heads/LIP-time-decomposition
Commit: 167ee872e4d3b52dcf15b014de20c66b51db0075
Parents: c40c553
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Feb 9 16:34:26 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Feb 9 16:34:26 2017 -0800

----------------------------------------------------------------------
 query_execution/PolicyEnforcerDistributed.cpp | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/167ee872/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 12d2037..8f0332d 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -159,6 +159,11 @@ void PolicyEnforcerDistributed::processInitiateRebuildResponseMessage(const tmb:
   query_manager->processInitiateRebuildResponseMessage(
       proto.operator_index(), num_rebuild_work_orders, shiftboss_index);
   shiftboss_directory_->addNumQueuedWorkOrders(shiftboss_index, num_rebuild_work_orders);
+
+  if (query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+    onQueryCompletion(query_manager);
+    removeQuery(query_id);
+  }
 }
 
 void PolicyEnforcerDistributed::getShiftbossIndexForAggregation(


[04/50] [abbrv] incubator-quickstep git commit: Reorder output attribute order to improve copy performance.

Posted by ji...@apache.org.
Reorder output attribute order to improve copy performance.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 6d83b46af25b35fb0b3a23452b6fbd2842b33793
Parents: 23e14b8
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jan 12 18:41:17 2017 -0600
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 31 00:10:45 2017 -0800

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/PhysicalGenerator.cpp           |  12 +
 query_optimizer/rules/CMakeLists.txt            |  14 +
 query_optimizer/rules/ReorderColumns.cpp        | 214 ++++++++++++++++
 query_optimizer/rules/ReorderColumns.hpp        |  75 ++++++
 query_optimizer/tests/OptimizerTextTest.cpp     |   6 +-
 relational_operators/CMakeLists.txt             |   1 +
 relational_operators/HashJoinOperator.cpp       | 254 +++++++++++--------
 relational_operators/HashJoinOperator.hpp       |   4 +
 storage/SplitRowStoreValueAccessor.hpp          |   5 +
 storage/ValueAccessor.hpp                       |  30 +++
 types/containers/ColumnVectorsValueAccessor.hpp |   5 +
 12 files changed, 515 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index b6c794d..e8bc21c 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..e12f8be 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
+#include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
@@ -44,6 +45,10 @@
 namespace quickstep {
 namespace optimizer {
 
+DEFINE_bool(reorder_columns, true,
+            "Adjust the ordering of intermediate relations' columns to improve "
+            "copy performance.");
+
 DEFINE_bool(reorder_hash_joins, true,
             "If true, apply hash join order optimization to each group of hash "
             "joins. The optimization applies a greedy algorithm to favor smaller "
@@ -109,6 +114,13 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   } else {
     rules.emplace_back(new SwapProbeBuild());
   }
+  if (FLAGS_reorder_columns) {
+    // NOTE(jianqiao): This optimization relies on the fact that the intermediate
+    // relations all have SPLIT_ROW_STORE layouts. If this fact gets changed, the
+    // optimization algorithm may need to be updated and the performance impact
+    // should be re-evaluated.
+    rules.emplace_back(new ReorderColumns());
+  }
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 7fffadc..fe2fd17 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -25,6 +25,7 @@ add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp Gener
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
+add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
@@ -118,6 +119,18 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_Rule
                       glog
                       quickstep_utility_Macros)
@@ -213,6 +226,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
+                      quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/rules/ReorderColumns.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.cpp b/query_optimizer/rules/ReorderColumns.cpp
new file mode 100644
index 0000000..f7e58d5
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.cpp
@@ -0,0 +1,214 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_optimizer/rules/ReorderColumns.hpp"
+
+#include <algorithm>
+#include <cstddef>
+#include <limits>
+#include <unordered_map>
+#include <vector>
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/NamedExpression.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/Selection.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr ReorderColumns::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr &input,
+                                             const bool lock_ordering) {
+  // We have to guarantee that the top level ordering of the columns remain
+  // unchanged so that the output columns are ordered as specified by the user.
+  // So here we use the flag "lock_ordering" to skip the first transformable
+  // node (i.e. the first Selection or HashJoin).
+  const bool is_not_transformable = !IsTransformable(input);
+  const bool skip_transform = lock_ordering || is_not_transformable;
+
+  if (skip_transform) {
+    std::vector<P::PhysicalPtr> new_children;
+    for (const P::PhysicalPtr &child : input->children()) {
+      new_children.emplace_back(applyInternal(child, lock_ordering && is_not_transformable));
+    }
+
+    if (new_children != input->children()) {
+      return input->copyWithNewChildren(new_children);
+    } else {
+      return input;
+    }
+  }
+
+  // Collect the maximal chain of transformable nodes.
+  std::vector<P::PhysicalPtr> nodes;
+  for (P::PhysicalPtr node = input; IsTransformable(node); node = node->children().front()) {
+    nodes.emplace_back(node);
+  }
+  // Arrange the nodes with bottom-up order.
+  std::reverse(nodes.begin(), nodes.end());
+
+  // A greedy algorithm that reorders the output attributes based on the GEN/KILL
+  // intervals. This algorithm works well with SSB/TPCH queries and is not likely
+  // to make the plans worse for whatever queries.
+  //
+  // Here is a brief explanation of the three data structure base/gen/kill.
+  //   (1) base: maps each attribute's id to its position in the BASE relation's
+  //             output attributes. Note that the base relation is the child
+  //             relation of nodes[0].
+  //   (2) gen:  maps each attribute's id to the MINIMUM index i such that the
+  //             attribute is among nodes[i]'s output attributes. I.e. node i
+  //             GENERATEs the attribute.
+  //   (3) kill: maps each attribute's id to the MAXIMUM index i such that the
+  //             attribute is among nodes[i]'s output attributes. I.e. node i+1
+  //             KILLs the attribute.
+  std::unordered_map<E::ExprId, std::size_t> base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+      applyInternal(nodes.front()->children().front(), false);
+  const std::vector<E::AttributeReferencePtr> base_attrs =
+      base_node->getOutputAttributes();
+  for (std::size_t i = 0; i < base_attrs.size(); ++i) {
+    base.emplace(base_attrs[i]->id(), i);
+  }
+
+  for (std::size_t i = 0; i < nodes.size(); ++i) {
+    for (const auto &attr : nodes[i]->getOutputAttributes()) {
+      const E::ExprId attr_id = attr->id();
+      if (gen.find(attr_id) == gen.end()) {
+        gen.emplace(attr_id, i);
+      }
+      kill[attr_id] = i;
+    }
+  }
+
+  // TODO(jianqiao): implement this comparator as a standalone and well-documented
+  // struct.
+  const auto comparator = [&gen, &kill, &base](const E::NamedExpressionPtr &lhs,
+                                               const E::NamedExpressionPtr &rhs) -> bool {
+    const E::ExprId lhs_id = lhs->id();
+    const E::ExprId rhs_id = rhs->id();
+
+    // Sort the attributes first by GEN location.
+    const std::size_t lhs_gen = gen.at(lhs_id);
+    const std::size_t rhs_gen = gen.at(rhs_id);
+    if (lhs_gen != rhs_gen) {
+      return lhs_gen < rhs_gen;
+    }
+
+    // Then by KILL location.
+    const std::size_t lhs_kill = kill.at(lhs_id);
+    const std::size_t rhs_kill = kill.at(rhs_id);
+    if (lhs_kill != rhs_kill) {
+      return lhs_kill < rhs_kill;
+    }
+
+    // Finally by the ordering in the base relaton.
+    const auto lhs_base_it = base.find(lhs_id);
+    const auto rhs_base_it = base.find(rhs_id);
+    const std::size_t lhs_base =
+        lhs_base_it == base.end() ? std::numeric_limits<std::size_t>::max()
+                                  : lhs_base_it->second;
+    const std::size_t rhs_base =
+        rhs_base_it == base.end() ? std::numeric_limits<std::size_t>::max()
+                                  : rhs_base_it->second;
+    if (lhs_base != rhs_base) {
+      return lhs_base < rhs_base;
+    }
+
+    return lhs_id < rhs_id;
+  };
+
+  P::PhysicalPtr output = base_node;
+
+  for (const auto &node : nodes) {
+    std::vector<E::NamedExpressionPtr> project_expressions;
+    switch (node->getPhysicalType()) {
+      case P::PhysicalType::kHashJoin: {
+        project_expressions =
+            std::static_pointer_cast<const P::HashJoin>(node)->project_expressions();
+        break;
+      }
+      case P::PhysicalType::kSelection: {
+        project_expressions =
+            std::static_pointer_cast<const P::Selection>(node)->project_expressions();
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unsupported physical type";
+    }
+
+    std::sort(project_expressions.begin(), project_expressions.end(), comparator);
+
+    switch (node->getPhysicalType()) {
+      case P::PhysicalType::kHashJoin: {
+        const P::HashJoinPtr old_node =
+            std::static_pointer_cast<const P::HashJoin>(node);
+        output = P::HashJoin::Create(output,
+                                     applyInternal(old_node->right(), false),
+                                     old_node->left_join_attributes(),
+                                     old_node->right_join_attributes(),
+                                     old_node->residual_predicate(),
+                                     project_expressions,
+                                     old_node->join_type());
+        break;
+      }
+      case P::PhysicalType::kSelection: {
+        const P::SelectionPtr old_node =
+            std::static_pointer_cast<const P::Selection>(node);
+        output = P::Selection::Create(output,
+                                      project_expressions,
+                                      old_node->filter_predicate());
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unsupported physical type";
+    }
+  }
+
+  return output;
+}
+
+bool ReorderColumns::IsTransformable(const physical::PhysicalPtr &input) {
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kHashJoin:  // Fall through
+    case P::PhysicalType::kSelection:
+      return true;
+    default:
+      return false;
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/rules/ReorderColumns.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.hpp b/query_optimizer/rules/ReorderColumns.hpp
new file mode 100644
index 0000000..36fa183
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.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_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_
+
+#include <string>
+
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to adjust the orderings of some
+ *        intermediate nodes' output attributes to improve copy performance.
+ *
+ * @note This optimization is based on the fact that the intermediate relations
+ *       all have SPLIT_ROW_STORE layouts. If this fact gets changed, the rule's
+ *       algorithm may need to be updated and the performance impact should be
+ *       re-evaluated.
+ */
+class ReorderColumns : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  ReorderColumns() {}
+
+  ~ReorderColumns() override {}
+
+  std::string getName() const override {
+    return "ReorderColumns";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
+                                      const bool lock_ordering);
+
+  // Whether the physical node can
+  inline static bool IsTransformable(const physical::PhysicalPtr &input);
+
+  DISALLOW_COPY_AND_ASSIGN(ReorderColumns);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/tests/OptimizerTextTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTest.cpp b/query_optimizer/tests/OptimizerTextTest.cpp
index 759c173..e17f5c4 100644
--- a/query_optimizer/tests/OptimizerTextTest.cpp
+++ b/query_optimizer/tests/OptimizerTextTest.cpp
@@ -31,6 +31,7 @@
 namespace quickstep {
 namespace optimizer {
 
+DECLARE_bool(reorder_columns);
 DECLARE_bool(reorder_hash_joins);
 DECLARE_bool(use_lip_filters);
 
@@ -58,8 +59,9 @@ int main(int argc, char** argv) {
   test_driver->registerOptions(
       quickstep::optimizer::OptimizerTextTestRunner::kTestOptions);
 
-  // Turn off join order optimization and LIPFilter for optimizer test since
-  // it is up to change and affects a large number of test cases.
+  // Turn off some optimization rules for optimizer test since they are up to
+  // change and affects a large number of test cases.
+  quickstep::optimizer::FLAGS_reorder_columns = false;
   quickstep::optimizer::FLAGS_reorder_hash_joins = false;
   quickstep::optimizer::FLAGS_use_lip_filters = false;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index b2e08cf..c8447f3 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_catalog_PartitionSchemeHeader
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
+                      quickstep_expressions_scalar_ScalarAttribute
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 7394554..0e75411 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -31,6 +31,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
+#include "expressions/scalar/ScalarAttribute.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
@@ -64,6 +65,9 @@ namespace quickstep {
 
 namespace {
 
+typedef std::vector<std::pair<tuple_id, tuple_id>> VectorOfTupleIdPair;
+typedef std::pair<std::vector<tuple_id>, std::vector<tuple_id>> PairOfTupleIdVector;
+
 // Functor passed to HashTable::getAllFromValueAccessor() to collect matching
 // tuples from the inner relation. It stores matching tuple ID pairs
 // in an unordered_map keyed by inner block ID and a vector of
@@ -83,8 +87,7 @@ class VectorsOfPairsJoinedTuplesCollector {
   // 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
   // right.
-  inline std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>>*
-      getJoinedTuples() {
+  inline std::unordered_map<block_id, VectorOfTupleIdPair>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -94,7 +97,7 @@ class VectorsOfPairsJoinedTuplesCollector {
   // cross-product of all tuples from both blocks, but simply using pairs of
   // tuple-IDs is expected to be more space efficient if the result set is less
   // than 1/64 the cardinality of the cross-product.
-  std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>> joined_tuples_;
+  std::unordered_map<block_id, VectorOfTupleIdPair> joined_tuples_;
 };
 
 // Another collector using an unordered_map keyed on inner block just like above,
@@ -107,15 +110,15 @@ class PairsOfVectorsJoinedTuplesCollector {
   template <typename ValueAccessorT>
   inline void operator()(const ValueAccessorT &accessor,
                          const TupleReference &tref) {
-    joined_tuples_[tref.block].first.push_back(tref.tuple);
-    joined_tuples_[tref.block].second.push_back(accessor.getCurrentPosition());
+    auto &entry = joined_tuples_[tref.block];
+    entry.first.emplace_back(tref.tuple);
+    entry.second.emplace_back(accessor.getCurrentPosition());
   }
 
   // Get a mutable pointer to the collected map of joined tuple ID pairs. The
   // key is inner block_id, value is a pair consisting of
   // inner block tuple IDs (first) and outer block tuple IDs (second).
-  inline std::unordered_map< block_id, std::pair<std::vector<tuple_id>, std::vector<tuple_id>>>*
-      getJoinedTuples() {
+  inline std::unordered_map<block_id, PairOfTupleIdVector>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -166,12 +169,6 @@ class OuterJoinTupleCollector {
   TupleIdSequence *filter_;
 };
 
-// For InnerJoin.
-constexpr std::size_t kNumValueAccessors = 3u;
-constexpr std::size_t kBuildValueAccessorIndex = 0,
-                      kProbeValueAccessorIndex = 1u,
-                      kTempResultValueAccessorIndex = 2u;
-
 }  // namespace
 
 bool HashJoinOperator::getAllWorkOrders(
@@ -473,16 +470,93 @@ void HashInnerJoinWorkOrder::execute() {
         base_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
   }
 
+  if (probe_accessor->getImplementationType() == ValueAccessor::Implementation::kSplitRowStore) {
+    executeWithCopyElision(probe_accessor.get());
+  } else {
+    executeWithoutCopyElision(probe_accessor.get());
+  }
+}
+
+void HashInnerJoinWorkOrder::executeWithoutCopyElision(ValueAccessor *probe_accessor) {
+  VectorsOfPairsJoinedTuplesCollector collector;
+  if (join_key_attributes_.size() == 1) {
+    hash_table_.getAllFromValueAccessor(
+        probe_accessor,
+        join_key_attributes_.front(),
+        any_join_key_attributes_nullable_,
+        &collector);
+  } else {
+    hash_table_.getAllFromValueAccessorCompositeKey(
+        probe_accessor,
+        join_key_attributes_,
+        any_join_key_attributes_nullable_,
+        &collector);
+  }
+
+  const relation_id build_relation_id = build_relation_.getID();
+  const relation_id probe_relation_id = probe_relation_.getID();
+
+  for (std::pair<const block_id, VectorOfTupleIdPair>
+           &build_block_entry : *collector.getJoinedTuples()) {
+    BlockReference build_block =
+        storage_manager_->getBlock(build_block_entry.first, build_relation_);
+    const TupleStorageSubBlock &build_store = build_block->getTupleStorageSubBlock();
+    std::unique_ptr<ValueAccessor> build_accessor(build_store.createValueAccessor());
+
+    // Evaluate '*residual_predicate_', if any.
+    //
+    // TODO(chasseur): We might consider implementing true vectorized
+    // evaluation for join predicates that are not equijoins (although in
+    // general that would require evaluating and materializing some expressions
+    // over the cross-product of all tuples in a pair of blocks in order to
+    // evaluate the predicate). We could use a heuristic where we only do the
+    // vectorized materialization and evaluation if the set of matches from the
+    // hash join is below a reasonable threshold so that we don't blow up
+    // temporary memory requirements to an unreasonable degree.
+    if (residual_predicate_ != nullptr) {
+      VectorOfTupleIdPair filtered_matches;
+
+      for (const std::pair<tuple_id, tuple_id> &hash_match
+           : build_block_entry.second) {
+        if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
+                                                        build_relation_id,
+                                                        hash_match.first,
+                                                        *probe_accessor,
+                                                        probe_relation_id,
+                                                        hash_match.second)) {
+          filtered_matches.emplace_back(hash_match);
+        }
+      }
+
+      build_block_entry.second = std::move(filtered_matches);
+    }
+
+    ColumnVectorsValueAccessor temp_result;
+    for (auto selection_cit = selection_.begin();
+         selection_cit != selection_.end();
+         ++selection_cit) {
+      temp_result.addColumn((*selection_cit)->getAllValuesForJoin(build_relation_id,
+                                                                  build_accessor.get(),
+                                                                  probe_relation_id,
+                                                                  probe_accessor,
+                                                                  build_block_entry.second));
+    }
+
+    output_destination_->bulkInsertTuples(&temp_result);
+  }
+}
+
+void HashInnerJoinWorkOrder::executeWithCopyElision(ValueAccessor *probe_accessor) {
   PairsOfVectorsJoinedTuplesCollector collector;
   if (join_key_attributes_.size() == 1) {
     hash_table_.getAllFromValueAccessor(
-        probe_accessor.get(),
+        probe_accessor,
         join_key_attributes_.front(),
         any_join_key_attributes_nullable_,
         &collector);
   } else {
     hash_table_.getAllFromValueAccessorCompositeKey(
-        probe_accessor.get(),
+        probe_accessor,
         join_key_attributes_,
         any_join_key_attributes_nullable_,
         &collector);
@@ -491,7 +565,37 @@ void HashInnerJoinWorkOrder::execute() {
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
 
-  for (std::pair<const block_id, std::pair<std::vector<tuple_id>, std::vector<tuple_id>>>
+  constexpr std::size_t kNumIndexes = 3u;
+  constexpr std::size_t kBuildIndex = 0, kProbeIndex = 1u, kTempIndex = 2u;
+
+  // Create a map of ValueAccessors and what attributes we want to pick from them.
+  std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map(
+      kNumIndexes, std::make_pair(nullptr /* late binding ValueAccessor */,
+                                  vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
+
+  std::vector<const Scalar *> non_trivial_expressions;
+  attribute_id dest_attr = 0;
+
+  for (const auto &scalar : selection_) {
+    // If the Scalar (column) is not an attribute in build/probe blocks, we will
+    // insert it into a ColumnVectorsValueAccessor.
+    if (scalar->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
+      // Current destination attribute maps to the column we'll create now.
+      accessor_attribute_map[kTempIndex].second[dest_attr] = non_trivial_expressions.size();
+      non_trivial_expressions.emplace_back(scalar.get());
+    } else {
+      const CatalogAttribute &attr = static_cast<const ScalarAttribute *>(scalar.get())->getAttribute();
+      const attribute_id attr_id = attr.getID();
+      if (attr.getParent().getID() == build_relation_id) {
+        accessor_attribute_map[kBuildIndex].second[dest_attr] = attr_id;
+      } else {
+        accessor_attribute_map[kProbeIndex].second[dest_attr] = attr_id;
+      }
+    }
+    ++dest_attr;
+  }
+
+  for (std::pair<const block_id, PairOfTupleIdVector>
            &build_block_entry : *collector.getJoinedTuples()) {
     BlockReference build_block =
         storage_manager_->getBlock(build_block_entry.first, build_relation_);
@@ -511,7 +615,8 @@ void HashInnerJoinWorkOrder::execute() {
     // hash join is below a reasonable threshold so that we don't blow up
     // temporary memory requirements to an unreasonable degree.
     if (residual_predicate_ != nullptr) {
-      std::pair<std::vector<tuple_id>, std::vector<tuple_id>> filtered_matches;
+      PairOfTupleIdVector filtered_matches;
+
       for (std::size_t i = 0; i < build_tids.size(); ++i) {
         if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
                                                         build_relation_id,
@@ -519,110 +624,51 @@ void HashInnerJoinWorkOrder::execute() {
                                                         *probe_accessor,
                                                         probe_relation_id,
                                                         probe_tids[i])) {
-          filtered_matches.first.push_back(build_tids[i]);
-          filtered_matches.second.push_back(probe_tids[i]);
+          filtered_matches.first.emplace_back(build_tids[i]);
+          filtered_matches.second.emplace_back(probe_tids[i]);
         }
       }
 
       build_block_entry.second = std::move(filtered_matches);
     }
 
-    // TODO(chasseur): If all the output expressions are ScalarAttributes,
-    // we could implement a similar fast-path to StorageBlock::selectSimple()
-    // that avoids a copy.
-    //
     // TODO(chasseur): See TODO in NestedLoopsJoinOperator.cpp about limiting
     // the size of materialized temporary results. In common usage, this
     // probably won't be an issue for hash-joins, but in the worst case a hash
     // join can still devolve into a cross-product.
-    //
-    // NOTE(chasseur): We could also create one big ColumnVectorsValueAccessor
-    // and accumulate all the results across multiple block pairs into it
-    // before inserting anything into output blocks, but this would require
-    // some significant API extensions to the expressions system for a dubious
-    // benefit (probably only a real performance win when there are very few
-    // matching tuples in each individual inner block but very many inner
-    // blocks with at least one match).
-
-    // We now create ordered value accessors for both build and probe side,
-    // using the joined tuple TIDs. Note that we have to use this Lambda-based
-    // invocation method here because the accessors don't have a virtual
-    // function that creates such an OrderedTupleIdSequenceAdapterValueAccessor.
-    std::unique_ptr<ValueAccessor> ordered_build_accessor, ordered_probe_accessor;
-    InvokeOnValueAccessorNotAdapter(
-        build_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_build_accessor.reset(
-              accessor->createSharedOrderedTupleIdSequenceAdapter(build_tids));
-        });
-
-    if (probe_accessor->isTupleIdSequenceAdapter()) {
-      InvokeOnTupleIdSequenceAdapterValueAccessor(
-        probe_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_probe_accessor.reset(
-            accessor->createSharedOrderedTupleIdSequenceAdapter(probe_tids));
-        });
-    } else {
-      InvokeOnValueAccessorNotAdapter(
-        probe_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_probe_accessor.reset(
-            accessor->createSharedOrderedTupleIdSequenceAdapter(probe_tids));
-        });
-    }
 
     // We also need a temp value accessor to store results of any scalar expressions.
     ColumnVectorsValueAccessor temp_result;
+    if (!non_trivial_expressions.empty()) {
+      // The getAllValuesForJoin function below needs joined tuple IDs as a
+      // vector of pair of (build-tuple-ID, probe-tuple-ID), and we have a pair
+      // of (build-tuple-IDs-vector, probe-tuple-IDs-vector). So we'll have to
+      // zip our two vectors together.
+      VectorOfTupleIdPair zipped_joined_tuple_ids;
+      for (std::size_t i = 0; i < build_tids.size(); ++i) {
+        zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
+      }
 
-    // Create a map of ValueAccessors and what attributes we want to pick from them
-    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map(
-        kNumValueAccessors, std::make_pair(nullptr,  // A late binding ValueAccessor.
-                                           vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
-
-    accessor_attribute_map[kBuildValueAccessorIndex].first = ordered_build_accessor.get();
-    accessor_attribute_map[kProbeValueAccessorIndex].first = ordered_probe_accessor.get();
-    accessor_attribute_map[kTempResultValueAccessorIndex].first = &temp_result;
-
-    attribute_id dest_attr = 0;
-    for (auto &selection_cit : selection_) {
-      // If the Scalar (column) is not an attribute in build/probe blocks, then
-      // insert it into a ColumnVectorsValueAccessor.
-      if (selection_cit->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
-        // Current destination attribute maps to the column we'll create now.
-        accessor_attribute_map[kTempResultValueAccessorIndex].second[dest_attr] = temp_result.getNumColumns();
-
-        std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
-        if (temp_result.getNumColumns() == 0) {
-          // The getAllValuesForJoin function below needs joined tuple IDs as
-          // a vector of pair of (build-tuple-ID, probe-tuple-ID), and we have
-          // a pair of (build-tuple-IDs-vector, probe-tuple-IDs-vector). So
-          // we'll have to zip our two vectors together. We do this inside
-          // the loop because most queries don't exercise this code since
-          // they don't have scalar expressions with attributes from both
-          // build and probe relations (other expressions would have been
-          // pushed down to before the join).
-          for (std::size_t i = 0; i < build_tids.size(); ++i) {
-            zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
-          }
-        }
-        temp_result.addColumn(
-            selection_cit
-                ->getAllValuesForJoin(build_relation_id, build_accessor.get(),
-                                      probe_relation_id, probe_accessor.get(),
-                                      zipped_joined_tuple_ids));
-      } else {
-        const CatalogAttribute &attr = static_cast<const ScalarAttribute *>(selection_cit.get())->getAttribute();
-        const attribute_id attr_id = attr.getID();
-        if (attr.getParent().getID() == build_relation_id) {
-          accessor_attribute_map[kBuildValueAccessorIndex].second[dest_attr] = attr_id;
-        } else {
-          accessor_attribute_map[kProbeValueAccessorIndex].second[dest_attr] = attr_id;
-        }
+      for (const Scalar *scalar : non_trivial_expressions) {
+        temp_result.addColumn(scalar->getAllValuesForJoin(build_relation_id,
+                                                          build_accessor.get(),
+                                                          probe_relation_id,
+                                                          probe_accessor,
+                                                          zipped_joined_tuple_ids));
       }
-      ++dest_attr;
     }
 
+    // We now create ordered value accessors for both build and probe side,
+    // using the joined tuple IDs.
+    std::unique_ptr<ValueAccessor> ordered_build_accessor(
+        build_accessor->createSharedOrderedTupleIdSequenceAdapterVirtual(build_tids));
+    std::unique_ptr<ValueAccessor> ordered_probe_accessor(
+        probe_accessor->createSharedOrderedTupleIdSequenceAdapterVirtual(probe_tids));
+
+    accessor_attribute_map[kBuildIndex].first = ordered_build_accessor.get();
+    accessor_attribute_map[kProbeIndex].first = ordered_probe_accessor.get();
+    accessor_attribute_map[kTempIndex].first = &temp_result;
+
     // NOTE(chasseur): calling the bulk-insert method of InsertDestination once
     // for each pair of joined blocks incurs some extra overhead that could be
     // avoided by keeping checked-out MutableBlockReferences across iterations

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index acfe3d2..5e9c5d8 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -423,6 +423,10 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   }
 
  private:
+  void executeWithoutCopyElision(ValueAccessor *probe_accesor);
+
+  void executeWithCopyElision(ValueAccessor *probe_accessor);
+
   const CatalogRelationSchema &build_relation_;
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 951a20a..46367b3 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -318,6 +318,11 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index 654bbf9..f183efe 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -305,6 +305,21 @@ class ValueAccessor {
       const TupleIdSequence &id_sequence) = 0;
 
   /**
+   * @brief Create a new OrderedTupleIdSequenceAdapterValueAccessor that wraps
+   *        this ValueAccessor.
+   * @warning The newly-created adapter does NOT take ownership of this
+   *          ValueAccessor nor the provided OrderedTupleIdSequence. Both must
+   *          remain valid so long as the adapter will be used.
+   *
+   * @param id_sequence An OrderedTupleIdSequence specifying some subset of the
+   *        tuples for this ValueAccessor that the adapter will iterate over.
+   * @return A new OrderedTupleIdSequenceAdapterValueAccessor that will iterate
+   *         over only the tuples specified in id_sequence.
+   **/
+  virtual ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) = 0;
+
+  /**
    * @brief Get a TupleIdSequence indicating which positions this ValueAccessor
    *        is iterating over.
    *
@@ -512,6 +527,11 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }
@@ -718,6 +738,11 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }
@@ -944,6 +969,11 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index fbbdc1b..6dc1124 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -290,6 +290,11 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }



[28/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index d851a0c..8f8c0d8 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -21,15 +21,14 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_MAX_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +39,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -86,42 +84,41 @@ class AggregationHandleMax : public AggregationConcreteHandle {
  public:
   ~AggregationHandleMax() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&type_};
+  }
+
+  const Type* getResultType() const override {
+    return &type_;
+  }
+
   AggregationState* createInitialState() const override {
     return new AggregationStateMax(type_);
   }
 
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
-  /**
-   * @brief Iterate with max aggregation state.
-   */
   inline void iterateUnaryInl(AggregationStateMax *state,
                               const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     compareAndUpdate(static_cast<AggregationStateMax *>(state), value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
-    compareAndUpdateFast(max_ptr, value);
+    compareAndUpdate(max_ptr, value);
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  std::size_t getPayloadSize() const override {
+    return sizeof(TypedValue);
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
@@ -136,38 +133,21 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     }
   }
 
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override;
-
-  void mergeStatesFast(const std::uint8_t *source,
-                       std::uint8_t *destination) const override;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
+  }
 
   TypedValue finalize(const AggregationState &state) const override {
     return TypedValue(static_cast<const AggregationStateMax &>(state).max_);
   }
 
-  inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    return TypedValue(static_cast<const AggregationStateMax &>(state).max_);
-  }
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
 
-  inline TypedValue finalizeHashTableEntryFast(
+  inline TypedValue finalizeHashTableEntry(
       const std::uint8_t *byte_ptr) const {
     const TypedValue *max_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
     return TypedValue(*max_ptr);
@@ -175,29 +155,16 @@ class AggregationHandleMax : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for MAX aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for MAX aggregation.
-   */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override { return sizeof(TypedValue); }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionMax;
@@ -227,8 +194,8 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     }
   }
 
-  inline void compareAndUpdateFast(TypedValue *max_ptr,
-                                   const TypedValue &value) const {
+  inline void compareAndUpdate(TypedValue *max_ptr,
+                               const TypedValue &value) const {
     if (value.isNull()) return;
     if (max_ptr->isNull() ||
         fast_comparator_->compareTypedValues(value, *max_ptr)) {
@@ -239,8 +206,6 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMax);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index a07f299..08fb141 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -19,15 +19,16 @@
 
 #include "expressions/aggregation/AggregationHandleMin.hpp"
 
+#include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
@@ -36,54 +37,32 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 AggregationHandleMin::AggregationHandleMin(const Type &type)
-    : type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kMin),
+      type_(type) {
   fast_comparator_.reset(
       ComparisonFactory::GetComparison(ComparisonID::kLess)
           .makeUncheckedComparatorForTypes(type, type.getNonNullableVersion()));
 }
 
-AggregationStateHashTableBase* AggregationHandleMin::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type *> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<AggregationStateMin>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
+AggregationState* AggregationHandleMin::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for MIN: " << argument_ids.size();
 
-AggregationState* AggregationHandleMin::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for MIN: " << column_vectors.size();
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  return new AggregationStateMin(fast_comparator_->accumulateColumnVector(
-      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleMin::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for MIN: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   return new AggregationStateMin(fast_comparator_->accumulateValueAccessor(
       type_.getNullableVersion().makeNullValue(),
-      accessor,
-      accessor_ids.front()));
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "Got wrong number of arguments for MIN: " << argument_ids.size();
+      accessor_mux.getValueAccessorBySource(argument_source),
+      argument_id));
 }
 
 void AggregationHandleMin::mergeStates(const AggregationState &source,
@@ -98,41 +77,37 @@ void AggregationHandleMin::mergeStates(const AggregationState &source,
   }
 }
 
-void AggregationHandleMin::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleMin::mergeStates(const std::uint8_t *source,
+                                       std::uint8_t *destination) const {
   const TypedValue *src_min_ptr = reinterpret_cast<const TypedValue *>(source);
   TypedValue *dst_min_ptr = reinterpret_cast<TypedValue *>(destination);
 
   if (!(src_min_ptr->isNull())) {
-    compareAndUpdateFast(dst_min_ptr, *src_min_ptr);
+    compareAndUpdate(dst_min_ptr, *src_min_ptr);
   }
 }
 
 ColumnVector* AggregationHandleMin::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleMin,
-                                     AggregationStateFastHashTable>(
-      type_.getNonNullableVersion(), hash_table, group_by_keys, index);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleMin>(
+      type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleMin,
-      AggregationStateMin>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleMin, AggregationStateMin>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleMin,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleMin>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index e3472ec..0e62be5 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -21,15 +21,14 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_MIN_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +39,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -88,42 +86,45 @@ class AggregationHandleMin : public AggregationConcreteHandle {
  public:
   ~AggregationHandleMin() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&type_};
+  }
+
+  const Type* getResultType() const override {
+    return &type_;
+  }
+
   AggregationState* createInitialState() const override {
     return new AggregationStateMin(type_);
   }
 
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
-  /**
-   * @brief Iterate with min aggregation state.
-   */
   inline void iterateUnaryInl(AggregationStateMin *state,
                               const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     compareAndUpdate(state, value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
-    compareAndUpdateFast(min_ptr, value);
+    compareAndUpdate(min_ptr, value);
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  TypedValue finalize(const AggregationState &state) const override {
+    return static_cast<const AggregationStateMin &>(state).min_;
+  }
+
+  std::size_t getPayloadSize() const override {
+    return sizeof(TypedValue);
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
@@ -138,68 +139,33 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     }
   }
 
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override;
-
-  void mergeStatesFast(const std::uint8_t *source,
-                       std::uint8_t *destination) const override;
-
-  TypedValue finalize(const AggregationState &state) const override {
-    return static_cast<const AggregationStateMin &>(state).min_;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
   }
 
-  inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    return static_cast<const AggregationStateMin &>(state).min_;
-  }
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
 
-  inline TypedValue finalizeHashTableEntryFast(
+  inline TypedValue finalizeHashTableEntry(
       const std::uint8_t *byte_ptr) const {
-    const TypedValue *min_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
-    return TypedValue(*min_ptr);
+    return *reinterpret_cast<const TypedValue *>(byte_ptr);
   }
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for MIN aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for MIN aggregation.
-   */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override { return sizeof(TypedValue); }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionMin;
@@ -228,8 +194,8 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     }
   }
 
-  inline void compareAndUpdateFast(TypedValue *min_ptr,
-                                   const TypedValue &value) const {
+  inline void compareAndUpdate(TypedValue *min_ptr,
+                               const TypedValue &value) const {
     if (value.isNull()) return;
     if (min_ptr->isNull() ||
         fast_comparator_->compareTypedValues(value, *min_ptr)) {
@@ -240,8 +206,6 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMin);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 642d88d..9f5f220 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -20,13 +20,13 @@
 #include "expressions/aggregation/AggregationHandleSum.hpp"
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
-#include <utility>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -40,10 +40,11 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 AggregationHandleSum::AggregationHandleSum(const Type &type)
-    : argument_type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kSum),
+      argument_type_(type) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -79,47 +80,27 @@ AggregationHandleSum::AggregationHandleSum(const Type &type)
   result_type_ = &sum_type.getNullableVersion();
 }
 
-AggregationStateHashTableBase* AggregationHandleSum::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type *> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<AggregationStateSum>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
+AggregationState* AggregationHandleSum::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for SUM: " << argument_ids.size();
 
-AggregationState* AggregationHandleSum::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for SUM: " << column_vectors.size();
-  std::size_t num_tuples = 0;
-  TypedValue cv_sum = fast_operator_->accumulateColumnVector(
-      blank_state_.sum_, *column_vectors.front(), &num_tuples);
-  return new AggregationStateSum(std::move(cv_sum), num_tuples == 0);
-}
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleSum::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for SUM: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   std::size_t num_tuples = 0;
-  TypedValue va_sum = fast_operator_->accumulateValueAccessor(
-      blank_state_.sum_, accessor, accessor_ids.front(), &num_tuples);
+  TypedValue va_sum =
+      fast_operator_->accumulateValueAccessor(
+          blank_state_.sum_,
+          accessor_mux.getValueAccessorBySource(argument_source),
+          argument_id,
+          &num_tuples);
   return new AggregationStateSum(std::move(va_sum), num_tuples == 0);
 }
-#endif
-
-void AggregationHandleSum::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "Got wrong number of arguments for SUM: " << argument_ids.size();
-}
 
 void AggregationHandleSum::mergeStates(const AggregationState &source,
                                        AggregationState *destination) const {
@@ -134,8 +115,8 @@ void AggregationHandleSum::mergeStates(const AggregationState &source,
   sum_destination->null_ = sum_destination->null_ && sum_source.null_;
 }
 
-void AggregationHandleSum::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleSum::mergeStates(const std::uint8_t *source,
+                                       std::uint8_t *destination) const {
   const TypedValue *src_sum_ptr =
       reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset_);
   const bool *src_null_ptr =
@@ -162,29 +143,25 @@ TypedValue AggregationHandleSum::finalize(const AggregationState &state) const {
 
 ColumnVector* AggregationHandleSum::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleSum,
-                                     AggregationStateFastHashTable>(
-      *result_type_, hash_table, group_by_keys, index);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleSum>(
+      *result_type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleSum,
-      AggregationStateSum>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleSum, AggregationStateSum>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleSum,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleSum>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index f0d23e1..ba4fa9b 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -21,15 +21,14 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_SUM_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +39,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -101,16 +99,18 @@ class AggregationHandleSum : public AggregationConcreteHandle {
  public:
   ~AggregationHandleSum() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&argument_type_};
+  }
+
+  const Type* getResultType() const override {
+    return result_type_;
+  }
+
   AggregationState* createInitialState() const override {
     return new AggregationStateSum(blank_state_);
   }
 
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
   inline void iterateUnaryInl(AggregationStateSum *state,
                               const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
@@ -121,8 +121,8 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     state->null_ = false;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
     TypedValue *sum_ptr =
@@ -133,16 +133,18 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     *null_ptr = false;
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  TypedValue finalize(const AggregationState &state) const override;
+
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr =
@@ -161,70 +163,32 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     }
   }
 
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_id) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override;
-
-  void mergeStatesFast(const std::uint8_t *source,
-                       std::uint8_t *destination) const override;
-
-  TypedValue finalize(const AggregationState &state) const override;
-
-  inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    return static_cast<const AggregationStateSum &>(state).sum_;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
   }
 
-  inline TypedValue finalizeHashTableEntryFast(
-      const std::uint8_t *byte_ptr) const {
-    std::uint8_t *value_ptr = const_cast<std::uint8_t *>(byte_ptr);
-    TypedValue *sum_ptr =
-        reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset_);
-    return *sum_ptr;
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
+
+  inline TypedValue finalizeHashTableEntry(const std::uint8_t *byte_ptr) const {
+    return *reinterpret_cast<const TypedValue *>(byte_ptr + blank_state_.sum_offset_);
   }
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for SUM aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for SUM aggregation.
-   */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override {
-    return blank_state_.getPayloadSize();
-  }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionSum;
@@ -242,8 +206,6 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> fast_operator_;
   std::unique_ptr<UncheckedBinaryOperator> merge_operator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleSum);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index e9503f7..4220a8d 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -55,9 +55,6 @@ add_library(quickstep_expressions_aggregation_AggregationHandleAvg
 add_library(quickstep_expressions_aggregation_AggregationHandleCount
             AggregationHandleCount.cpp
             AggregationHandleCount.hpp)
-add_library(quickstep_expressions_aggregation_AggregationHandleDistinct
-            AggregationHandleDistinct.cpp
-            AggregationHandleDistinct.hpp)
 add_library(quickstep_expressions_aggregation_AggregationHandleMax
             AggregationHandleMax.cpp
             AggregationHandleMax.hpp)
@@ -144,20 +141,21 @@ target_link_libraries(quickstep_expressions_aggregation_AggregateFunctionSum
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandle
                       glog
-                      quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadHashTable
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandle
                       glog
-                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
@@ -165,10 +163,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
@@ -183,39 +179,25 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_storage_ValueAccessorUtil
+                      quickstep_types_LongType
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_containers_ColumnVectorUtil
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_expressions_aggregation_AggregationHandleDistinct
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
@@ -225,14 +207,11 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
@@ -242,10 +221,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
@@ -271,7 +248,6 @@ target_link_libraries(quickstep_expressions_aggregation
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_expressions_aggregation_AggregationHandleCount
-                      quickstep_expressions_aggregation_AggregationHandleDistinct
                       quickstep_expressions_aggregation_AggregationHandleMax
                       quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_expressions_aggregation_AggregationHandleSum
@@ -301,7 +277,9 @@ target_link_libraries(AggregationHandle_tests
                       quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_AggregationOperationState
                       quickstep_storage_HashTableBase
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_types_CharType
                       quickstep_types_DateOperatorOverloads
                       quickstep_types_DatetimeIntervalType

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
index 79d4448..0ad50d5 100644
--- a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
@@ -29,8 +29,9 @@
 #include "expressions/aggregation/AggregationHandleAvg.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DateOperatorOverloads.hpp"
 #include "types/DatetimeIntervalType.hpp"
@@ -46,10 +47,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "gtest/gtest.h"
 
@@ -192,39 +190,6 @@ class AggregationHandleAvgTest : public ::testing::Test {
   }
 
   template <typename GenericType, typename OutputType = DoubleType>
-  void checkAggregationAvgGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_)
-            .isNull());
-
-    typename GenericType::cpptype sum;
-    SetDataType(0, &sum);
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType>(type, &sum));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_avg_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckAvgValue<typename OutputType::cpptype>(
-        static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
-        *aggregation_handle_avg_,
-        *cv_state);
-
-    aggregation_handle_avg_->mergeStates(*cv_state,
-                                         aggregation_handle_avg_state_.get());
-    CheckAvgValue<typename OutputType::cpptype>(
-        static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
-        *aggregation_handle_avg_,
-        *aggregation_handle_avg_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType, typename OutputType = DoubleType>
   void checkAggregationAvgGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
@@ -240,7 +205,8 @@ class AggregationHandleAvgTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_avg_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -256,7 +222,6 @@ class AggregationHandleAvgTest : public ::testing::Test {
         *aggregation_handle_avg_,
         *aggregation_handle_avg_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_avg_;
   std::unique_ptr<AggregationState> aggregation_handle_avg_state_;
@@ -311,33 +276,6 @@ TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeTest) {
   checkAggregationAvgGeneric<YearMonthIntervalType, YearMonthIntervalType>();
 }
 
-TEST_F(AggregationHandleAvgTest, IntTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<IntType>();
-}
-
-TEST_F(AggregationHandleAvgTest, LongTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<LongType>();
-}
-
-TEST_F(AggregationHandleAvgTest, FloatTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<FloatType>();
-}
-
-TEST_F(AggregationHandleAvgTest, DoubleTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<DoubleType>();
-}
-
-TEST_F(AggregationHandleAvgTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<DatetimeIntervalType,
-                                         DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<YearMonthIntervalType,
-                                         YearMonthIntervalType>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleAvgTest, IntTypeValueAccessorTest) {
   checkAggregationAvgGenericValueAccessor<IntType>();
 }
@@ -363,7 +301,6 @@ TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeValueAccessorTest) {
   checkAggregationAvgGenericValueAccessor<YearMonthIntervalType,
                                           YearMonthIntervalType>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleAvgDeathTest, CharTypeTest) {
@@ -468,28 +405,25 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
   initializeHandle(long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_avg_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_avg_.get()->getPayloadSize()},
           {aggregation_handle_avg_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_avg_.get()->getPayloadSize()},
           {aggregation_handle_avg_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleAvg *aggregation_handle_avg_derived =
       static_cast<AggregationHandleAvg *>(aggregation_handle_avg_.get());
@@ -546,29 +480,29 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
                                                       buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
@@ -576,21 +510,19 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
       (common_key_destination_avg_val.getLiteral<std::int64_t>() +
        common_key_source_avg_val.getLiteral<std::int64_t>()) /
           static_cast<double>(2),
-      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+      aggregation_handle_avg_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
   CheckAvgValue<double>(
       exclusive_key_destination_avg_val.getLiteral<std::int64_t>(),
-      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+      aggregation_handle_avg_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(
-              exclusive_destination_key) +
-          1));
+              exclusive_destination_key) + 1));
   CheckAvgValue<double>(
       exclusive_key_source_avg_val.getLiteral<std::int64_t>(),
-      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+      aggregation_handle_avg_derived->finalizeHashTableEntry(
           source_hash_table_derived->getSingleCompositeKey(
-              exclusive_source_key) +
-          1));
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
index 78bd249..86a014b 100644
--- a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
@@ -30,8 +30,9 @@
 #include "expressions/aggregation/AggregationHandleCount.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DoubleType.hpp"
 #include "types/FloatType.hpp"
@@ -43,10 +44,7 @@
 #include "types/TypedValue.hpp"
 #include "types/VarCharType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "gtest/gtest.h"
 
@@ -216,32 +214,6 @@ class AggregationHandleCountTest : public ::testing::Test {
   }
 
   template <typename NumericType>
-  void checkAggregationCountNumericColumnVector(int test_count) {
-    const NumericType &type = NumericType::Instance(true);
-    initializeHandle(&type);
-    CheckCountValue(
-        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
-
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorNumeric<NumericType>(type, test_count));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_count_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckCountValue(test_count, *aggregation_handle_count_, *cv_state);
-
-    aggregation_handle_count_->mergeStates(
-        *cv_state, aggregation_handle_count_state_.get());
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *aggregation_handle_count_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename NumericType>
   void checkAggregationCountNumericValueAccessor(int test_count) {
     const NumericType &type = NumericType::Instance(true);
     initializeHandle(&type);
@@ -255,7 +227,8 @@ class AggregationHandleCountTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_count_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -267,7 +240,6 @@ class AggregationHandleCountTest : public ::testing::Test {
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   template <typename StringType>
   void checkAggregationCountString(int test_count) {
@@ -326,32 +298,6 @@ class AggregationHandleCountTest : public ::testing::Test {
   }
 
   template <typename StringType, typename ColumnVectorType>
-  void checkAggregationCountStringColumnVector(int test_count) {
-    const StringType &type = StringType::Instance(10, true);
-    initializeHandle(&type);
-    CheckCountValue(
-        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
-
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorString<ColumnVectorType>(type, test_count));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_count_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckCountValue(test_count, *aggregation_handle_count_, *cv_state);
-
-    aggregation_handle_count_->mergeStates(
-        *cv_state, aggregation_handle_count_state_.get());
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *aggregation_handle_count_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename StringType, typename ColumnVectorType>
   void checkAggregationCountStringValueAccessor(int test_count) {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(&type);
@@ -365,7 +311,8 @@ class AggregationHandleCountTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_count_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -377,7 +324,6 @@ class AggregationHandleCountTest : public ::testing::Test {
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_count_;
   std::unique_ptr<AggregationState> aggregation_handle_count_state_;
@@ -425,38 +371,6 @@ TEST_F(AggregationHandleCountTest, VarCharTypeTest) {
   checkAggregationCountString<VarCharType>(10000);
 }
 
-TEST_F(AggregationHandleCountTest, IntTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<IntType>(0);
-  checkAggregationCountNumericColumnVector<IntType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, LongTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<LongType>(0);
-  checkAggregationCountNumericColumnVector<LongType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, FloatTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<FloatType>(0);
-  checkAggregationCountNumericColumnVector<FloatType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, DoubleTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<DoubleType>(0);
-  checkAggregationCountNumericColumnVector<DoubleType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, CharTypeColumnVectorTest) {
-  checkAggregationCountStringColumnVector<CharType, NativeColumnVector>(0);
-  checkAggregationCountStringColumnVector<CharType, NativeColumnVector>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, VarCharTypeColumnVectorTest) {
-  checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(0);
-  checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(
-      10000);
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleCountTest, IntTypeValueAccessorTest) {
   checkAggregationCountNumericValueAccessor<IntType>(0);
   checkAggregationCountNumericValueAccessor<IntType>(10000);
@@ -488,7 +402,6 @@ TEST_F(AggregationHandleCountTest, VarCharTypeValueAccessorTest) {
   checkAggregationCountStringValueAccessor<VarCharType, IndirectColumnVector>(
       10000);
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 TEST_F(AggregationHandleCountTest, canApplyToTypeTest) {
   EXPECT_TRUE(ApplyToTypesTest(kInt));
@@ -511,28 +424,25 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
   initializeHandle(&long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_count_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_count_.get()->getPayloadSize()},
           {aggregation_handle_count_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_count_.get()->getPayloadSize()},
           {aggregation_handle_count_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   // TODO(harshad) - Use TemplateUtil::CreateBoolInstantiatedInstance to
   // generate all the combinations of the bool template arguments and test them.
@@ -612,49 +522,48 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckCountValue(
       common_key_destination_count_val.getLiteral<std::int64_t>() +
           common_key_source_count_val.getLiteral<std::int64_t>(),
-      aggregation_handle_count_derived->finalizeHashTableEntryFast(
+      aggregation_handle_count_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
   CheckCountValue(
       exclusive_key_destination_count_val.getLiteral<std::int64_t>(),
-      aggregation_handle_count_derived->finalizeHashTableEntryFast(
+      aggregation_handle_count_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(
-              exclusive_destination_key) +
-          1));
-  CheckCountValue(exclusive_key_source_count_val.getLiteral<std::int64_t>(),
-                  aggregation_handle_count_derived->finalizeHashTableEntryFast(
-                      source_hash_table_derived->getSingleCompositeKey(
-                          exclusive_source_key) +
-                      1));
+              exclusive_destination_key) + 1));
+  CheckCountValue(
+      exclusive_key_source_count_val.getLiteral<std::int64_t>(),
+      aggregation_handle_count_derived->finalizeHashTableEntry(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
index 026bd1d..d5e8d18 100644
--- a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
@@ -32,9 +32,10 @@
 #include "expressions/aggregation/AggregationHandleMax.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
 #include "types/DatetimeLit.hpp"
@@ -51,10 +52,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
@@ -223,34 +221,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
   }
 
   template <typename GenericType>
-  void checkAggregationMaxGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
-            .isNull());
-
-    typename GenericType::cpptype max;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType>(type, &max));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_max_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMaxValue<typename GenericType::cpptype>(
-        max, *aggregation_handle_max_, *cv_state);
-
-    aggregation_handle_max_->mergeStates(*cv_state,
-                                         aggregation_handle_max_state_.get());
-    CheckMaxValue<typename GenericType::cpptype>(
-        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType>
   void checkAggregationMaxGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
@@ -266,7 +236,8 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_max_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -278,7 +249,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
     CheckMaxValue<typename GenericType::cpptype>(
         max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   template <typename StringType>
   void checkAggregationMaxString() {
@@ -385,33 +355,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
   }
 
   template <typename StringType, typename ColumnVectorType>
-  void checkAggregationMaxStringColumnVector() {
-    const StringType &type = StringType::Instance(10, true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
-            .isNull());
-
-    std::string max;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorString<ColumnVectorType>(type, &max));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_max_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMaxString(max, *aggregation_handle_max_, *cv_state);
-
-    aggregation_handle_max_->mergeStates(*cv_state,
-                                         aggregation_handle_max_state_.get());
-    CheckMaxString(
-        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename StringType, typename ColumnVectorType>
   void checkAggregationMaxStringValueAccessor() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
@@ -426,7 +369,8 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_max_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -437,7 +381,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
     CheckMaxString(
         max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_max_;
   std::unique_ptr<AggregationState> aggregation_handle_max_state_;
@@ -514,43 +457,6 @@ TEST_F(AggregationHandleMaxTest, VarCharTypeTest) {
   checkAggregationMaxString<VarCharType>();
 }
 
-TEST_F(AggregationHandleMaxTest, IntTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<IntType>();
-}
-
-TEST_F(AggregationHandleMaxTest, LongTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<LongType>();
-}
-
-TEST_F(AggregationHandleMaxTest, FloatTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<FloatType>();
-}
-
-TEST_F(AggregationHandleMaxTest, DoubleTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<DoubleType>();
-}
-
-TEST_F(AggregationHandleMaxTest, DatetimeTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<DatetimeType>();
-}
-
-TEST_F(AggregationHandleMaxTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleMaxTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<YearMonthIntervalType>();
-}
-
-TEST_F(AggregationHandleMaxTest, CharTypeColumnVectorTest) {
-  checkAggregationMaxStringColumnVector<CharType, NativeColumnVector>();
-}
-
-TEST_F(AggregationHandleMaxTest, VarCharColumnVectorTypeTest) {
-  checkAggregationMaxStringColumnVector<VarCharType, IndirectColumnVector>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleMaxTest, IntTypeValueAccessorTest) {
   checkAggregationMaxGenericValueAccessor<IntType>();
 }
@@ -586,7 +492,6 @@ TEST_F(AggregationHandleMaxTest, CharTypeValueAccessorTest) {
 TEST_F(AggregationHandleMaxTest, VarCharValueAccessorTypeTest) {
   checkAggregationMaxStringValueAccessor<VarCharType, IndirectColumnVector>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleMaxDeathTest, WrongTypeTest) {
@@ -689,28 +594,25 @@ TEST_F(AggregationHandleMaxTest, GroupByTableMergeTest) {
   initializeHandle(int_non_null_type);
   storage_manager_.reset(new StorageManager("./test_max_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_max_.get()->getPayloadSize()},
           {aggregation_handle_max_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_max_.get()->getPayloadSize()},
           {aggregation_handle_max_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleMax *aggregation_handle_max_derived =
       static_cast<AggregationHandleMax *>(aggregation_handle_max_.get());
@@ -780,47 +682,47 @@ TEST_F(AggregationHandleMaxTest, GroupByTableMergeTest) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckMaxValue<int>(
       common_key_destination_max_val.getLiteral<int>(),
-      aggregation_handle_max_derived->finalizeHashTableEntryFast(
+      aggregation_handle_max_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
-  CheckMaxValue<int>(exclusive_key_destination_max_val.getLiteral<int>(),
-                     aggregation_handle_max_derived->finalizeHashTableEntryFast(
-                         destination_hash_table_derived->getSingleCompositeKey(
-                             exclusive_destination_key) +
-                         1));
-  CheckMaxValue<int>(exclusive_key_source_max_val.getLiteral<int>(),
-                     aggregation_handle_max_derived->finalizeHashTableEntryFast(
-                         source_hash_table_derived->getSingleCompositeKey(
-                             exclusive_source_key) +
-                         1));
+  CheckMaxValue<int>(
+      exclusive_key_destination_max_val.getLiteral<int>(),
+      aggregation_handle_max_derived->finalizeHashTableEntry(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) + 1));
+  CheckMaxValue<int>(
+      exclusive_key_source_max_val.getLiteral<int>(),
+      aggregation_handle_max_derived->finalizeHashTableEntry(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep


[29/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
- Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key.
- Supports copy elision for aggregation.


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

Branch: refs/heads/LIP-time-decomposition
Commit: 2d89e4fbf3b51d7768d928d56b25ec0ab52faabb
Parents: ec76096
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 30 14:46:39 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Tue Feb 7 15:29:19 2017 -0600

----------------------------------------------------------------------
 .../aggregation/AggregateFunctionCount.cpp      |    6 +-
 .../aggregation/AggregationConcreteHandle.cpp   |   32 +-
 .../aggregation/AggregationConcreteHandle.hpp   |  160 +-
 expressions/aggregation/AggregationHandle.hpp   |  193 +-
 .../aggregation/AggregationHandleAvg.cpp        |   95 +-
 .../aggregation/AggregationHandleAvg.hpp        |  115 +-
 .../aggregation/AggregationHandleCount.cpp      |  147 +-
 .../aggregation/AggregationHandleCount.hpp      |  154 +-
 .../aggregation/AggregationHandleDistinct.cpp   |   81 -
 .../aggregation/AggregationHandleDistinct.hpp   |  130 -
 .../aggregation/AggregationHandleMax.cpp        |   91 +-
 .../aggregation/AggregationHandleMax.hpp        |  111 +-
 .../aggregation/AggregationHandleMin.cpp        |   91 +-
 .../aggregation/AggregationHandleMin.hpp        |  120 +-
 .../aggregation/AggregationHandleSum.cpp        |   93 +-
 .../aggregation/AggregationHandleSum.hpp        |  114 +-
 expressions/aggregation/CMakeLists.txt          |   58 +-
 .../tests/AggregationHandleAvg_unittest.cpp     |  110 +-
 .../tests/AggregationHandleCount_unittest.cpp   |  145 +-
 .../tests/AggregationHandleMax_unittest.cpp     |  158 +-
 .../tests/AggregationHandleMin_unittest.cpp     |  158 +-
 .../tests/AggregationHandleSum_unittest.cpp     |  109 +-
 query_execution/QueryContext.hpp                |   14 -
 query_optimizer/CMakeLists.txt                  |    4 +
 query_optimizer/ExecutionGenerator.cpp          |  147 +-
 query_optimizer/ExecutionGenerator.hpp          |   20 +-
 relational_operators/CMakeLists.txt             |   15 +
 .../DestroyAggregationStateOperator.cpp         |    7 -
 .../FinalizeAggregationOperator.cpp             |   16 +-
 .../FinalizeAggregationOperator.hpp             |   14 +-
 .../InitializeAggregationOperator.cpp           |   72 +
 .../InitializeAggregationOperator.hpp           |  122 +
 relational_operators/WorkOrderFactory.cpp       |    3 +
 storage/AggregationOperationState.cpp           |  834 +++---
 storage/AggregationOperationState.hpp           |  178 +-
 storage/CMakeLists.txt                          |  131 +-
 storage/CollisionFreeVectorTable.cpp            |  285 +++
 storage/CollisionFreeVectorTable.hpp            |  730 ++++++
 storage/FastHashTable.hpp                       | 2403 ------------------
 storage/FastHashTableFactory.hpp                |  224 --
 storage/FastSeparateChainingHashTable.hpp       | 1551 -----------
 storage/HashTable.proto                         |    7 +-
 storage/HashTableBase.hpp                       |   42 +-
 storage/HashTableFactory.hpp                    |   63 +-
 storage/HashTablePool.hpp                       |   79 +-
 storage/PackedPayloadHashTable.cpp              |  463 ++++
 storage/PackedPayloadHashTable.hpp              |  995 ++++++++
 storage/PartitionedHashTablePool.hpp            |   56 +-
 storage/StorageBlock.cpp                        |  274 +-
 storage/StorageBlock.hpp                        |  167 --
 storage/ValueAccessorMultiplexer.hpp            |  145 ++
 .../BarrieredReadWriteConcurrentBitVector.hpp   |  282 ++
 utility/CMakeLists.txt                          |    7 +
 53 files changed, 4722 insertions(+), 7099 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregateFunctionCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionCount.cpp b/expressions/aggregation/AggregateFunctionCount.cpp
index 466ff2f..9795b4a 100644
--- a/expressions/aggregation/AggregateFunctionCount.cpp
+++ b/expressions/aggregation/AggregateFunctionCount.cpp
@@ -53,16 +53,16 @@ AggregationHandle* AggregateFunctionCount::createHandle(
 
   if (argument_types.empty()) {
     // COUNT(*)
-    return new AggregationHandleCount<true, false>();
+    return new AggregationHandleCount<true, false>(nullptr);
   } else if (argument_types.front()->isNullable()) {
     // COUNT(some_nullable_argument)
-    return new AggregationHandleCount<false, true>();
+    return new AggregationHandleCount<false, true>(argument_types.front());
   } else {
     // COUNT(non_nullable_argument)
     //
     // TODO(chasseur): Modify query optimizer to optimize-away COUNT with a
     // non-nullable argument and convert it to COUNT(*).
-    return new AggregationHandleCount<false, false>();
+    return new AggregationHandleCount<false, false>(argument_types.front());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index e3fb520..fa21056 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -22,16 +22,14 @@
 #include <cstddef>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTable.hpp"
 #include "storage/HashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 
 namespace quickstep {
 
 class StorageManager;
 class Type;
-class ValueAccessor;
 
 AggregationStateHashTableBase* AggregationConcreteHandle::createDistinctifyHashTable(
     const HashTableImplType hash_table_impl,
@@ -39,30 +37,26 @@ AggregationStateHashTableBase* AggregationConcreteHandle::createDistinctifyHashT
     const std::size_t estimated_num_distinct_keys,
     StorageManager *storage_manager) const {
   // Create a hash table with key types as key_types and value type as bool.
-  return AggregationStateHashTableFactory<bool>::CreateResizable(
+  return AggregationStateHashTableFactory::CreateResizable(
       hash_table_impl,
       key_types,
       estimated_num_distinct_keys,
+      {},
       storage_manager);
 }
 
 void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &key_ids,
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_ids,
+    const ValueAccessorMultiplexer &accessor_mux,
     AggregationStateHashTableBase *distinctify_hash_table) const {
-  // If the key-value pair is already there, we don't need to update the value,
-  // which should always be "true". I.e. the value is just a placeholder.
-
-  AggregationStateFastHashTable *hash_table =
-      static_cast<AggregationStateFastHashTable *>(distinctify_hash_table);
-  if (key_ids.size() == 1) {
-    hash_table->upsertValueAccessorFast(
-        key_ids, accessor, key_ids[0], true /* check_for_null_keys */);
-  } else {
-    std::vector<attribute_id> empty_args {kInvalidAttributeID};
-    hash_table->upsertValueAccessorCompositeKeyFast(
-        empty_args, accessor, key_ids, true /* check_for_null_keys */);
+  std::vector<MultiSourceAttributeId> concatenated_ids(key_ids);
+  for (const MultiSourceAttributeId &arg_id : argument_ids) {
+    concatenated_ids.emplace_back(arg_id);
   }
+
+  static_cast<PackedPayloadHashTable *>(distinctify_hash_table)
+      ->upsertValueAccessorCompositeKey({}, concatenated_ids, accessor_mux);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index 398a032..2287ca1 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -21,14 +21,15 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTable.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
@@ -40,7 +41,6 @@ namespace quickstep {
 
 class StorageManager;
 class Type;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -51,7 +51,7 @@ class ValueAccessor;
  *        merging two group by hash tables.
  **/
 template <typename HandleT>
-class HashTableStateUpserterFast {
+class HashTableStateUpserter {
  public:
   /**
    * @brief Constructor.
@@ -61,8 +61,8 @@ class HashTableStateUpserterFast {
    *        table. The corresponding state (for the same key) in the destination
    *        hash table will be upserted.
    **/
-  HashTableStateUpserterFast(const HandleT &handle,
-                             const std::uint8_t *source_state)
+  HashTableStateUpserter(const HandleT &handle,
+                         const std::uint8_t *source_state)
       : handle_(handle), source_state_(source_state) {}
 
   /**
@@ -72,14 +72,14 @@ class HashTableStateUpserterFast {
    *        table that is being upserted.
    **/
   void operator()(std::uint8_t *destination_state) {
-    handle_.mergeStatesFast(source_state_, destination_state);
+    handle_.mergeStates(source_state_, destination_state);
   }
 
  private:
   const HandleT &handle_;
   const std::uint8_t *source_state_;
 
-  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserterFast);
+  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserter);
 };
 
 /**
@@ -93,74 +93,62 @@ class HashTableStateUpserterFast {
  **/
 class AggregationConcreteHandle : public AggregationHandle {
  public:
-  /**
-   * @brief Default implementaion for AggregationHandle::accumulateNullary().
-   */
-  AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const override {
-    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
-               << "takes at least one argument.";
-  }
-
-  /**
-   * @brief Implementaion for AggregationHandle::createDistinctifyHashTable()
-   *        that creates a new HashTable for the distinctify step for
-   *        DISTINCT aggregation.
-   */
   AggregationStateHashTableBase* createDistinctifyHashTable(
       const HashTableImplType hash_table_impl,
       const std::vector<const Type *> &key_types,
       const std::size_t estimated_num_distinct_keys,
       StorageManager *storage_manager) const override;
 
-  /**
-   * @brief Implementaion for
-   * AggregationHandle::insertValueAccessorIntoDistinctifyHashTable()
-   * that inserts the GROUP BY expressions and aggregation arguments together
-   * as keys into the distinctify hash table.
-   */
   void insertValueAccessorIntoDistinctifyHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_ids,
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux,
       AggregationStateHashTableBase *distinctify_hash_table) const override;
 
+  void blockUpdate() override {
+    block_update_ = true;
+  }
+
+  void allowUpdate() override {
+    block_update_ = false;
+  }
+
  protected:
-  AggregationConcreteHandle() {}
+  explicit AggregationConcreteHandle(const AggregationID agg_id)
+      : AggregationHandle(agg_id),
+        block_update_(false) {}
 
   template <typename HandleT, typename StateT>
-  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelper(
       const AggregationStateHashTableBase &distinctify_hash_table) const;
 
-  template <typename HandleT, typename HashTableT>
-  void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+  template <typename HandleT>
+  void aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *hash_table,
-      std::size_t index) const;
+      const std::size_t index,
+      AggregationStateHashTableBase *hash_table) const;
 
-  template <typename HandleT, typename HashTableT>
-  ColumnVector* finalizeHashTableHelperFast(
+  template <typename HandleT>
+  ColumnVector* finalizeHashTableHelper(
       const Type &result_type,
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const;
 
-  template <typename HandleT, typename HashTableT>
-  inline TypedValue finalizeGroupInHashTableFast(
+  template <typename HandleT>
+  inline TypedValue finalizeGroupInHashTable(
       const AggregationStateHashTableBase &hash_table,
-      const std::vector<TypedValue> &group_key,
-      int index) const {
+      const std::size_t index,
+      const std::vector<TypedValue> &group_key) const {
     const std::uint8_t *group_state =
-        static_cast<const HashTableT &>(hash_table).getSingleCompositeKey(group_key, index);
+        static_cast<const PackedPayloadHashTable &>(hash_table)
+            .getSingleCompositeKey(group_key, index);
     DCHECK(group_state != nullptr)
         << "Could not find entry for specified group_key in HashTable";
-    return static_cast<const HandleT *>(this)->finalizeHashTableEntryFast(
-        group_state);
+    return static_cast<const HandleT *>(this)->finalizeHashTableEntry(group_state);
   }
 
-  template <typename HandleT, typename HashTableT>
-  void mergeGroupByHashTablesHelperFast(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const;
+  bool block_update_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationConcreteHandle);
@@ -185,17 +173,10 @@ class HashTableAggregateFinalizer {
         output_column_vector_(output_column_vector) {}
 
   inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const AggregationState &group_state) {
-    group_by_keys_->emplace_back(group_by_key);
-    output_column_vector_->appendTypedValue(
-        handle_.finalizeHashTableEntry(group_state));
-  }
-
-  inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const unsigned char *byte_ptr) {
+                         const std::uint8_t *byte_ptr) {
     group_by_keys_->emplace_back(group_by_key);
     output_column_vector_->appendTypedValue(
-        handle_.finalizeHashTableEntryFast(byte_ptr));
+        handle_.finalizeHashTableEntry(byte_ptr));
   }
 
  private:
@@ -211,7 +192,7 @@ class HashTableAggregateFinalizer {
 
 template <typename HandleT, typename StateT>
 StateT* AggregationConcreteHandle::
-    aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+    aggregateOnDistinctifyHashTableForSingleUnaryHelper(
         const AggregationStateHashTableBase &distinctify_hash_table) const {
   const HandleT &handle = static_cast<const HandleT &>(*this);
   StateT *state = static_cast<StateT *>(createInitialState());
@@ -219,15 +200,14 @@ StateT* AggregationConcreteHandle::
   // A lambda function which will be called on each key from the distinctify
   // hash table.
   const auto aggregate_functor = [&handle, &state](
-      const TypedValue &key, const std::uint8_t &dumb_placeholder) {
+      const TypedValue &key, const std::uint8_t *dumb_placeholder) {
     // For each (unary) key in the distinctify hash table, aggregate the key
     // into "state".
     handle.iterateUnaryInl(state, key);
   };
 
-  const AggregationStateFastHashTable &hash_table =
-      static_cast<const AggregationStateFastHashTable &>(
-          distinctify_hash_table);
+  const auto &hash_table =
+      static_cast<const PackedPayloadHashTable &>(distinctify_hash_table);
   // Invoke the lambda function "aggregate_functor" on each key from the
   // distinctify hash table.
   hash_table.forEach(&aggregate_functor);
@@ -235,20 +215,20 @@ StateT* AggregationConcreteHandle::
   return state;
 }
 
-template <typename HandleT, typename HashTableT>
+template <typename HandleT>
 void AggregationConcreteHandle::
-    aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+    aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
         const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table,
-        std::size_t index) const {
+        const std::size_t index,
+        AggregationStateHashTableBase *aggregation_hash_table) const {
   const HandleT &handle = static_cast<const HandleT &>(*this);
-  HashTableT *target_hash_table =
-      static_cast<HashTableT *>(aggregation_hash_table);
+  PackedPayloadHashTable *target_hash_table =
+      static_cast<PackedPayloadHashTable *>(aggregation_hash_table);
 
   // A lambda function which will be called on each key-value pair from the
   // distinctify hash table.
   const auto aggregate_functor = [&handle, &target_hash_table, &index](
-      std::vector<TypedValue> &key, const bool &dumb_placeholder) {
+      std::vector<TypedValue> &key, const std::uint8_t *dumb_placeholder) {
     // For each (composite) key vector in the distinctify hash table with size N.
     // The first N-1 entries are GROUP BY columns and the last entry is the
     // argument to be aggregated on.
@@ -258,28 +238,28 @@ void AggregationConcreteHandle::
     // An upserter as lambda function for aggregating the argument into its
     // GROUP BY group's entry inside aggregation_hash_table.
     const auto upserter = [&handle, &argument](std::uint8_t *state) {
-      handle.iterateUnaryInlFast(argument, state);
+      handle.iterateUnaryInl(argument, state);
     };
 
-    target_hash_table->upsertCompositeKeyFast(key, nullptr, &upserter, index);
+    target_hash_table->upsertCompositeKey(key, &upserter, index);
   };
 
-  const HashTableT &source_hash_table =
-      static_cast<const HashTableT &>(distinctify_hash_table);
+  const PackedPayloadHashTable &source_hash_table =
+      static_cast<const PackedPayloadHashTable &>(distinctify_hash_table);
   // Invoke the lambda function "aggregate_functor" on each composite key vector
   // from the distinctify hash table.
-  source_hash_table.forEachCompositeKeyFast(&aggregate_functor);
+  source_hash_table.forEachCompositeKey(&aggregate_functor);
 }
 
-template <typename HandleT, typename HashTableT>
-ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
+template <typename HandleT>
+ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
     const Type &result_type,
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
   const HandleT &handle = static_cast<const HandleT &>(*this);
-  const HashTableT &hash_table_concrete =
-      static_cast<const HashTableT &>(hash_table);
+  const PackedPayloadHashTable &hash_table_concrete =
+      static_cast<const PackedPayloadHashTable &>(hash_table);
 
   if (group_by_keys->empty()) {
     if (NativeColumnVector::UsableForType(result_type)) {
@@ -287,14 +267,14 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
           new NativeColumnVector(result_type, hash_table_concrete.numEntries());
       HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
           handle, group_by_keys, result);
-      hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
+      hash_table_concrete.forEachCompositeKey(&finalizer, index);
       return result;
     } else {
       IndirectColumnVector *result = new IndirectColumnVector(
           result_type, hash_table_concrete.numEntries());
       HashTableAggregateFinalizer<HandleT, IndirectColumnVector> finalizer(
           handle, group_by_keys, result);
-      hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
+      hash_table_concrete.forEachCompositeKey(&finalizer, index);
       return result;
     }
   } else {
@@ -303,8 +283,8 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
           new NativeColumnVector(result_type, group_by_keys->size());
       for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
         result->appendTypedValue(
-            finalizeGroupInHashTableFast<HandleT, HashTableT>(
-                hash_table, group_by_key, index));
+            finalizeGroupInHashTable<HandleT>(
+                hash_table, index, group_by_key));
       }
       return result;
     } else {
@@ -312,8 +292,8 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
           result_type, hash_table_concrete.numEntries());
       for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
         result->appendTypedValue(
-            finalizeGroupInHashTableFast<HandleT, HashTableT>(
-                hash_table, group_by_key, index));
+            finalizeGroupInHashTable<HandleT>(
+                hash_table, index, group_by_key));
       }
       return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 4b51179..9c7b166 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -21,20 +21,21 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_HPP_
 
 #include <cstddef>
-#include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 class ColumnVector;
 class StorageManager;
 class Type;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -109,34 +110,34 @@ class AggregationHandle {
   virtual ~AggregationHandle() {}
 
   /**
-   * @brief Create an initial "blank" state for this aggregation.
+   * @brief Get the ID of this aggregation.
    *
-   * @return An initial "blank" state for this particular aggregation.
+   * @return The AggregationID of this AggregationHandle.
    **/
-  virtual AggregationState* createInitialState() const = 0;
+  inline AggregationID getAggregationID() const {
+    return agg_id_;
+  }
 
   /**
-   * @brief Create a new HashTable for aggregation with GROUP BY.
+   * @brief Get the list of Types (in order) for arguments to this aggregation.
    *
-   * @param hash_table_impl The choice of which concrete HashTable
-   *        implementation to use.
-   * @param group_by_types The types of the GROUP BY columns/expressions. These
-   *        correspond to the (composite) key type for the HashTable.
-   * @param estimated_num_groups The estimated number of distinct groups for
-   *        the GROUP BY aggregation. This is used to size the initial
-   *        HashTable. This is an estimate only, and the HashTable will be
-   *        resized if it becomes over-full.
-   * @param storage_manager The StorageManager to use to create the HashTable.
-   *        A StorageBlob will be allocated to serve as the HashTable's
-   *        in-memory storage.
-   * @return A new HashTable instance with the appropriate state type for this
-   *         aggregate.
+   * @return The list of Types for arguments to this aggregation.
+   */
+  virtual std::vector<const Type *> getArgumentTypes() const = 0;
+
+  /**
+   * @brief Get the result Type of this aggregation.
+   *
+   * @return The result Type of this aggregation.
+   */
+  virtual const Type* getResultType() const = 0;
+
+  /**
+   * @brief Create an initial "blank" state for this aggregation.
+   *
+   * @return An initial "blank" state for this particular aggregation.
    **/
-  virtual AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const = 0;
+  virtual AggregationState* createInitialState() const = 0;
 
   /**
    * @brief Accumulate over tuples for a nullary aggregate function (one that
@@ -146,70 +147,31 @@ class AggregationHandle {
    *        data is accessed, the only thing that a nullary aggeregate can know
    *        about input is its cardinality.
    * @return A new AggregationState which contains the accumulated results from
-   *         applying the (nullary) aggregate to the specified number of
-   *         tuples.
+   *         applying the (nullary) aggregate to the specified number of tuples.
    **/
   virtual AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const = 0;
-
-  /**
-   * @brief Accumulate (iterate over) all values in one or more ColumnVectors
-   *        and return a new AggregationState which can be merged with other
-   *        states or finalized.
-   *
-   * @param column_vectors One or more ColumnVectors that the aggregate will be
-   *        applied to. These correspond to the aggregate function's arguments,
-   *        in order.
-   * @return A new AggregationState which contains the accumulated results from
-   *         applying the aggregate to column_vectors. Caller is responsible
-   *         for deleting the returned AggregationState.
-   **/
-  virtual AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const = 0;
+      const std::size_t num_tuples) const {
+    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
+               << "takes at least one argument.";
+  }
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   /**
    * @brief Accumulate (iterate over) all values in columns accessible through
-   *        a ValueAccessor and return a new AggregationState which can be
-   *        merged with other states or finalized.
+   *        the ValueAccessors from a ValueAccessorMultiplexer and return a new
+   *        AggregationState which can be merged with other states or finalized.
    *
-   * @param accessor A ValueAccessor that the columns to be aggregated can be
-   *        accessed through.
-   * @param accessor_ids The attribute_ids that correspond to the columns in
-   *        accessor to aggeregate. These correspond to the aggregate
-   *        function's arguments, in order.
+   * @param argument_ids The multi-source attribute ids that correspond to the
+   *        columns in \p accessor_mux to aggeregate. These correspond to the
+   *        aggregate function's arguments, in order.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors.
    * @return A new AggregationState which contains the accumulated results from
-   *         applying the aggregate to the specified columns in accessor.
+   *         applying the aggregate to the specified columns.
    *         Caller is responsible for deleting the returned AggregationState.
    **/
   virtual AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const = 0;
-#endif
-
-  /**
-   * @brief Perform an aggregation with GROUP BY over all the tuples accessible
-   *        through a ValueAccessor, upserting states in a HashTable.
-   *
-   * @note Implementations of this method are threadsafe with respect to
-   *       hash_table, and can be called concurrently from multiple threads
-   *       with the same HashTable object.
-   *
-   * @param accessor The ValueAccessor that will be iterated over to read
-   *        tuples.
-   * @param argument_ids The attribute_ids of the arguments to this aggregate
-   *        in accessor, in order.
-   * @param group_by_key_ids The attribute_ids of the group-by
-   *        columns/expressions in accessor.
-   * @param hash_table The HashTable to upsert AggregationStates in. This
-   *        should have been created by calling createGroupByHashTable() on
-   *        this same AggregationHandle.
-   **/
-  virtual void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const = 0;
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const = 0;
 
   /**
    * @brief Merge two AggregationStates, updating one in-place. This computes a
@@ -253,24 +215,24 @@ class AggregationHandle {
    * @param hash_table The HashTable to finalize states from. This should have
    *        have been created by calling createGroupByHashTable() on this same
    *        AggregationHandle.
+   * @param index The index of the AggregationHandle to be finalized.
    * @param group_by_keys A pointer to a vector of vectors of GROUP BY keys. If
    *        this is initially empty, it will be filled in with the GROUP BY
    *        keys visited by this method in the same order as the finalized
    *        values returned in the ColumnVector. If this is already filled in,
    *        then this method will visit the GROUP BY keys in the exact order
    *        specified.
-   * @param index The index of the AggregationHandle to be finalized.
    *
    * @return A ColumnVector containing each group's finalized aggregate value.
    **/
   virtual ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const = 0;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const = 0;
 
   /**
    * @brief Create a new HashTable for the distinctify step for DISTINCT
-   * aggregation.
+   *        aggregation.
    *
    * Distinctify is the first step for DISTINCT aggregation. This step inserts
    * the GROUP BY expression values and aggregation arguments together as keys
@@ -283,8 +245,8 @@ class AggregationHandle {
    * we simply treat it as a special GROUP BY case that the GROUP BY expression
    * vector is empty.
    *
-   * @param hash_table_impl The choice of which concrete HashTable
-   *        implementation to use.
+   * @param hash_table_impl The choice of which concrete HashTable implementation
+   *        to use.
    * @param key_types The types of the GROUP BY expressions together with the
    *        types of the aggregation arguments.
    * @param estimated_num_distinct_keys The estimated number of distinct keys
@@ -307,13 +269,14 @@ class AggregationHandle {
 
   /**
    * @brief Inserts the GROUP BY expressions and aggregation arguments together
-   * as keys into the distinctify hash table.
+   *        as keys into the distinctify hash table.
    *
-   * @param accessor The ValueAccessor that will be iterated over to read
-   *        tuples.
-   * @param key_ids The attribute_ids of the GROUP BY expressions in accessor
-   *        together with the attribute_ids of the arguments to this aggregate
-   *        in accessor, in order.
+   * @param argument_ids The argument ids that correspond to the columns in
+   *        \p accessor_mux.
+   * @param key_ids The group-by key ids that correspond to the columns in
+   *        \p accessor_mux.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors to be iterated over to read tuples.
    * @param distinctify_hash_table The HashTable to store the GROUP BY
    *        expressions and the aggregation arguments together as hash table
    *        keys and a bool constant \c true as hash table value (So the hash
@@ -321,13 +284,14 @@ class AggregationHandle {
    *        by calling createDistinctifyHashTable();
    */
   virtual void insertValueAccessorIntoDistinctifyHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_ids,
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux,
       AggregationStateHashTableBase *distinctify_hash_table) const = 0;
 
   /**
    * @brief Perform single (i.e. without GROUP BY) aggregation on the keys from
-   * the distinctify hash table to actually compute the aggregated results.
+   *        the distinctify hash table to actually compute the aggregated results.
    *
    * @param distinctify_hash_table Hash table which stores the distinctified
    *        aggregation arguments as hash table keys. This should have been
@@ -346,25 +310,26 @@ class AggregationHandle {
    * @param distinctify_hash_table Hash table which stores the GROUP BY
    *        expression values and aggregation arguments together as hash table
    *        keys.
+   * @param index The index of the AggregationHandle to perform aggregation.
    * @param aggregation_hash_table The HashTable to upsert AggregationStates in.
    *        This should have been created by calling createGroupByHashTable() on
    *        this same AggregationHandle.
-   * @param index The index of the distinctify hash table for which we perform
-   *        the DISTINCT aggregation.
    */
   virtual void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const = 0;
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const = 0;
 
   /**
    * @brief Get the number of bytes needed to store the aggregation handle's
    *        state.
    **/
-  virtual std::size_t getPayloadSize() const { return 1; }
+  virtual std::size_t getPayloadSize() const {
+    return 1u;
+  }
 
   /**
-   * @brief Update the aggregation state for nullary aggregation function e.g.
+   * @brief Update the aggregation state for nullary aggregation function, e.g.
    *        COUNT(*).
    *
    * @note This function should be overloaded by those aggregation function
@@ -372,7 +337,10 @@ class AggregationHandle {
    *
    * @param byte_ptr The pointer where the aggregation state is stored.
    **/
-  virtual void updateStateNullary(std::uint8_t *byte_ptr) const {}
+  virtual void updateStateNullary(std::uint8_t *byte_ptr) const {
+    LOG(FATAL) << "Called updateStateNullary on an AggregationHandle that "
+               << "takes at least one argument.";
+  }
 
   /**
    * @brief Update the aggregation state for unary aggregation function e.g.
@@ -383,7 +351,7 @@ class AggregationHandle {
    * @param byte_ptr The pointer where the aggregation state is stored.
    **/
   virtual void updateStateUnary(const TypedValue &argument,
-                                std::uint8_t *byte_ptr) const {}
+                                std::uint8_t *byte_ptr) const = 0;
 
   /**
    * @brief Merge two aggregation states for this aggregation handle.
@@ -394,8 +362,8 @@ class AggregationHandle {
    * @param src A pointer to the source aggregation state.
    * @param dst A pointer to the destination aggregation state.
    **/
-  virtual void mergeStatesFast(const std::uint8_t *src,
-                               std::uint8_t *dst) const {}
+  virtual void mergeStates(const std::uint8_t *src,
+                           std::uint8_t *dst) const = 0;
 
   /**
    * @brief Initialize the payload (in the aggregation hash table) for the given
@@ -403,7 +371,7 @@ class AggregationHandle {
    *
    * @param byte_ptr The pointer to the aggregation state in the hash table.
    **/
-  virtual void initPayload(std::uint8_t *byte_ptr) const {}
+  virtual void initPayload(std::uint8_t *byte_ptr) const = 0;
 
   /**
    * @brief Destroy the payload (in the aggregation hash table) for the given
@@ -411,22 +379,25 @@ class AggregationHandle {
    *
    * @param byte_ptr The pointer to the aggregation state in the hash table.
    **/
-  virtual void destroyPayload(std::uint8_t *byte_ptr) const {}
+  virtual void destroyPayload(std::uint8_t *byte_ptr) const = 0;
 
   /**
    * @brief Inform the aggregation handle to block (prohibit) updates on the
    *        aggregation state.
    **/
-  virtual void blockUpdate() {}
+  virtual void blockUpdate() = 0;
 
   /**
-   * @brief Inform the aggregation handle to allow updates on the
-   *        aggregation state.
+   * @brief Inform the aggregation handle to allow updates on the aggregation
+   *        state.
    **/
-  virtual void allowUpdate() {}
+  virtual void allowUpdate() = 0;
 
  protected:
-  AggregationHandle() {}
+  explicit AggregationHandle(const AggregationID agg_id)
+      : agg_id_(agg_id) {}
+
+  const AggregationID agg_id_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationHandle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 2481092..46bec1e 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -20,12 +20,13 @@
 #include "expressions/aggregation/AggregationHandleAvg.hpp"
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -39,10 +40,11 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 AggregationHandleAvg::AggregationHandleAvg(const Type &type)
-    : argument_type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kAvg),
+      argument_type_(type) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -87,52 +89,29 @@ AggregationHandleAvg::AggregationHandleAvg(const Type &type)
             ->getNullableVersion());
 }
 
-AggregationStateHashTableBase* AggregationHandleAvg::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type *> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<AggregationStateAvg>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
+AggregationState* AggregationHandleAvg::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for AVG: " << argument_ids.size();
 
-AggregationState* AggregationHandleAvg::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for AVG: " << column_vectors.size();
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
-  std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateColumnVector(
-      state->sum_, *column_vectors.front(), &count);
-  state->count_ = count;
-  return state;
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleAvg::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for AVG: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
   std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateValueAccessor(
-      state->sum_, accessor, accessor_ids.front(), &count);
+  state->sum_ =
+      fast_add_operator_->accumulateValueAccessor(
+          state->sum_,
+          accessor_mux.getValueAccessorBySource(argument_source),
+          argument_id,
+          &count);
   state->count_ = count;
   return state;
 }
-#endif
-
-void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "Got wrong number of arguments for AVG: " << argument_ids.size();
-}
 
 void AggregationHandleAvg::mergeStates(const AggregationState &source,
                                        AggregationState *destination) const {
@@ -147,8 +126,8 @@ void AggregationHandleAvg::mergeStates(const AggregationState &source,
       avg_destination->sum_, avg_source.sum_);
 }
 
-void AggregationHandleAvg::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleAvg::mergeStates(const std::uint8_t *source,
+                                       std::uint8_t *destination) const {
   const TypedValue *src_sum_ptr =
       reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset_);
   const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(
@@ -177,29 +156,25 @@ TypedValue AggregationHandleAvg::finalize(const AggregationState &state) const {
 
 ColumnVector* AggregationHandleAvg::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleAvg,
-                                     AggregationStateFastHashTable>(
-      *result_type_, hash_table, group_by_keys, index);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleAvg>(
+      *result_type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleAvg,
-      AggregationStateAvg>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleAvg, AggregationStateAvg>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleAvg,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleAvg>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 47132c6..970561c 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -25,11 +25,9 @@
 #include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +38,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -106,19 +103,18 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
  public:
   ~AggregationHandleAvg() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&argument_type_};
+  }
+
+  const Type* getResultType() const override {
+    return result_type_;
+  }
+
   AggregationState* createInitialState() const override {
     return new AggregationStateAvg(blank_state_);
   }
 
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
-  /**
-   * @brief Iterate method with average aggregation state.
-   **/
   inline void iterateUnaryInl(AggregationStateAvg *state,
                               const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
@@ -129,8 +125,8 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++state->count_;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
     TypedValue *sum_ptr =
@@ -141,16 +137,18 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++(*count_ptr);
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  TypedValue finalize(const AggregationState &state) const override;
+
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr =
@@ -169,43 +167,17 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     }
   }
 
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_id) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override;
-
-  void mergeStatesFast(const std::uint8_t *source,
-                       std::uint8_t *destination) const override;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
+  }
 
-  TypedValue finalize(const AggregationState &state) const override;
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
 
   inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    const AggregationStateAvg &agg_state =
-        static_cast<const AggregationStateAvg &>(state);
-    // TODO(chasseur): Could improve performance further if we made a special
-    // version of finalizeHashTable() that collects all the sums into one
-    // ColumnVector and all the counts into another and then applies
-    // '*divide_operator_' to them in bulk.
-    return divide_operator_->applyToTypedValues(
-        agg_state.sum_, TypedValue(static_cast<double>(agg_state.count_)));
-  }
-
-  inline TypedValue finalizeHashTableEntryFast(
       const std::uint8_t *byte_ptr) const {
     std::uint8_t *value_ptr = const_cast<std::uint8_t *>(byte_ptr);
     TypedValue *sum_ptr =
@@ -218,31 +190,16 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for AVG aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for AVG aggregation.
-   */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override {
-    return blank_state_.getPayloadSize();
-  }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionAvg;
@@ -261,8 +218,6 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> merge_add_operator_;
   std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleAvg);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index 034c942..cf92ec7 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -21,100 +21,48 @@
 
 #include <atomic>
 #include <cstddef>
-#include <memory>
+#include <cstdint>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "storage/ValueAccessorUtil.hpp"
-#endif
-
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/containers/ColumnVectorUtil.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-class StorageManager;
-class Type;
-class ValueAccessor;
+class ColumnVector;
 
 template <bool count_star, bool nullable_type>
-AggregationStateHashTableBase*
-AggregationHandleCount<count_star, nullable_type>::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type *> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<
-      AggregationStateCount>::CreateResizable(hash_table_impl,
-                                              group_by_types,
-                                              estimated_num_groups,
-                                              storage_manager);
-}
-
-template <bool count_star, bool nullable_type>
-AggregationState*
-AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+AggregationState* AggregationHandleCount<count_star, nullable_type>
+    ::accumulateValueAccessor(
+        const std::vector<MultiSourceAttributeId> &argument_ids,
+        const ValueAccessorMultiplexer &accessor_mux) const {
   DCHECK(!count_star)
       << "Called non-nullary accumulation method on an AggregationHandleCount "
       << "set up for nullary COUNT(*)";
 
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for COUNT: "
-      << column_vectors.size();
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for COUNT: " << argument_ids.size();
 
-  std::size_t count = 0;
-  InvokeOnColumnVector(
-      *column_vectors.front(),
-      [&](const auto &column_vector) -> void {  // NOLINT(build/c++11)
-        if (nullable_type) {
-          // TODO(shoban): Iterating over the ColumnVector is a rather slow way
-          // to do this. We should look at extending the ColumnVector interface
-          // to do a quick count of the non-null values (i.e. the length minus
-          // the population count of the null bitmap). We should do something
-          // similar for ValueAccessor too.
-          for (std::size_t pos = 0; pos < column_vector.size(); ++pos) {
-            count += !column_vector.getTypedValue(pos).isNull();
-          }
-        } else {
-          count = column_vector.size();
-        }
-      });
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  return new AggregationStateCount(count);
-}
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-template <bool count_star, bool nullable_type>
-AggregationState*
-AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK(!count_star)
-      << "Called non-nullary accumulation method on an AggregationHandleCount "
-      << "set up for nullary COUNT(*)";
-
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for COUNT: " << accessor_ids.size();
-
-  const attribute_id accessor_id = accessor_ids.front();
   std::size_t count = 0;
   InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&accessor_id, &count](auto *accessor) -> void {  // NOLINT(build/c++11)
+      accessor_mux.getValueAccessorBySource(argument_source),
+      [&argument_id, &count](auto *accessor) -> void {  // NOLINT(build/c++11)
         if (nullable_type) {
           while (accessor->next()) {
-            count += !accessor->getTypedValue(accessor_id).isNull();
+            count += !accessor->getTypedValue(argument_id).isNull();
           }
         } else {
           count = accessor->getNumTuples();
@@ -123,24 +71,6 @@ AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
 
   return new AggregationStateCount(count);
 }
-#endif
-
-template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::
-    aggregateValueAccessorIntoHashTable(
-        ValueAccessor *accessor,
-        const std::vector<attribute_id> &argument_ids,
-        const std::vector<attribute_id> &group_by_key_ids,
-        AggregationStateHashTableBase *hash_table) const {
-  if (count_star) {
-    DCHECK_EQ(0u, argument_ids.size())
-        << "Got wrong number of arguments for COUNT(*): "
-        << argument_ids.size();
-  } else {
-    DCHECK_EQ(1u, argument_ids.size())
-        << "Got wrong number of arguments for COUNT: " << argument_ids.size();
-  }
-}
 
 template <bool count_star, bool nullable_type>
 void AggregationHandleCount<count_star, nullable_type>::mergeStates(
@@ -156,7 +86,7 @@ void AggregationHandleCount<count_star, nullable_type>::mergeStates(
 }
 
 template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
+void AggregationHandleCount<count_star, nullable_type>::mergeStates(
     const std::uint8_t *source, std::uint8_t *destination) const {
   const std::int64_t *src_count_ptr =
       reinterpret_cast<const std::int64_t *>(source);
@@ -165,38 +95,35 @@ void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
 }
 
 template <bool count_star, bool nullable_type>
-ColumnVector*
-AggregationHandleCount<count_star, nullable_type>::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<
-      AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateFastHashTable>(
-      TypeFactory::GetType(kLong), hash_table, group_by_keys, index);
+ColumnVector* AggregationHandleCount<count_star, nullable_type>
+    ::finalizeHashTable(
+        const AggregationStateHashTableBase &hash_table,
+        const std::size_t index,
+        std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<
+      AggregationHandleCount<count_star, nullable_type>>(
+          TypeFactory::GetType(kLong), hash_table, index, group_by_keys);
 }
 
 template <bool count_star, bool nullable_type>
-AggregationState* AggregationHandleCount<count_star, nullable_type>::
-    aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleCount<count_star, nullable_type>
+    ::aggregateOnDistinctifyHashTableForSingle(
         const AggregationStateHashTableBase &distinctify_hash_table) const {
-  DCHECK_EQ(count_star, false);
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
       AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateCount>(distinctify_hash_table);
+      AggregationStateCount>(
+          distinctify_hash_table);
 }
 
 template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::
-    aggregateOnDistinctifyHashTableForGroupBy(
+void AggregationHandleCount<count_star, nullable_type>
+    ::aggregateOnDistinctifyHashTableForGroupBy(
         const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table,
-        std::size_t index) const {
-  DCHECK_EQ(count_star, false);
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+        const std::size_t index,
+        AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleCount<count_star, nullable_type>>(
+          distinctify_hash_table, index, aggregation_hash_table);
 }
 
 // Explicitly instantiate and compile in the different versions of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 6aab0cd..72ea923 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -23,23 +23,21 @@
 #include <atomic>
 #include <cstddef>
 #include <cstdint>
-#include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "types/LongType.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
 class Type;
-class ValueAccessor;
 
 template <bool, bool>
 class AggregationHandleCount;
@@ -98,28 +96,31 @@ class AggregationHandleCount : public AggregationConcreteHandle {
  public:
   ~AggregationHandleCount() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    if (argument_type_ == nullptr) {
+      return {};
+    } else {
+      return {argument_type_};
+    }
+  }
+
+  const Type* getResultType() const override {
+    return &LongType::InstanceNonNullable();
+  }
+
   AggregationState* createInitialState() const override {
     return new AggregationStateCount();
   }
 
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
   inline void iterateNullaryInl(AggregationStateCount *state) const {
     state->count_.fetch_add(1, std::memory_order_relaxed);
   }
 
-  inline void iterateNullaryInlFast(std::uint8_t *byte_ptr) const {
+  inline void iterateNullaryInl(std::uint8_t *byte_ptr) const {
     std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-    (*count_ptr)++;
+    ++(*count_ptr);
   }
 
-  /**
-   * @brief Iterate with count aggregation state.
-   */
   inline void iterateUnaryInl(AggregationStateCount *state,
                               const TypedValue &value) const {
     if ((!nullable_type) || (!value.isNull())) {
@@ -127,118 +128,89 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     }
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     if ((!nullable_type) || (!value.isNull())) {
-      std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-      (*count_ptr)++;
-    }
-  }
-
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
+      ++(*reinterpret_cast<std::int64_t *>(byte_ptr));
     }
   }
 
-  inline void updateStateNullary(std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateNullaryInlFast(byte_ptr);
-    }
-  }
-
-  void blockUpdate() override { block_update_ = true; }
-
-  void allowUpdate() override { block_update_ = false; }
-
-  void initPayload(std::uint8_t *byte_ptr) const override {
-    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-    *count_ptr = 0;
-  }
-
   AggregationState* accumulateNullary(
       const std::size_t num_tuples) const override {
     return new AggregationStateCount(num_tuples);
   }
 
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_id) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
-  void mergeStatesFast(const std::uint8_t *source,
-                       std::uint8_t *destination) const override;
-
   TypedValue finalize(const AggregationState &state) const override {
     return TypedValue(
         static_cast<const AggregationStateCount &>(state).count_.load(
             std::memory_order_relaxed));
   }
 
-  inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    return TypedValue(
-        static_cast<const AggregationStateCount &>(state).count_.load(
-            std::memory_order_relaxed));
+  std::size_t getPayloadSize() const override {
+    return sizeof(std::int64_t);
+  }
+
+  void initPayload(std::uint8_t *byte_ptr) const override {
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+    *count_ptr = 0;
   }
 
-  inline TypedValue finalizeHashTableEntryFast(
-      const std::uint8_t *byte_ptr) const {
-    const std::int64_t *count_ptr =
-        reinterpret_cast<const std::int64_t *>(byte_ptr);
-    return TypedValue(*count_ptr);
+  void destroyPayload(std::uint8_t *byte_ptr) const override {}
+
+  inline void updateStateNullary(std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateNullaryInl(byte_ptr);
+    }
+  }
+
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
+  }
+
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
+
+  inline TypedValue finalizeHashTableEntry(const std::uint8_t *byte_ptr) const {
+    return TypedValue(*reinterpret_cast<const std::int64_t *>(byte_ptr));
   }
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for SUM aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for SUM aggregation.
-   */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override { return sizeof(std::int64_t); }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionCount;
 
   /**
-   * @brief Constructor.
+   * @brief Initialize handle for count.
+   *
+   * @param argument_type Type of the value to be counted. The parameter should
+   *        be nullptr for nullary aggregation (i.e. COUNT(*)).
    **/
-  AggregationHandleCount() : block_update_(false) {}
+  explicit AggregationHandleCount(const Type *argument_type)
+      : AggregationConcreteHandle(AggregationID::kCount),
+        argument_type_(argument_type) {}
 
-  bool block_update_;
+  const Type *argument_type_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleCount);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
deleted file mode 100644
index 0dc8b56..0000000
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * 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/aggregation/AggregationHandleDistinct.hpp"
-
-#include <cstddef>
-#include <memory>
-#include <vector>
-#include <utility>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-
-#include "types/TypedValue.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class ColumnVector;
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-AggregationStateHashTableBase* AggregationHandleDistinct::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return createDistinctifyHashTable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
-}
-
-void AggregationHandleDistinct::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(argument_ids.size(), 0u);
-
-  insertValueAccessorIntoDistinctifyHashTable(
-      accessor,
-      group_by_key_ids,
-      hash_table);
-}
-
-ColumnVector* AggregationHandleDistinct::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  DCHECK(group_by_keys->empty());
-
-  const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
-                                               const bool &dumb_placeholder) -> void {
-    group_by_keys->emplace_back(std::move(group_by_key));
-  };
-  static_cast<const AggregationStateFastHashTable&>(hash_table).forEachCompositeKeyFast(&keys_retriever);
-
-  return nullptr;
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
deleted file mode 100644
index 838bfdd..0000000
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * 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_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_
-#define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_
-
-#include <cstddef>
-#include <memory>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
-#include "storage/HashTableBase.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class AggregationState;
-class ColumnVector;
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-/** \addtogroup Expressions
- *  @{
- */
-
-class AggregationHandleDistinct : public AggregationConcreteHandle {
- public:
-  /**
-   * @brief Constructor.
-   **/
-  AggregationHandleDistinct() {}
-
-  AggregationState* createInitialState() const override {
-    LOG(FATAL)
-        << "AggregationHandleDistinct does not support createInitialState().";
-  }
-
-  AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const override {
-    LOG(FATAL)
-        << "AggregationHandleDistinct does not support accumulateNullary().";
-  }
-
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-                  "accumulateColumnVectors().";
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-                  "accumulateValueAccessor().";
-  }
-#endif
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support mergeStates().";
-  }
-
-  TypedValue finalize(const AggregationState &state) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support finalize().";
-  }
-
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-               << "aggregateOnDistinctifyHashTableForSingle().";
-  }
-
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *groupby_hash_table,
-      std::size_t index) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-               << "aggregateOnDistinctifyHashTableForGroupBy().";
-  }
-
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  ColumnVector* finalizeHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(AggregationHandleDistinct);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index c2d571b..fe1773f 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -19,15 +19,16 @@
 
 #include "expressions/aggregation/AggregationHandleMax.hpp"
 
+#include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
@@ -36,54 +37,32 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 AggregationHandleMax::AggregationHandleMax(const Type &type)
-    : type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kMax),
+      type_(type) {
   fast_comparator_.reset(
       ComparisonFactory::GetComparison(ComparisonID::kGreater)
           .makeUncheckedComparatorForTypes(type, type.getNonNullableVersion()));
 }
 
-AggregationStateHashTableBase* AggregationHandleMax::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type *> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<AggregationStateMax>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
+AggregationState* AggregationHandleMax::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for MAX: " << argument_ids.size();
 
-AggregationState* AggregationHandleMax::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for MAX: " << column_vectors.size();
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  return new AggregationStateMax(fast_comparator_->accumulateColumnVector(
-      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleMax::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for MAX: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   return new AggregationStateMax(fast_comparator_->accumulateValueAccessor(
       type_.getNullableVersion().makeNullValue(),
-      accessor,
-      accessor_ids.front()));
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "Got wrong number of arguments for MAX: " << argument_ids.size();
+      accessor_mux.getValueAccessorBySource(argument_source),
+      argument_id));
 }
 
 void AggregationHandleMax::mergeStates(const AggregationState &source,
@@ -98,40 +77,36 @@ void AggregationHandleMax::mergeStates(const AggregationState &source,
   }
 }
 
-void AggregationHandleMax::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleMax::mergeStates(const std::uint8_t *source,
+                                       std::uint8_t *destination) const {
   const TypedValue *src_max_ptr = reinterpret_cast<const TypedValue *>(source);
   TypedValue *dst_max_ptr = reinterpret_cast<TypedValue *>(destination);
   if (!(src_max_ptr->isNull())) {
-    compareAndUpdateFast(dst_max_ptr, *src_max_ptr);
+    compareAndUpdate(dst_max_ptr, *src_max_ptr);
   }
 }
 
 ColumnVector* AggregationHandleMax::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleMax,
-                                     AggregationStateFastHashTable>(
-      type_.getNullableVersion(), hash_table, group_by_keys, index);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleMax>(
+      type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleMax,
-      AggregationStateMax>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleMax, AggregationStateMax>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleMax,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleMax>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep


[17/50] [abbrv] incubator-quickstep git commit: Saved catalog in the distributed version.

Posted by ji...@apache.org.
Saved catalog in 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/27a80558
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/27a80558
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/27a80558

Branch: refs/heads/LIP-time-decomposition
Commit: 27a8055872f82737c35f6f0914ce43bcbe272ce3
Parents: dda085c
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sun Feb 5 02:16:34 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Sun Feb 5 02:16:34 2017 -0800

----------------------------------------------------------------------
 cli/distributed/Conductor.cpp                             |  5 ++++-
 query_execution/ForemanDistributed.cpp                    |  3 +++
 query_execution/ForemanDistributed.hpp                    |  4 ++++
 query_execution/PolicyEnforcerDistributed.cpp             |  2 ++
 query_execution/PolicyEnforcerDistributed.hpp             |  7 +++++++
 .../tests/DistributedExecutionGeneratorTestRunner.cpp     | 10 +++++++++-
 6 files changed, 29 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27a80558/cli/distributed/Conductor.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Conductor.cpp b/cli/distributed/Conductor.cpp
index c4a2721..13d4d57 100644
--- a/cli/distributed/Conductor.cpp
+++ b/cli/distributed/Conductor.cpp
@@ -22,6 +22,7 @@
 #include <cstddef>
 #include <cstdlib>
 #include <exception>
+#include <functional>
 #include <memory>
 #include <sstream>
 #include <string>
@@ -95,7 +96,9 @@ void Conductor::init() {
   block_locator_ = make_unique<BlockLocator>(&bus_);
   block_locator_->start();
 
-  foreman_ = make_unique<ForemanDistributed>(*block_locator_, &bus_, query_processor_->getDefaultDatabase());
+  foreman_ = make_unique<ForemanDistributed>(*block_locator_,
+                                             std::bind(&QueryProcessor::saveCatalog, query_processor_.get()), &bus_,
+                                             query_processor_->getDefaultDatabase());
   foreman_->start();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27a80558/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index fe4c483..4d95f16 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -17,6 +17,7 @@
 #include <cstddef>
 #include <cstdio>
 #include <cstdlib>
+#include <functional>
 #include <memory>
 #include <unordered_map>
 #include <unordered_set>
@@ -67,6 +68,7 @@ class QueryHandle;
 
 ForemanDistributed::ForemanDistributed(
     const BlockLocator &block_locator,
+    std::function<void()> &&save_catalog_callback,
     MessageBus *bus,
     CatalogDatabaseLite *catalog_database,
     const int cpu_id)
@@ -106,6 +108,7 @@ ForemanDistributed::ForemanDistributed(
 
   policy_enforcer_ = std::make_unique<PolicyEnforcerDistributed>(
       foreman_client_id_,
+      move(save_catalog_callback),
       catalog_database_,
       &shiftboss_directory_,
       bus_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27a80558/query_execution/ForemanDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.hpp b/query_execution/ForemanDistributed.hpp
index ed09fda..5f1a14b 100644
--- a/query_execution/ForemanDistributed.hpp
+++ b/query_execution/ForemanDistributed.hpp
@@ -17,6 +17,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <functional>
 #include <memory>
 #include <unordered_map>
 #include <unordered_set>
@@ -55,6 +56,8 @@ class ForemanDistributed final : public ForemanBase {
    * @param block_locator The block locator that manages block location info.
    * @param bus A pointer to the TMB.
    * @param catalog_database The catalog database where this query is executed.
+   * @param save_catalog_callback The callback used to save catalog upon the query
+   *        completion.
    * @param cpu_id The ID of the CPU to which the Foreman thread can be pinned.
    *
    * @note If cpu_id is not specified, Foreman thread can be possibly moved
@@ -62,6 +65,7 @@ class ForemanDistributed final : public ForemanBase {
   **/
   ForemanDistributed(
       const BlockLocator &block_locator,
+      std::function<void()> &&save_catalog_callback,
       tmb::MessageBus *bus,
       CatalogDatabaseLite *catalog_database,
       const int cpu_id = -1);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27a80558/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index e9f57d3..38b8a34 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -227,6 +227,8 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
   }
 
   if (query_result == nullptr) {
+    save_catalog_callback_();
+
     // Clean up query execution states, i.e., QueryContext, in Shiftbosses.
     serialization::QueryTeardownMessage proto;
     proto.set_query_id(query_id);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27a80558/query_execution/PolicyEnforcerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.hpp b/query_execution/PolicyEnforcerDistributed.hpp
index 248948a..f8476c8 100644
--- a/query_execution/PolicyEnforcerDistributed.hpp
+++ b/query_execution/PolicyEnforcerDistributed.hpp
@@ -16,6 +16,7 @@
 #define QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
 
 #include <cstddef>
+#include <functional>
 #include <memory>
 #include <unordered_map>
 #include <utility>
@@ -57,15 +58,19 @@ class PolicyEnforcerDistributed final : public PolicyEnforcerBase {
    * @brief Constructor.
    *
    * @param foreman_client_id The TMB client ID of the Foreman.
+   * @param save_catalog_callback The callback used to save catalog upon the query
+   *        completion.
    * @param catalog_database The CatalogDatabase used.
    * @param bus The TMB.
    **/
   PolicyEnforcerDistributed(const tmb::client_id foreman_client_id,
+                            std::function<void()> &&save_catalog_callback,
                             CatalogDatabaseLite *catalog_database,
                             ShiftbossDirectory *shiftboss_directory,
                             tmb::MessageBus *bus)
       : PolicyEnforcerBase(catalog_database),
         foreman_client_id_(foreman_client_id),
+        save_catalog_callback_(std::move(save_catalog_callback)),
         shiftboss_directory_(shiftboss_directory),
         bus_(bus) {}
 
@@ -154,6 +159,8 @@ class PolicyEnforcerDistributed final : public PolicyEnforcerBase {
 
   const tmb::client_id foreman_client_id_;
 
+  const std::function<void()> save_catalog_callback_;
+
   ShiftbossDirectory *shiftboss_directory_;
 
   tmb::MessageBus *bus_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27a80558/query_optimizer/tests/DistributedExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/DistributedExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/DistributedExecutionGeneratorTestRunner.cpp
index 45d4fdf..2e18467 100644
--- a/query_optimizer/tests/DistributedExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/DistributedExecutionGeneratorTestRunner.cpp
@@ -21,6 +21,7 @@
 
 #include <cstdio>
 #include <cstdlib>
+#include <functional>
 #include <memory>
 #include <set>
 #include <string>
@@ -64,6 +65,12 @@ class CatalogRelation;
 
 namespace optimizer {
 
+namespace {
+
+void nop() {}
+
+}  // namespace
+
 const char *DistributedExecutionGeneratorTestRunner::kResetOption =
     "reset_before_execution";
 
@@ -98,7 +105,8 @@ DistributedExecutionGeneratorTestRunner::DistributedExecutionGeneratorTestRunner
 
   // NOTE(zuyu): Foreman should initialize before Shiftboss so that the former
   // could receive a registration message from the latter.
-  foreman_ = make_unique<ForemanDistributed>(*block_locator_, &bus_, test_database_loader_->catalog_database());
+  foreman_ = make_unique<ForemanDistributed>(*block_locator_, std::bind(&nop), &bus_,
+                                             test_database_loader_->catalog_database());
 
   // We don't use the NUMA aware version of worker code.
   const vector<numa_node_id> numa_nodes(1 /* Number of worker threads per instance */,


[06/50] [abbrv] incubator-quickstep git commit: Fixed the linking issue for the distributed cli.

Posted by ji...@apache.org.
Fixed the linking issue for the distributed cli.


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

Branch: refs/heads/LIP-time-decomposition
Commit: dff4a145e2c2d3d7b84fb259e48e425310a52a8a
Parents: 259cd5e
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 31 12:19:00 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 31 12:19:00 2017 -0800

----------------------------------------------------------------------
 cli/distributed/CMakeLists.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dff4a145/cli/distributed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/distributed/CMakeLists.txt b/cli/distributed/CMakeLists.txt
index a00ffda..1069abd 100644
--- a/cli/distributed/CMakeLists.txt
+++ b/cli/distributed/CMakeLists.txt
@@ -28,6 +28,7 @@ target_link_libraries(quickstep_cli_distributed_Cli
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_cli_Flags
+                      quickstep_cli_LineReader
                       quickstep_cli_PrintToScreen
                       quickstep_cli_distributed_Role
                       quickstep_parser_ParseStatement


[21/50] [abbrv] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 16ea50f..25d675c 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -27,7 +27,6 @@
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/CountedReference.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/StorageBlockBase.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -39,11 +38,7 @@
 
 namespace quickstep {
 
-class AggregationHandle;
-class AggregationState;
 class CatalogRelationSchema;
-class ColumnVector;
-class ColumnVectorsValueAccessor;
 class InsertDestinationInterface;
 class Predicate;
 class Scalar;
@@ -431,156 +426,6 @@ class StorageBlock : public StorageBlockBase {
                     InsertDestinationInterface *destination) const;
 
   /**
-   * @brief Perform non GROUP BY aggregation on the tuples in the this storage
-   *        block, returning the aggregated result (for this block) in an
-   *        AggregationState.
-   *
-   * @param handle Aggregation handle that will be used to compute aggregate.
-   * @param arguments The arguments of the aggregate function as expressions.
-   * @param arguments_as_attributes If non-NULL, indicates a valid attribute_id
-   *        for each of the elements in arguments, and is used to elide a copy.
-   *        Has no effect if NULL, or if VECTOR_COPY_ELISION_LEVEL is NONE.
-   * @param filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   *
-   * @return Aggregated state for this block in the form of an
-   *         AggregationState. AggregationHandle::mergeStates() can be called
-   *         to merge with states from other blocks, and
-   *         AggregationHandle::finalize() can be used to generate a final
-   *         result.
-   **/
-  AggregationState* aggregate(
-      const AggregationHandle &handle,
-      const std::vector<std::unique_ptr<const Scalar>> &arguments,
-      const std::vector<attribute_id> *arguments_as_attributes,
-      const TupleIdSequence *filter) const;
-
-  /**
-   * @brief Perform GROUP BY aggregation on the tuples in the this storage
-   *        block.
-   *
-   * @param arguments The arguments to the aggregation function as Scalars.
-   * @param group_by The list of GROUP BY attributes/expressions. The tuples in
-   *        this storage block are grouped by these attributes before
-   *        aggregation.
-   * @param filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   * @param hash_table Hash table to store aggregation state mapped based on
-   *        GROUP BY value list (defined by \c group_by).
-   * @param reuse_group_by_vectors This parameter is used to store and reuse
-   *        GROUP BY attribute vectors pre-computed in an earlier invocation of
-   *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
-   *        for ease of use. Current invocation of aggregateGroupBy() will reuse
-   *        ColumnVectors if non-empty, otherwise computes ColumnVectors based
-   *        on \c group_by and stores them in \c reuse_group_by_vectors.
-   *
-   * For sample usage of aggregateGroupBy, see this relevant pseudo-C++ code:
-   * \code
-   * std::vector<std::unique_ptr<ColumnVector>> group_by_vectors;
-   * for each aggregate {
-   *   block.aggregateGroupBy(..., &group_by_vectors);
-   * }
-   * \endcode
-   **/
-  /*
-   * TODO(shoban): Currently, we use ColumnVectorsValueAccessor to compute
-   * temporary result for Scalars of aggregation attributes and GROUP BY
-   * attributes.  We will have to support specifying aggregation and GROUP BY
-   * attributes as std::vector<attribute_id> (like in selectSimple()) for fast
-   * path when there are no expressions specified in the query.
-   */
-  void aggregateGroupBy(
-      const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-      const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const TupleIdSequence *filter,
-      AggregationStateHashTableBase *hash_table,
-      std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
-
-
-  /**
-   * @brief Perform the GROUP BY aggregation for the case when aggregation is
-   *        partitioned.
-   *
-   * TODO(harshad) - Refactor this class to use only one function
-   *       aggregateGroupBy.
-   * @note The difference between this method and the aggregateGroupBy method
-   *       is that in this method, the tuples are routed to different HashTables
-   *       based on the partition to which they belong to. The partition is
-   *       determined by the GROUP BY attributes. Right now hash based
-   *       partitioning is performed.
-   *
-   * @note This function only creates the ColumnVectorsValueAccessor needed for
-   *       the insertion in the hash table. The actual insertion in respective
-   *       hash tables should be handled by the caller. See
-   *       AggregationOperationState::aggregateHashTable() for one such
-   *       implementation.
-   *
-   * @param arguments The arguments to the aggregation function as Scalars.
-   * @param group_by The list of GROUP BY attributes/expressions. The tuples in
-   *        this storage block are grouped by these attributes before
-   *        aggregation.
-   * @param filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   * @param num_partitions The number of partitions used for the aggregation.
-   * @param temp_result The ColumnVectorsValueAccessor used for collecting
-   *        the attribute values from this StorageBlock.
-   * @param arguments_ids The attribute IDs used for the aggregation, which
-   *        come from the arguments vector. If arguments is empty, this vector
-   *        is filled with invalid attribute IDs.
-   * @param key_ids The attribute IDs of the group by attributes.
-   * @param reuse_group_by_vectors This parameter is used to store and reuse
-   *        GROUP BY attribute vectors pre-computed in an earlier invocation of
-   *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
-   *        for ease of use. Current invocation of aggregateGroupBy() will reuse
-   *        ColumnVectors if non-empty, otherwise computes ColumnVectors based
-   *        on \c group_by and stores them in \c reuse_group_by_vectors.
-   **/
-  void aggregateGroupByPartitioned(
-      const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-      const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const TupleIdSequence *filter,
-      const std::size_t num_partitions,
-      ColumnVectorsValueAccessor *temp_result,
-      std::vector<attribute_id> *argument_ids,
-      std::vector<attribute_id> *key_ids,
-      std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
-
-  /**
-   * @brief Inserts the GROUP BY expressions and aggregation arguments together
-   *        as keys into the distinctify hash table.
-   *
-   * This is the first step for DISTINCT aggregation. It populates the distinctify
-   * hash table so that arguments are distinctified within each GROUP BY group.
-   * Later, a second-round aggregation on the distinctify hash table will be
-   * performed to actually compute the aggregated result for each GROUP BY group.
-   *
-   * @param handle Aggregation handle to compute aggregates with.
-   * @param arguments The arguments to the aggregation function as Scalars.
-   * @param arguments_as_attributes If non-NULL, indicates a valid attribute_id
-   *        for each of the elements in arguments, and is used to elide a copy.
-   *        Has no effect if NULL, or if VECTOR_COPY_ELISION_LEVEL is NONE.
-   * @param group_by The list of GROUP BY attributes/expressions.
-   * @param filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   * @param distinctify_hash_table Hash table to store the arguments and GROUP
-   *        BY expressions together as hash table key and a bool constant \c true
-   *        as hash table value. (So the hash table actually serves as a hash set.)
-   * @param reuse_group_by_vectors This parameter is used to store and reuse
-   *        GROUP BY attribute vectors pre-computed in an earlier invocation of
-   *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
-   *        for ease of use. Current invocation of aggregateGroupBy() will reuse
-   *        ColumnVectors if non-empty, otherwise computes ColumnVectors based
-   *        on \c group_by and stores them in \c reuse_group_by_vectors.
-   */
-  void aggregateDistinct(const AggregationHandle &handle,
-                         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 TupleIdSequence *filter,
-                         AggregationStateHashTableBase *distinctify_hash_table,
-                         std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
-
-  /**
    * @brief Perform an UPDATE query over the tuples in this StorageBlock.
    * @warning In some edge cases, calling this method may cause IndexSubBlocks
    *          in this block to become inconsistent (the TupleStorageSubBlock
@@ -702,18 +547,6 @@ class StorageBlock : public StorageBlockBase {
       const tuple_id tuple,
       const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments) const;
 
-  AggregationState* aggregateHelperColumnVector(
-      const AggregationHandle &handle,
-      const std::vector<std::unique_ptr<const Scalar>> &arguments,
-      const TupleIdSequence *matches) const;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* aggregateHelperValueAccessor(
-      const AggregationHandle &handle,
-      const std::vector<attribute_id> &argument_ids,
-      const TupleIdSequence *matches) const;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
   // Sort the tuples in storage block based on `sort_attribute'. If
   // `use_input_sequence' is set, we assume a pre-existing order of tuple-id
   // sequence specified by `sorted_sequence' and use stable sort to maintain

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/ValueAccessorMultiplexer.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessorMultiplexer.hpp b/storage/ValueAccessorMultiplexer.hpp
new file mode 100644
index 0000000..fe2fa8e
--- /dev/null
+++ b/storage/ValueAccessorMultiplexer.hpp
@@ -0,0 +1,145 @@
+/**
+ * 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_STORAGE_VALUE_ACCESSOR_MULTIPLEXER_HPP_
+#define QUICKSTEP_STORAGE_VALUE_ACCESSOR_MULTIPLEXER_HPP_
+
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class ValueAccessor;
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+enum class ValueAccessorSource {
+  kBase = 0,
+  kDerived,
+  kInvalid
+};
+
+/**
+ * @brief A data structure for representing attribute ids referring multiple
+ *        ValueAccessors.
+ */
+struct MultiSourceAttributeId {
+  MultiSourceAttributeId(const ValueAccessorSource in_source,
+                         const attribute_id in_attr_id)
+      : source(in_source),
+        attr_id(in_attr_id) {}
+
+  MultiSourceAttributeId(const MultiSourceAttributeId &other)
+      : source(other.source),
+        attr_id(other.attr_id) {}
+
+  const ValueAccessorSource source;
+  const attribute_id attr_id;
+};
+
+/**
+ * @brief A class that encapsulates multiple ValueAccessors and provides helper
+ *        methods for accessing the ValueAccessors with MultiSourceAttributeId.
+ *
+ * This class is in its very initial form that serves a small set of essential
+ * functionalities for the purpose of aggregation copy elision. That is, given a
+ * storage block to be aggregated on, we may have aggregations on a storage
+ * attribute (e.g. SUM(x)) or on a non-trivial expression (e.g. SUM(x * y)).
+ * For the former case, copy elision is applicable that the attribute gets accessed
+ * directly from the storage block. In the later case, we have to create a
+ * temporary data structure (i.e. ColumnVectorsValueAccessor) that stores the
+ * intermediate results. Thus, we refer to the ValueAccessor created directly
+ * from the storage block as the BASE accessor and the intermediate result
+ * ColumnVectorsValueAccessor as the DERIVED accessor. And we utilize this class
+ * (ValueAccessorMultiplexer) to pass both accessors around to enable copy elision.
+ *
+ * This class (together with ValueAccessorSource and MultiSourceAttributeId)
+ * may be rewritten or exteneded later to more generally support copy elisions
+ * in various scenarios.
+ */
+class ValueAccessorMultiplexer {
+ public:
+  /**
+   * @brief Constructor for base accessor only.
+   *
+   * @param base_accessor The base accessor.
+   */
+  explicit ValueAccessorMultiplexer(ValueAccessor *base_accessor)
+      : base_accessor_(base_accessor),
+        derived_accessor_(nullptr) {}
+
+  /**
+   * @brief Constructor.
+   *
+   * @param base_accessor The base accessor.
+   * @param derived_accessor The derived accessor.
+   */
+  ValueAccessorMultiplexer(ValueAccessor *base_accessor,
+                           ValueAccessor *derived_accessor)
+      : base_accessor_(base_accessor),
+        derived_accessor_(derived_accessor) {}
+
+  /**
+   * @return The base accessor.
+   */
+  inline ValueAccessor* getBaseAccessor() const {
+    return base_accessor_;
+  }
+
+  /**
+   * @return The derived accessor.
+   */
+  inline ValueAccessor* getDerivedAccessor() const {
+    return derived_accessor_;
+  }
+
+  /**
+   * @brief Get the value accessor that corresponds to the specified source.
+   *
+   * @param source The value accessor source.
+   * @return The value accessor that corresponds to \p source.
+   */
+  inline ValueAccessor* getValueAccessorBySource(
+      const ValueAccessorSource &source) const {
+    switch (source) {
+      case ValueAccessorSource::kBase:
+        return base_accessor_;
+      case ValueAccessorSource::kDerived:
+        return derived_accessor_;
+      default:
+        return nullptr;
+    }
+  }
+
+ private:
+  ValueAccessor *base_accessor_;
+  ValueAccessor *derived_accessor_;
+
+  DISALLOW_COPY_AND_ASSIGN(ValueAccessorMultiplexer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_VALUE_ACCESSOR_MULTIPLEXER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/utility/BarrieredReadWriteConcurrentBitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BarrieredReadWriteConcurrentBitVector.hpp b/utility/BarrieredReadWriteConcurrentBitVector.hpp
new file mode 100644
index 0000000..0086c7f
--- /dev/null
+++ b/utility/BarrieredReadWriteConcurrentBitVector.hpp
@@ -0,0 +1,282 @@
+/**
+ * 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_UTILITY_BARRIERED_READ_WRITE_CONCURRENT_BIT_VECTOR_HPP_
+#define QUICKSTEP_UTILITY_BARRIERED_READ_WRITE_CONCURRENT_BIT_VECTOR_HPP_
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <limits>
+
+#include "utility/BitManipulation.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A bit vector that supports concurrent read/write operations, with a
+ *        RESTRICTED CONCURRENCY LEVEL that the read operations and the write
+ *        operations must be isolated with a (mostly implicit) barrier.
+ * 
+ * In other words, when using this bit vector, the read operations and write
+ * operations must be grouped into phases. Within a phase there can be either
+ * concurrent read operations or concurrent write operations, but not both (or
+ * the bit vector's behavior will be undefined).
+ **/
+class BarrieredReadWriteConcurrentBitVector {
+ public:
+  /**
+   * @brief Constructor for a bit vector with external storage.
+   *
+   * @param memory_location The location of memory to use for the bit vector.
+   * @param num_bits The length of the bit vector in bits.
+   * @param initialize If true, initialize all the bytes of the memory to 0.
+   */
+  BarrieredReadWriteConcurrentBitVector(void *memory_location,
+                                        const std::size_t num_bits,
+                                        const bool initialize)
+      : owned_(false),
+        num_bits_(num_bits),
+        data_array_(static_cast<DataType *>(memory_location)),
+        data_array_size_((num_bits >> kHigherOrderShift) + (num_bits & kLowerOrderMask ? 1 : 0)) {
+    DCHECK_GT(num_bits, 0u);
+    DCHECK(data_array_ != nullptr);
+
+    if (initialize) {
+      clear();
+    }
+  }
+
+  /**
+   * @brief Constructor for a bit vector which owns its own storage.
+   *
+   * @param num_bits The length of the bit vector in bits.
+   **/
+  explicit BarrieredReadWriteConcurrentBitVector(const std::size_t num_bits)
+      : owned_(true),
+        num_bits_(num_bits),
+        data_array_(static_cast<DataType *>(std::malloc(BytesNeeded(num_bits)))),
+        data_array_size_((num_bits >> kHigherOrderShift) + (num_bits & kLowerOrderMask ? 1 : 0)) {
+    DCHECK_GT(num_bits, 0u);
+    clear();
+  }
+
+  /**
+   * @brief Destructor. Frees any owned memory.
+   */
+  ~BarrieredReadWriteConcurrentBitVector() {
+    if (owned_) {
+      std::free(data_array_);
+    }
+  }
+
+  /**
+   * @brief Calculate the number of bytes needed to store a bit vector of the
+   *        given number of bits.
+   *
+   * @param num_bits The desired length of a BitVector in bits.
+   * @return The number of bytes needed for the BitVector.
+   **/
+  inline static std::size_t BytesNeeded(const std::size_t num_bits) {
+    if (num_bits & kLowerOrderMask) {
+      return ((num_bits >> kHigherOrderShift) + 1) * kDataSize;
+    } else {
+      return (num_bits >> kHigherOrderShift) * kDataSize;
+    }
+  }
+
+  /**
+   * @return The length of this bit vector in bits.
+   **/
+  inline std::size_t size() const {
+    return num_bits_;
+  }
+
+  /**
+   * @brief Clear this bit vector, setting all bits to zero.
+   **/
+  inline void clear() {
+    std::memset(data_array_, 0, BytesNeeded(num_bits_));
+  }
+
+  /**
+   * @brief Get the value of a single bit.
+   *
+   * @param bit_num The position of the desired bit.
+   * @return The value of the bit at bit_num.
+   **/
+  inline bool getBit(const std::size_t bit_num) const {
+    const std::size_t data_value =
+        data_array_[bit_num >> kHigherOrderShift].load(std::memory_order_relaxed);
+    return (data_value << (bit_num & kLowerOrderMask)) & kTopBit;
+  }
+
+  /**
+   * @brief Set the value of a single bit.
+   *
+   * @param bit_num The position of the desired bit.
+   * @param value The new value to set for the bit at bit_num.
+   **/
+  inline void setBit(const std::size_t bit_num) const {
+    data_array_[bit_num >> kHigherOrderShift].fetch_or(
+        kTopBit >> (bit_num & kLowerOrderMask), std::memory_order_relaxed);
+  }
+
+  /**
+   * @brief Find the first 1-bit AT or AFTER the specified position in this bit
+   *        vector.
+   *
+   * @param position The first bit to search for a one.
+   * @return The position of the first one AT or AFTER \p position in this bit
+   *         vector.
+   **/
+  inline std::size_t firstOne(std::size_t position = 0) const {
+    DCHECK_LT(position, num_bits_);
+
+    const std::size_t position_index = position >> kHigherOrderShift;
+    const std::size_t data_value =
+        data_array_[position_index].load(std::memory_order_relaxed)
+            & (std::numeric_limits<std::size_t>::max() >> (position & kLowerOrderMask));
+    if (data_value) {
+      return (position & ~kLowerOrderMask) | leading_zero_count<std::size_t>(data_value);
+    }
+
+    for (std::size_t array_idx = position_index + 1;
+         array_idx < data_array_size_;
+         ++array_idx) {
+      const std::size_t data_value =
+          data_array_[array_idx].load(std::memory_order_relaxed);
+      if (data_value) {
+        return (array_idx << kHigherOrderShift) | leading_zero_count<std::size_t>(data_value);
+      }
+    }
+
+    return num_bits_;
+  }
+
+  /**
+   * @brief Find the first 1-bit AFTER the specified position in this bit vector.
+   *
+   * @param position The first bit to search for a one.
+   * @return The position of the first one AFTER \p position in this bit vector.
+   **/
+  inline std::size_t nextOne(const std::size_t position) const {
+    const std::size_t search_pos = position + 1;
+    return search_pos >= num_bits_ ? num_bits_ : firstOne(search_pos);
+  }
+
+  /**
+   * @brief Count the total number of 1-bits in this bit vector.
+   *
+   * @return The number of ones in this bit vector.
+   **/
+  inline std::size_t onesCount() const {
+    std::size_t count = 0;
+    for (std::size_t array_idx = 0;
+         array_idx < data_array_size_;
+         ++array_idx) {
+      count += population_count<std::size_t>(
+          data_array_[array_idx].load(std::memory_order_relaxed));
+    }
+    return count;
+  }
+
+  /**
+   * @brief Count the total number of 1-bits in this bit vector within the
+   *        specified range (start point INCLUSIVE but end point EXCLUSIVE).
+   *
+   * @param The start position of the range.
+   * @param The end position of the range.
+   *
+   * @return The number of ones within the range, INCLUDING the 1-bit at
+   *         \p start_position, but EXCLUDING the 1-bit at \p end_position.
+   **/
+  inline std::size_t onesCountInRange(const std::size_t start_position,
+                                      const std::size_t end_position) const {
+    DCHECK_LE(start_position, end_position);
+    DCHECK_LT(start_position, num_bits_);
+    DCHECK_LE(end_position, num_bits_);
+
+    const std::size_t start_index = start_position >> kHigherOrderShift;
+    const std::size_t end_index = end_position >> kHigherOrderShift;
+    if (start_index == end_index) {
+      const std::size_t data_value =
+          data_array_[start_index].load(std::memory_order_relaxed)
+              & (std::numeric_limits<std::size_t>::max() >> (start_position & kLowerOrderMask))
+              &  ~(std::numeric_limits<std::size_t>::max() >> (end_position & kLowerOrderMask));
+      return population_count<std::size_t>(data_value);
+    } else {
+      const std::size_t first_data =
+          data_array_[start_index].load(std::memory_order_relaxed)
+              & (std::numeric_limits<std::size_t>::max() >> (start_position & kLowerOrderMask));
+      std::size_t count = population_count<std::size_t>(first_data);
+
+      for (std::size_t array_idx = start_index + 1;
+           array_idx < end_index;
+           ++array_idx) {
+        count += population_count<std::size_t>(
+            data_array_[array_idx].load(std::memory_order_relaxed));
+      }
+
+      const std::size_t last_offset = end_position & kLowerOrderMask;
+      if (last_offset != 0) {
+        const std::size_t last_data =
+            data_array_[end_index].load(std::memory_order_relaxed)
+                &  ~(std::numeric_limits<std::size_t>::max() >> last_offset);
+        count += population_count<std::size_t>(last_data);
+      }
+
+      return count;
+    }
+  }
+
+ private:
+  typedef std::atomic<std::size_t> DataType;
+  static constexpr std::size_t kDataSize = sizeof(DataType);
+
+  // This works as long as the bit-width of size_t is power of 2:
+  static constexpr std::size_t kLowerOrderMask = (sizeof(std::size_t) << 3) - 1;
+  // This works for 32-bit or 64-bit size_t:
+  static constexpr std::size_t kHigherOrderShift = sizeof(std::size_t) == 4 ? 5 : 6;
+
+  static constexpr std::size_t kOne = static_cast<std::size_t>(1);
+  static constexpr std::size_t kTopBit = kOne << kLowerOrderMask;
+
+  const bool owned_;
+  const std::size_t num_bits_;
+  DataType *data_array_;
+  const std::size_t data_array_size_;
+
+  DISALLOW_COPY_AND_ASSIGN(BarrieredReadWriteConcurrentBitVector);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_BARRIERED_READ_WRITE_CONCURRENT_BIT_VECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index aeff388..ca04462 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -172,6 +172,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_CompositeHash ../empty_src.cpp CompositeHash.hpp)
+add_library(quickstep_utility_BarrieredReadWriteConcurrentBitVector
+            ../empty_src.cpp
+            BarrieredReadWriteConcurrentBitVector.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
@@ -238,6 +241,9 @@ target_link_libraries(quickstep_utility_CompositeHash
                       quickstep_types_TypedValue
                       quickstep_utility_HashPair
                       glog)
+target_link_libraries(quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_BitManipulation
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
@@ -330,6 +336,7 @@ target_link_libraries(quickstep_utility_TreeStringSerializable
 add_library(quickstep_utility ../empty_src.cpp UtilityModule.hpp)
 target_link_libraries(quickstep_utility
                       quickstep_utility_Alignment
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
                       quickstep_utility_BitManipulation
                       quickstep_utility_BitVector
                       quickstep_utility_BloomFilter