[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-31 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/incubator-quickstep/pull/171


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98576390
  
--- Diff: storage/ValueAccessor.hpp ---
@@ -305,6 +305,21 @@ class ValueAccessor {
   const TupleIdSequence _sequence) = 0;
 
   /**
+   * @brief Create a new OrderedTupleIdSequenceAdapterValueAccessor that 
wraps
+   *this ValueAccessor.
--- End diff --

I understand `OrderedTupleIdSequence`, but was wondering what does `Shared` 
mean here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98546082
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 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 base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+  applyInternal(nodes.front()->children().front(), false);
+  P::TableReferencePtr base_table;
+  if (P::SomeTableReference::MatchesWithConditionalCast(base_node, 
_table)) {
+  }
+
+  const std::vector 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  : 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 = [, , ](const E::NamedExpressionPtr 
,
+   const E::NamedExpressionPtr 
) -> bool {
+const E::ExprId lhs_id = lhs->id();
+const E::ExprId rhs_id = rhs->id();
+
+// Sort the attributes first by GEN location.
+const 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98545487
  
--- Diff: query_optimizer/PhysicalGenerator.cpp ---
@@ -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.
--- End diff --

Currently this cannot be checked because the temporary relations' layouts 
are generated in `ExecutionGenerator` when they get created.

In the future we will reason about the layouts during query optimization 
and add corresponding data structures.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98543218
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -504,123 +616,61 @@ 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 
filtered_matches;
+  PairOfVectors filtered_matches;
+
   for (std::size_t i = 0; i < build_tids.size(); ++i) {
 if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
 build_relation_id,
 build_tids[i],
 *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 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.
+  VectorOfPairs zipped_joined_tuple_ids;
+  zipped_joined_tuple_ids.reserve(build_tids.size());
+  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> 
accessor_attribute_map;
-const std::vector accessors{
-ordered_build_accessor.get(), ordered_probe_accessor.get(), 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98544852
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -484,7 +565,38 @@ void HashInnerJoinWorkOrder::execute() {
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
 
-  for (std::pair, 
std::vector>>
+  // Create a map of ValueAccessors and what attributes we want to pick 
from them.
+  std::vector> 
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) {
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98544534
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -484,7 +565,38 @@ void HashInnerJoinWorkOrder::execute() {
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
 
-  for (std::pair, 
std::vector>>
+  // Create a map of ValueAccessors and what attributes we want to pick 
from them.
+  std::vector> 
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 */,
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98544439
  
--- Diff: storage/ValueAccessor.hpp ---
@@ -305,6 +305,21 @@ class ValueAccessor {
   const TupleIdSequence _sequence) = 0;
 
   /**
+   * @brief Create a new OrderedTupleIdSequenceAdapterValueAccessor that 
wraps
+   *this ValueAccessor.
--- End diff --

`OrderedTupleIdSequenceAdapterValueAccessor` has its header comments at 
`ValueAccessor.hpp` line 548.
`OrderedTupleIdSequence` has its parameter comment below.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98543320
  
--- Diff: 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 
+
+#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 {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  ReorderColumns() {}
+
+  ~ReorderColumns() override {}
+
+  std::string getName() const override {
+return "ReorderColumns";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr ) override;
+
+ private:
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr ,
+  bool lock_ordering);
+
+  // Whether the physical node can
+  inline static bool IsTransformable(const physical::PhysicalPtr );
+
+  DISALLOW_COPY_AND_ASSIGN(ReorderColumns);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_ */
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98543122
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -504,123 +616,61 @@ 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 
filtered_matches;
+  PairOfVectors filtered_matches;
+
   for (std::size_t i = 0; i < build_tids.size(); ++i) {
 if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
 build_relation_id,
 build_tids[i],
 *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 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.
+  VectorOfPairs zipped_joined_tuple_ids;
+  zipped_joined_tuple_ids.reserve(build_tids.size());
--- End diff --

`reserve` is for pre-allocating memory and avoids buffer reallocation 
during the loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98542466
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -63,6 +65,9 @@ namespace quickstep {
 
 namespace {
 
+typedef std::vector> VectorOfPairs;
+typedef std::pair 
PairOfVectors;
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98541504
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 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 base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+  applyInternal(nodes.front()->children().front(), false);
+  P::TableReferencePtr base_table;
+  if (P::SomeTableReference::MatchesWithConditionalCast(base_node, 
_table)) {
+  }
+
+  const std::vector 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  : 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 = [, , ](const E::NamedExpressionPtr 
,
+   const E::NamedExpressionPtr 
) -> bool {
+const E::ExprId lhs_id = lhs->id();
+const E::ExprId rhs_id = rhs->id();
+
+// Sort the attributes first by GEN location.
+const 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98541098
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 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 base, gen, kill;
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98541113
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 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 base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+  applyInternal(nodes.front()->children().front(), false);
+  P::TableReferencePtr base_table;
+  if (P::SomeTableReference::MatchesWithConditionalCast(base_node, 
_table)) {
+  }
--- End diff --

Removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98538859
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 nodes;
+  for (P::PhysicalPtr node = input; IsTransformable(node); node = 
node->children().front()) {
+nodes.emplace_back(node);
+  }
+  std::reverse(nodes.begin(), nodes.end());
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-30 Thread jianqiao
Github user jianqiao commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98538359
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : input->children()) {
+  new_children.emplace_back(applyInternal(child, lock_ordering));
--- End diff --

Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98383705
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 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 base, gen, kill;
--- End diff --

Please add more comments about the usage for both `gen` and `kill`, i.e., 
why calling `kill`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98385758
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -20,6 +20,7 @@
 #include "relational_operators/HashJoinOperator.hpp"
 
 #include 
+#include 
--- End diff --

Could you please point out the location of using `map` in this file? I 
could not find it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98384243
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -63,6 +65,9 @@ namespace quickstep {
 
 namespace {
 
+typedef std::vector> VectorOfPairs;
+typedef std::pair 
PairOfVectors;
--- End diff --

Suggest to rename to `VectorOfTupleIdPair` and `PairOfTupleIdVector`, 
respectively.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98385208
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : input->children()) {
+  new_children.emplace_back(applyInternal(child, lock_ordering));
--- End diff --

Implementation tricks: I think we could still mark the `lock_ordering` 
argument `const`, while keeping the same semantics by 
`new_children.emplace_back(applyInternal(child, lock_ordering && 
is_not_transformable));`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98384533
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -484,7 +565,38 @@ void HashInnerJoinWorkOrder::execute() {
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
 
-  for (std::pair, 
std::vector>>
+  // Create a map of ValueAccessors and what attributes we want to pick 
from them.
+  std::vector> 
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) {
--- End diff --

Please replace the magic number `3` with some constant like `kNumIndexes`.

Also please rename above three `indexes` to `kBuildIndex`, `kProbeIndex`, 
and `kTempIndex`, respectively.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98384815
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -504,123 +616,61 @@ 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 
filtered_matches;
+  PairOfVectors filtered_matches;
+
   for (std::size_t i = 0; i < build_tids.size(); ++i) {
 if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
 build_relation_id,
 build_tids[i],
 *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 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.
+  VectorOfPairs zipped_joined_tuple_ids;
+  zipped_joined_tuple_ids.reserve(build_tids.size());
--- End diff --

Please remove this line, as its effect is duplicated after executing the 
following `looped emplace_back`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98385945
  
--- Diff: relational_operators/HashJoinOperator.cpp ---
@@ -504,123 +616,61 @@ 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 
filtered_matches;
+  PairOfVectors filtered_matches;
+
   for (std::size_t i = 0; i < build_tids.size(); ++i) {
 if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
 build_relation_id,
 build_tids[i],
 *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 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.
+  VectorOfPairs zipped_joined_tuple_ids;
+  zipped_joined_tuple_ids.reserve(build_tids.size());
+  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> 
accessor_attribute_map;
-const std::vector accessors{
-ordered_build_accessor.get(), ordered_probe_accessor.get(), 
_result};
 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98385665
  
--- Diff: 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 
+
+#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 {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  ReorderColumns() {}
+
+  ~ReorderColumns() override {}
+
+  std::string getName() const override {
+return "ReorderColumns";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr ) override;
+
+ private:
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr ,
+  bool lock_ordering);
+
+  // Whether the physical node can
+  inline static bool IsTransformable(const physical::PhysicalPtr );
+
+  DISALLOW_COPY_AND_ASSIGN(ReorderColumns);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_ */
--- End diff --

Code style: `#endif  // 
QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98383321
  
--- Diff: 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 
+#include 
+#include 
+#include 
+#include 
+
+#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 ) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr ,
+ 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 new_children;
+for (const P::PhysicalPtr  : 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 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 base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+  applyInternal(nodes.front()->children().front(), false);
+  P::TableReferencePtr base_table;
+  if (P::SomeTableReference::MatchesWithConditionalCast(base_node, 
_table)) {
+  }
+
+  const std::vector 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  : 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 = [, , ](const E::NamedExpressionPtr 
,
+   const E::NamedExpressionPtr 
) -> bool {
+const E::ExprId lhs_id = lhs->id();
+const E::ExprId rhs_id = rhs->id();
+
+// Sort the attributes first by GEN location.
+const 

[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread zuyu
Github user zuyu commented on a diff in the pull request:

https://github.com/apache/incubator-quickstep/pull/171#discussion_r98383388
  
--- Diff: query_optimizer/PhysicalGenerator.cpp ---
@@ -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.
--- End diff --

I suggest to check this assumption in `ReorderColumns::IsTransformable`, so 
we could remove this comment here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread jianqiao
GitHub user jianqiao reopened a pull request:

https://github.com/apache/incubator-quickstep/pull/171

QUICKSTEP-68 Reorder intermediate relations'  attributes to improve copy 
performance.

Currently, all the intermediate relations (i.e. temporary relations created 
during query execution) have `SPLIT_ROW_STORE` layouts. One existing 
optimization that improves copy performance for `SPLIT_ROW_STORE` layout is 
that, if two (or more) attributes are consecutive in both the source relation 
and the destination relation, then we can memory-copy the attributes' byte 
stream with just one `std::memcpy` call.

This PR is a complementary improvement that it adjusts the ordering of the 
attributes in the physical plan to maximize the opportunity of copying 
consecutive attributes – i.e. to minimize the number of `std::memcpy` calls.

This PR uses a simple greedy algorithm to reorder the attributes, which 
works well with SSB/TPCH workloads. The algorithm may be further improved later.

The feature can be turned on/off (default is **on**) with the 
`-reorder_columns` flag.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/apache/incubator-quickstep reorder-attrs

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/incubator-quickstep/pull/171.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #171


commit 62b1742ef29bc8dea19d08fac6e0e04f45dc3f8a
Author: Jianqiao Zhu 
Date:   2017-01-13T00:41:17Z

Reorder output attribute order to improve copy performance.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread jianqiao
Github user jianqiao closed the pull request at:

https://github.com/apache/incubator-quickstep/pull/171


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-quickstep pull request #171: QUICKSTEP-68 Reorder intermediate rel...

2017-01-29 Thread jianqiao
GitHub user jianqiao opened a pull request:

https://github.com/apache/incubator-quickstep/pull/171

QUICKSTEP-68 Reorder intermediate relations'  attributes to improve copy 
performance.

Currently, all the intermediate relations (i.e. temporary relations created 
during query execution) have `SPLIT_ROW_STORE` layouts. One existing 
optimization that improves copy performance for `SPLIT_ROW_STORE` layout is 
that, if two (or more) attributes are consecutive in both the source relation 
and the destination relation, then we can memory-copy the attributes' byte 
stream with just one `std::memcpy` call.

This PR is a complementary improvement that it adjusts the ordering of the 
attributes in the physical plan to maximize the opportunity of copying 
consecutive attributes – i.e. to minimize the number of `std::memcpy` calls.

This PR uses a simple greedy algorithm to reorder the attributes, which 
works well with SSB/TPCH workloads. The algorithm may be further improved later.

The feature can be turned on/off (default is **on**) with the 
`-reorder_columns` flag.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/apache/incubator-quickstep reorder-attrs

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/incubator-quickstep/pull/171.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #171


commit 2c1c0905b18e7ecd018a3a05b9db03fdf987720b
Author: Jianqiao Zhu 
Date:   2017-01-13T00:41:17Z

Reorder output attribute order to improve copy performance.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---