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/01/29 06:41:11 UTC

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

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/c2743f72
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/c2743f72
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/c2743f72

Branch: refs/heads/reorder-attrs
Commit: c2743f724441cb09e6e743bafd1bc23f1d88a092
Parents: 3210500
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jan 12 18:41:17 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Sun Jan 29 00:36:40 2017 -0600

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


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c2743f72/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/c2743f72/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..dc2763d 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' attributes 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 is based 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/c2743f72/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/c2743f72/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..5cc8d69
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.cpp
@@ -0,0 +1,205 @@
+/**
+ * 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,
+                                             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;
+  lock_ordering = 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));
+    }
+
+    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);
+  }
+  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.
+  std::unordered_map<E::ExprId, std::size_t> base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+      applyInternal(nodes.front()->children().front(), false);
+  P::TableReferencePtr base_table;
+  if (P::SomeTableReference::MatchesWithConditionalCast(base_node, &base_table)) {
+  }
+
+  const std::vector<E::AttributeReferencePtr> base_attrs =
+      nodes.front()->children().front()->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;
+    }
+  }
+
+  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/c2743f72/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..41248ca
--- /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,
+                                      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/c2743f72/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/c2743f72/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c1caaa3..8b68ccc 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -206,6 +206,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/c2743f72/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index fd3841f..b72096c 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -20,6 +20,7 @@
 #include "relational_operators/HashJoinOperator.hpp"
 
 #include <algorithm>
+#include <map>
 #include <memory>
 #include <unordered_map>
 #include <utility>
@@ -30,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"
@@ -63,6 +65,9 @@ namespace quickstep {
 
 namespace {
 
+typedef std::vector<std::pair<tuple_id, tuple_id>> VectorOfPairs;
+typedef std::pair<std::vector<tuple_id>, std::vector<tuple_id>> PairOfVectors;
+
 // 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
@@ -82,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, VectorOfPairs>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -93,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, VectorOfPairs> joined_tuples_;
 };
 
 // Another collector using an unordered_map keyed on inner block just like above,
@@ -106,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, PairOfVectors>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -466,16 +470,24 @@ void HashInnerJoinWorkOrder::execute() {
         base_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
   }
 
-  PairsOfVectorsJoinedTuplesCollector collector;
+  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.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);
@@ -484,14 +496,12 @@ 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>>>
+  for (std::pair<const block_id, VectorOfPairs>
            &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());
-    const std::vector<tuple_id> &build_tids = build_block_entry.second.first;
-    const std::vector<tuple_id> &probe_tids = build_block_entry.second.second;
 
     // Evaluate '*residual_predicate_', if any.
     //
@@ -504,123 +514,160 @@ 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;
-      for (std::size_t i = 0; i < build_tids.size(); ++i) {
+      VectorOfPairs 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,
-                                                        build_tids[i],
+                                                        hash_match.first,
                                                         *probe_accessor,
                                                         probe_relation_id,
-                                                        probe_tids[i])) {
-          filtered_matches.first.push_back(build_tids[i]);
-          filtered_matches.second.push_back(probe_tids[i]);
+                                                        hash_match.second)) {
+          filtered_matches.emplace_back(hash_match);
         }
       }
 
       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.
+    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) {
+  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();
+
+  // 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::size_t build_index = 0, probe_index = 1, temp_index = 2;
+  for (std::size_t i = 0; i < 3; ++i) {
+    accessor_attribute_map.emplace_back(
+        nullptr /* place holder */,
+        std::vector<attribute_id>(selection_.size(), kInvalidCatalogId));
+  }
+
+  std::vector<const Scalar *> non_trivial_expressions;
+  attribute_id dest_attr = 0;
+
+  for (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[temp_index].second[dest_attr] = non_trivial_expressions.size();
+      non_trivial_expressions.emplace_back(scalar.get());
+    } else {
+      auto scalar_attr = static_cast<const ScalarAttribute *>(scalar.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;
+      } else {
+        accessor_attribute_map[probe_index].second[dest_attr] = attr_id;
+      }
+    }
+    ++dest_attr;
+  }
+
+  std::vector<tuple_id> build_tids;
+  std::vector<tuple_id> probe_tids;
+  for (std::pair<const block_id, VectorOfPairs>
+           &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) {
+      VectorOfPairs 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);
+    }
+
     // 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;
-
-    // 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)));
+    if (!non_trivial_expressions.empty()) {
+      for (const Scalar *scalar : non_trivial_expressions) {
+        temp_result.addColumn(scalar->getAllValuesForJoin(build_relation_id,
+                                                          build_accessor.get(),
+                                                          probe_relation_id,
+                                                          probe_accessor,
+                                                          build_block_entry.second));
+      }
     }
 
-    attribute_id dest_attr = 0;
-    std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
-
-    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();
-
-        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).
-          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]));
-          }
-        }
-        temp_result.addColumn(
-            selection_cit
-                ->getAllValuesForJoin(build_relation_id, build_accessor.get(),
-                                      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;
-        } else {
-          accessor_attribute_map[probe_index].second[dest_attr] = attr_id;
-        }
-      }
-      ++dest_attr;
+    build_tids.clear();
+    probe_tids.clear();
+    for (const auto &pair : build_block_entry.second) {
+      build_tids.emplace_back(pair.first);
+      probe_tids.emplace_back(pair.second);
     }
 
+    // We now create ordered value accessors for both build and probe side,
+    // using the joined tuple TIDs.
+    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[build_index].first = ordered_build_accessor.get();
+    accessor_attribute_map[probe_index].first = ordered_probe_accessor.get();
+    accessor_attribute_map[temp_index].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/c2743f72/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/c2743f72/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/c2743f72/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/c2743f72/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();
   }